mirror of
https://github.com/dragonflydb/dragonfly.git
synced 2025-05-11 10:25:47 +02:00
feat: DispatchTracker to replace everything (#2179)
* feat: DispatchTracker Use a DispatchTracker to track ongoing dispatches for commands that change global state --------- Signed-off-by: Vladislav Oleshko <vlad@dragonflydb.io>
This commit is contained in:
parent
f39eac5bca
commit
11ef6623dc
17 changed files with 165 additions and 143 deletions
|
@ -80,6 +80,7 @@ class ConnectionContext {
|
|||
bool async_dispatch : 1; // whether this connection is amid an async dispatch
|
||||
bool sync_dispatch : 1; // whether this connection is amid a sync dispatch
|
||||
bool journal_emulated : 1; // whether it is used to dispatch journal commands
|
||||
bool paused : 1; // whether this connection is paused due to CLIENT PAUSE
|
||||
|
||||
// How many async subscription sources are active: monitor and/or pubsub - at most 2.
|
||||
uint8_t subscriptions;
|
||||
|
|
|
@ -272,6 +272,7 @@ void Connection::DispatchOperations::operator()(Connection::PipelineMessage& msg
|
|||
|
||||
self->service_->DispatchCommand(CmdArgList{msg.args.data(), msg.args.size()}, self->cc_.get());
|
||||
self->last_interaction_ = time(nullptr);
|
||||
self->skip_next_squashing_ = false;
|
||||
}
|
||||
|
||||
void Connection::DispatchOperations::operator()(const MigrationRequestMessage& msg) {
|
||||
|
@ -279,6 +280,8 @@ void Connection::DispatchOperations::operator()(const MigrationRequestMessage& m
|
|||
}
|
||||
|
||||
void Connection::DispatchOperations::operator()(CheckpointMessage msg) {
|
||||
VLOG(1) << "Decremented checkpoint at " << self->DebugInfo();
|
||||
|
||||
msg.bc.Dec();
|
||||
}
|
||||
|
||||
|
@ -881,7 +884,7 @@ auto Connection::IoLoop(util::FiberSocketBase* peer, SinkReplyBuilder* orig_buil
|
|||
|
||||
phase_ = PROCESS;
|
||||
bool is_iobuf_full = io_buf_.AppendLen() == 0;
|
||||
service_->AwaitOnPauseDispatch();
|
||||
|
||||
if (redis_parser_) {
|
||||
parse_status = ParseRedis(orig_builder);
|
||||
} else {
|
||||
|
@ -971,24 +974,19 @@ void Connection::SquashPipeline(facade::SinkReplyBuilder* builder) {
|
|||
DCHECK_EQ(dispatch_q_.size(), pending_pipeline_cmd_cnt_);
|
||||
|
||||
vector<CmdArgList> squash_cmds;
|
||||
vector<PipelineMessagePtr> squash_msgs;
|
||||
|
||||
squash_cmds.reserve(dispatch_q_.size());
|
||||
squash_msgs.reserve(dispatch_q_.size());
|
||||
|
||||
while (!dispatch_q_.empty()) {
|
||||
auto& msg = dispatch_q_.front();
|
||||
for (auto& msg : dispatch_q_) {
|
||||
CHECK(holds_alternative<PipelineMessagePtr>(msg.handle))
|
||||
<< "Found " << msg.handle.index() << " on " << DebugInfo();
|
||||
<< msg.handle.index() << " on " << DebugInfo();
|
||||
|
||||
squash_msgs.push_back(std::move(std::get<PipelineMessagePtr>(msg.handle)));
|
||||
squash_cmds.push_back(absl::MakeSpan(squash_msgs.back()->args));
|
||||
dispatch_q_.pop_front();
|
||||
auto& pmsg = get<PipelineMessagePtr>(msg.handle);
|
||||
squash_cmds.push_back(absl::MakeSpan(pmsg->args));
|
||||
}
|
||||
|
||||
cc_->async_dispatch = true;
|
||||
|
||||
service_->DispatchManyCommands(absl::MakeSpan(squash_cmds), cc_.get());
|
||||
size_t dispatched = service_->DispatchManyCommands(absl::MakeSpan(squash_cmds), cc_.get());
|
||||
|
||||
if (pending_pipeline_cmd_cnt_ == squash_cmds.size()) { // Flush if no new commands appeared
|
||||
builder->FlushBatch();
|
||||
|
@ -997,8 +995,17 @@ void Connection::SquashPipeline(facade::SinkReplyBuilder* builder) {
|
|||
|
||||
cc_->async_dispatch = false;
|
||||
|
||||
for (auto& msg : squash_msgs)
|
||||
RecycleMessage(MessageHandle{std::move(msg)});
|
||||
auto it = dispatch_q_.begin();
|
||||
while (it->IsIntrusive()) // Skip all newly received intrusive messages
|
||||
++it;
|
||||
|
||||
for (auto rit = it; rit != it + dispatched; ++rit)
|
||||
RecycleMessage(std::move(*rit));
|
||||
|
||||
dispatch_q_.erase(it, it + dispatched);
|
||||
|
||||
// If interrupted due to pause, fall back to regular dispatch
|
||||
skip_next_squashing_ = dispatched != squash_cmds.size();
|
||||
}
|
||||
|
||||
void Connection::ClearPipelinedMessages() {
|
||||
|
@ -1020,6 +1027,7 @@ void Connection::ClearPipelinedMessages() {
|
|||
std::string Connection::DebugInfo() const {
|
||||
std::string info = "{";
|
||||
|
||||
absl::StrAppend(&info, "address=", uint64_t(this), ", ");
|
||||
absl::StrAppend(&info, "phase=", phase_, ", ");
|
||||
absl::StrAppend(&info, "dispatch(s/a)=", cc_->sync_dispatch, " ", cc_->async_dispatch, ", ");
|
||||
absl::StrAppend(&info, "closing=", cc_->conn_closing, ", ");
|
||||
|
@ -1079,7 +1087,7 @@ void Connection::DispatchFiber(util::FiberSocketBase* peer) {
|
|||
bool squashing_enabled = squashing_threshold > 0;
|
||||
bool threshold_reached = pending_pipeline_cmd_cnt_ > squashing_threshold;
|
||||
bool are_all_plain_cmds = pending_pipeline_cmd_cnt_ == dispatch_q_.size();
|
||||
if (squashing_enabled && threshold_reached && are_all_plain_cmds) {
|
||||
if (squashing_enabled && threshold_reached && are_all_plain_cmds && !skip_next_squashing_) {
|
||||
SquashPipeline(builder);
|
||||
} else {
|
||||
MessageHandle msg = move(dispatch_q_.front());
|
||||
|
@ -1197,10 +1205,15 @@ void Connection::SendAclUpdateAsync(AclUpdateMessage msg) {
|
|||
SendAsync({make_unique<AclUpdateMessage>(std::move(msg))});
|
||||
}
|
||||
|
||||
void Connection::SendCheckpoint(fb2::BlockingCounter bc) {
|
||||
void Connection::SendCheckpoint(fb2::BlockingCounter bc, bool ignore_paused) {
|
||||
if (!IsCurrentlyDispatching())
|
||||
return;
|
||||
|
||||
if (cc_->paused && !ignore_paused)
|
||||
return;
|
||||
|
||||
VLOG(1) << "Sent checkpoint to " << DebugInfo();
|
||||
|
||||
bc.Add(1);
|
||||
SendAsync({CheckpointMessage{bc}});
|
||||
}
|
||||
|
|
|
@ -155,8 +155,8 @@ class Connection : public util::Connection {
|
|||
void SendAclUpdateAsync(AclUpdateMessage msg);
|
||||
|
||||
// If any dispatch is currently in progress, increment counter and send checkpoint message to
|
||||
// decrement it once finished.
|
||||
void SendCheckpoint(util::fb2::BlockingCounter bc);
|
||||
// decrement it once finished. It ignore_paused is true, paused dispatches are ignored.
|
||||
void SendCheckpoint(util::fb2::BlockingCounter bc, bool ignore_paused = false);
|
||||
|
||||
// Must be called before sending pubsub messages to ensure the threads pipeline queue limit is not
|
||||
// reached. Blocks until free space is available. Controlled with `pipeline_queue_limit` flag.
|
||||
|
@ -348,6 +348,8 @@ class Connection : public util::Connection {
|
|||
bool migration_enabled_;
|
||||
util::fb2::ProactorBase* migration_request_ = nullptr;
|
||||
|
||||
bool skip_next_squashing_ = false; // Forcefully skip next squashing
|
||||
|
||||
// Pooled pipeline messages per-thread
|
||||
// Aggregated while handling pipelines, gradually released while handling regular commands.
|
||||
static thread_local std::vector<PipelineMessagePtr> pipeline_req_pool_;
|
||||
|
|
|
@ -8,6 +8,7 @@
|
|||
|
||||
#include <memory>
|
||||
|
||||
#include "absl/functional/bind_front.h"
|
||||
#include "facade/tls_error.h"
|
||||
|
||||
#ifdef DFLY_USE_SSL
|
||||
|
@ -234,32 +235,6 @@ void Listener::PreAcceptLoop(util::ProactorBase* pb) {
|
|||
per_thread_.resize(pool()->size());
|
||||
}
|
||||
|
||||
bool Listener::AwaitCurrentDispatches(absl::Duration timeout, util::Connection* issuer) {
|
||||
// Fill blocking counter with ongoing dispatches
|
||||
util::fb2::BlockingCounter bc{0};
|
||||
this->TraverseConnections([bc, issuer](unsigned thread_index, util::Connection* conn) {
|
||||
if (conn != issuer)
|
||||
static_cast<Connection*>(conn)->SendCheckpoint(bc);
|
||||
});
|
||||
|
||||
auto cancelled = make_shared<bool>(false);
|
||||
|
||||
// TODO: Add wait with timeout or polling to helio (including cancel flag)
|
||||
util::MakeFiber([bc, cancelled = weak_ptr{cancelled}, start = absl::Now(), timeout]() mutable {
|
||||
while (!cancelled.expired()) {
|
||||
if (absl::Now() - start > timeout) {
|
||||
VLOG(1) << "AwaitCurrentDispatches timed out";
|
||||
*cancelled.lock() = true; // same thread, no promotion race
|
||||
bc.Cancel();
|
||||
}
|
||||
ThisFiber::SleepFor(10ms);
|
||||
}
|
||||
}).Detach();
|
||||
|
||||
bc.Wait();
|
||||
return !*cancelled;
|
||||
}
|
||||
|
||||
bool Listener::IsPrivilegedInterface() const {
|
||||
return role_ == Role::PRIVILEGED;
|
||||
}
|
||||
|
@ -276,8 +251,10 @@ void Listener::PreShutdown() {
|
|||
// This shouldn't take a long time: All clients should reject incoming commands
|
||||
// at this stage since we're in SHUTDOWN mode.
|
||||
// If a command is running for too long we give up and proceed.
|
||||
const absl::Duration kDispatchShutdownTimeout = absl::Milliseconds(10);
|
||||
if (!AwaitCurrentDispatches(kDispatchShutdownTimeout, nullptr)) {
|
||||
DispatchTracker tracker{{this}};
|
||||
tracker.TrackAll();
|
||||
|
||||
if (!tracker.Wait(absl::Milliseconds(10))) {
|
||||
LOG(WARNING) << "Some commands are still being dispatched but didn't conclude in time. "
|
||||
"Proceeding in shutdown.";
|
||||
}
|
||||
|
@ -396,4 +373,30 @@ ProactorBase* Listener::PickConnectionProactor(util::FiberSocketBase* sock) {
|
|||
return pp->at(res_id);
|
||||
}
|
||||
|
||||
DispatchTracker::DispatchTracker(absl::Span<facade::Listener* const> listeners,
|
||||
facade::Connection* issuer, bool ignore_paused)
|
||||
: listeners_{listeners.begin(), listeners.end()},
|
||||
issuer_{issuer},
|
||||
ignore_paused_{ignore_paused} {
|
||||
}
|
||||
|
||||
void DispatchTracker::TrackOnThread() {
|
||||
for (auto* listener : listeners_)
|
||||
listener->TraverseConnectionsOnThread(absl::bind_front(&DispatchTracker::Handle, this));
|
||||
}
|
||||
|
||||
bool DispatchTracker::Wait(absl::Duration duration) {
|
||||
return bc_.WaitFor(absl::ToChronoMilliseconds(duration));
|
||||
}
|
||||
|
||||
void DispatchTracker::TrackAll() {
|
||||
for (auto* listener : listeners_)
|
||||
listener->TraverseConnections(absl::bind_front(&DispatchTracker::Handle, this));
|
||||
}
|
||||
|
||||
void DispatchTracker::Handle(unsigned thread_index, util::Connection* conn) {
|
||||
if (auto* fconn = static_cast<facade::Connection*>(conn); fconn != issuer_)
|
||||
fconn->SendCheckpoint(bc_, ignore_paused_);
|
||||
}
|
||||
|
||||
} // namespace facade
|
||||
|
|
|
@ -5,6 +5,7 @@
|
|||
#pragma once
|
||||
|
||||
#include <absl/base/internal/spinlock.h>
|
||||
#include <absl/time/time.h>
|
||||
|
||||
#include <memory>
|
||||
#include <system_error>
|
||||
|
@ -22,6 +23,7 @@ typedef struct ssl_ctx_st SSL_CTX;
|
|||
namespace facade {
|
||||
|
||||
class ServiceInterface;
|
||||
class Connection;
|
||||
|
||||
class Listener : public util::ListenerInterface {
|
||||
public:
|
||||
|
@ -79,4 +81,30 @@ class Listener : public util::ListenerInterface {
|
|||
SSL_CTX* ctx_ = nullptr;
|
||||
};
|
||||
|
||||
// Dispatch tracker allows tracking the dispatch state of connections and blocking until all
|
||||
// detected busy connections finished dispatching. Ignores issuer connection.
|
||||
//
|
||||
// Mostly used to detect when global state changes (takeover, pause, cluster config update) are
|
||||
// visible to all commands and no commands are still running according to the old state / config.
|
||||
class DispatchTracker {
|
||||
public:
|
||||
DispatchTracker(absl::Span<facade::Listener* const>, facade::Connection* issuer = nullptr,
|
||||
bool ignore_paused = false);
|
||||
|
||||
void TrackAll(); // Track busy connection on all threads
|
||||
void TrackOnThread(); // Track busy connections on current thread
|
||||
|
||||
// Wait until all tracked connections finished dispatching.
|
||||
// Returns true on success, false if timeout was reached.
|
||||
bool Wait(absl::Duration timeout);
|
||||
|
||||
private:
|
||||
void Handle(unsigned thread_index, util::Connection* conn);
|
||||
|
||||
std::vector<facade::Listener*> listeners_;
|
||||
facade::Connection* issuer_;
|
||||
util::fb2::BlockingCounter bc_{0};
|
||||
bool ignore_paused_;
|
||||
};
|
||||
|
||||
} // namespace facade
|
||||
|
|
|
@ -133,6 +133,7 @@ ConnectionContext::ConnectionContext(::io::Sink* stream, Connection* owner) : ow
|
|||
async_dispatch = false;
|
||||
sync_dispatch = false;
|
||||
journal_emulated = false;
|
||||
paused = false;
|
||||
|
||||
subscriptions = 0;
|
||||
}
|
||||
|
|
|
@ -27,9 +27,10 @@ class OkService : public ServiceInterface {
|
|||
(*cntx)->SendOk();
|
||||
}
|
||||
|
||||
void DispatchManyCommands(absl::Span<CmdArgList> args_lists, ConnectionContext* cntx) final {
|
||||
size_t DispatchManyCommands(absl::Span<CmdArgList> args_lists, ConnectionContext* cntx) final {
|
||||
for (auto args : args_lists)
|
||||
DispatchCommand(args, cntx);
|
||||
return args_lists.size();
|
||||
}
|
||||
|
||||
void DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
||||
|
@ -44,10 +45,6 @@ class OkService : public ServiceInterface {
|
|||
ConnectionStats* GetThreadLocalConnectionStats() final {
|
||||
return &tl_stats;
|
||||
}
|
||||
|
||||
void AwaitOnPauseDispatch() {
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
void RunEngine(ProactorPool* pool, AcceptServer* acceptor) {
|
||||
|
|
|
@ -27,7 +27,9 @@ class ServiceInterface {
|
|||
|
||||
virtual void DispatchCommand(CmdArgList args, ConnectionContext* cntx) = 0;
|
||||
|
||||
virtual void DispatchManyCommands(absl::Span<CmdArgList> args_list, ConnectionContext* cntx) = 0;
|
||||
// Returns number of processed commands
|
||||
virtual size_t DispatchManyCommands(absl::Span<CmdArgList> args_list,
|
||||
ConnectionContext* cntx) = 0;
|
||||
|
||||
virtual void DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
||||
ConnectionContext* cntx) = 0;
|
||||
|
@ -35,7 +37,6 @@ class ServiceInterface {
|
|||
virtual ConnectionContext* CreateContext(util::FiberSocketBase* peer, Connection* owner) = 0;
|
||||
|
||||
virtual ConnectionStats* GetThreadLocalConnectionStats() = 0;
|
||||
virtual void AwaitOnPauseDispatch() = 0;
|
||||
|
||||
virtual void ConfigureHttpHandlers(util::HttpListenerBase* base, bool is_privileged) {
|
||||
}
|
||||
|
|
|
@ -508,10 +508,18 @@ void ClusterFamily::DflyClusterConfig(CmdArgList args, ConnectionContext* cntx)
|
|||
before = tl_cluster_config->GetOwnedSlots();
|
||||
}
|
||||
|
||||
auto cb = [&](util::ProactorBase* pb) { tl_cluster_config = new_config; };
|
||||
DispatchTracker tracker{server_family_->GetListeners(), cntx->conn()};
|
||||
auto cb = [&tracker, &new_config](util::ProactorBase* pb) {
|
||||
tl_cluster_config = new_config;
|
||||
tracker.TrackOnThread();
|
||||
};
|
||||
server_family_->service().proactor_pool().AwaitFiberOnAll(std::move(cb));
|
||||
DCHECK(tl_cluster_config != nullptr);
|
||||
|
||||
if (!tracker.Wait(absl::Seconds(1))) {
|
||||
LOG(WARNING) << "Cluster config change timed out";
|
||||
}
|
||||
|
||||
SlotSet after = tl_cluster_config->GetOwnedSlots();
|
||||
if (ServerState::tlocal()->is_master) {
|
||||
auto deleted_slots = GetDeletedSlots(is_first_config, before, after);
|
||||
|
|
|
@ -427,7 +427,10 @@ void DflyCmd::TakeOver(CmdArgList args, ConnectionContext* cntx) {
|
|||
|
||||
// We need to await for all dispatches to finish: Otherwise a transaction might be scheduled
|
||||
// after this function exits but before the actual shutdown.
|
||||
if (!sf_->AwaitCurrentDispatches(timeout_dur, cntx->conn())) {
|
||||
facade::DispatchTracker tracker{sf_->GetListeners(), cntx->conn()};
|
||||
tracker.TrackAll();
|
||||
|
||||
if (!tracker.Wait(timeout_dur)) {
|
||||
LOG(WARNING) << "Couldn't wait for commands to finish dispatching. " << timeout_dur;
|
||||
status = OpStatus::TIMED_OUT;
|
||||
}
|
||||
|
|
|
@ -1056,12 +1056,21 @@ void Service::DispatchCommand(CmdArgList args, facade::ConnectionContext* cntx)
|
|||
ConnectionContext* dfly_cntx = static_cast<ConnectionContext*>(cntx);
|
||||
bool under_script = bool(dfly_cntx->conn_state.script_info);
|
||||
bool under_multi = dfly_cntx->conn_state.exec_info.IsRunning();
|
||||
bool dispatching_in_multi = under_script || under_multi;
|
||||
|
||||
if (VLOG_IS_ON(2) &&
|
||||
cntx->conn()) { // owner may not exists in case of this being called from replica context
|
||||
const char* lua = under_script ? "LUA " : "";
|
||||
LOG(INFO) << "Got (" << cntx->conn()->GetClientId() << "): " << lua << args
|
||||
<< " in dbid=" << dfly_cntx->conn_state.db_index;
|
||||
if (VLOG_IS_ON(2) && cntx->conn() /* no owner in replica context */) {
|
||||
LOG(INFO) << "Got (" << cntx->conn()->GetClientId() << "): " << (under_script ? "LUA " : "")
|
||||
<< args << " in dbid=" << dfly_cntx->conn_state.db_index;
|
||||
}
|
||||
|
||||
if (!dispatching_in_multi) { // Don't interrupt running multi commands
|
||||
bool is_write = (cid->opt_mask() & CO::WRITE);
|
||||
is_write |= cid->name() == "PUBLISH" || cid->name() == "EVAL" || cid->name() == "EVALSHA";
|
||||
is_write |= cid->name() == "EXEC" && dfly_cntx->conn_state.exec_info.is_write;
|
||||
|
||||
cntx->paused = true;
|
||||
etl.AwaitPauseState(is_write);
|
||||
cntx->paused = false;
|
||||
}
|
||||
|
||||
etl.RecordCmd();
|
||||
|
@ -1098,8 +1107,6 @@ void Service::DispatchCommand(CmdArgList args, facade::ConnectionContext* cntx)
|
|||
// Create command transaction
|
||||
intrusive_ptr<Transaction> dist_trans;
|
||||
|
||||
bool dispatching_in_multi = under_script || under_multi;
|
||||
|
||||
if (dispatching_in_multi) {
|
||||
DCHECK(dfly_cntx->transaction);
|
||||
if (cid->IsTransactional()) {
|
||||
|
@ -1183,14 +1190,6 @@ bool Service::InvokeCmd(const CommandId* cid, CmdArgList tail_args, ConnectionCo
|
|||
|
||||
ServerState& etl = *ServerState::tlocal();
|
||||
|
||||
string_view cmd_name(cid->name());
|
||||
bool is_write = (cid->opt_mask() & CO::WRITE) || cmd_name == "PUBLISH" || cmd_name == "EVAL" ||
|
||||
cmd_name == "EVALSHA";
|
||||
if (cmd_name == "EXEC" && cntx->conn_state.exec_info.is_write) {
|
||||
is_write = true;
|
||||
}
|
||||
etl.AwaitPauseState(is_write);
|
||||
|
||||
// We are not sending any admin command in the monitor, and we do not want to
|
||||
// do any processing if we don't have any waiting connections with monitor
|
||||
// enabled on them - see https://redis.io/commands/monitor/
|
||||
|
@ -1221,14 +1220,16 @@ bool Service::InvokeCmd(const CommandId* cid, CmdArgList tail_args, ConnectionCo
|
|||
return true;
|
||||
}
|
||||
|
||||
void Service::DispatchManyCommands(absl::Span<CmdArgList> args_list,
|
||||
facade::ConnectionContext* cntx) {
|
||||
size_t Service::DispatchManyCommands(absl::Span<CmdArgList> args_list,
|
||||
facade::ConnectionContext* cntx) {
|
||||
ConnectionContext* dfly_cntx = static_cast<ConnectionContext*>(cntx);
|
||||
DCHECK(!dfly_cntx->conn_state.exec_info.IsRunning());
|
||||
|
||||
vector<StoredCmd> stored_cmds;
|
||||
intrusive_ptr<Transaction> dist_trans;
|
||||
|
||||
size_t dispatched = 0;
|
||||
|
||||
auto perform_squash = [&] {
|
||||
if (stored_cmds.empty())
|
||||
return;
|
||||
|
@ -1244,9 +1245,15 @@ void Service::DispatchManyCommands(absl::Span<CmdArgList> args_list,
|
|||
dfly_cntx->transaction = dist_trans.get();
|
||||
MultiCommandSquasher::Execute(absl::MakeSpan(stored_cmds), dfly_cntx, this, true, false);
|
||||
dfly_cntx->transaction = nullptr;
|
||||
|
||||
dispatched += stored_cmds.size();
|
||||
stored_cmds.clear();
|
||||
};
|
||||
|
||||
// Don't even start when paused. We can only continue if DispatchTracker is aware of us running.
|
||||
if (dfly::ServerState::tlocal()->IsPaused())
|
||||
return 0;
|
||||
|
||||
for (auto args : args_list) {
|
||||
ToUpper(&args[0]);
|
||||
const auto [cid, tail_args] = FindCmd(args);
|
||||
|
@ -1272,14 +1279,21 @@ void Service::DispatchManyCommands(absl::Span<CmdArgList> args_list,
|
|||
// Squash accumulated commands
|
||||
perform_squash();
|
||||
|
||||
// Stop accumulating when a pause is requested, fall back to regular dispatch
|
||||
if (dfly::ServerState::tlocal()->IsPaused())
|
||||
break;
|
||||
|
||||
// Dispatch non squashed command only after all squshed commands were executed and replied
|
||||
DispatchCommand(args, cntx);
|
||||
dispatched++;
|
||||
}
|
||||
|
||||
perform_squash();
|
||||
|
||||
if (dist_trans)
|
||||
dist_trans->UnlockMulti();
|
||||
|
||||
return dispatched;
|
||||
}
|
||||
|
||||
void Service::DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
||||
|
@ -1418,10 +1432,6 @@ facade::ConnectionStats* Service::GetThreadLocalConnectionStats() {
|
|||
return ServerState::tl_connection_stats();
|
||||
}
|
||||
|
||||
void Service::AwaitOnPauseDispatch() {
|
||||
ServerState::tlocal()->AwaitOnPauseDispatch();
|
||||
}
|
||||
|
||||
const CommandId* Service::FindCmd(std::string_view cmd) const {
|
||||
return registry_.Find(cmd);
|
||||
}
|
||||
|
|
|
@ -48,8 +48,8 @@ class Service : public facade::ServiceInterface {
|
|||
void DispatchCommand(CmdArgList args, facade::ConnectionContext* cntx) final;
|
||||
|
||||
// Execute multiple consecutive commands, possibly in parallel by squashing
|
||||
void DispatchManyCommands(absl::Span<CmdArgList> args_list,
|
||||
facade::ConnectionContext* cntx) final;
|
||||
size_t DispatchManyCommands(absl::Span<CmdArgList> args_list,
|
||||
facade::ConnectionContext* cntx) final;
|
||||
|
||||
// Check VerifyCommandExecution and invoke command with args
|
||||
bool InvokeCmd(const CommandId* cid, CmdArgList tail_args, ConnectionContext* reply_cntx,
|
||||
|
@ -73,7 +73,6 @@ class Service : public facade::ServiceInterface {
|
|||
facade::Connection* owner) final;
|
||||
|
||||
facade::ConnectionStats* GetThreadLocalConnectionStats() final;
|
||||
void AwaitOnPauseDispatch() final;
|
||||
|
||||
std::pair<const CommandId*, CmdArgList> FindCmd(CmdArgList args) const;
|
||||
const CommandId* FindCmd(std::string_view) const;
|
||||
|
|
|
@ -1134,22 +1134,6 @@ void ServerFamily::CancelBlockingCommands() {
|
|||
}
|
||||
}
|
||||
|
||||
bool ServerFamily::AwaitCurrentDispatches(absl::Duration timeout, util::Connection* issuer) {
|
||||
vector<Fiber> fibers;
|
||||
bool successful = true;
|
||||
|
||||
for (auto* listener : listeners_) {
|
||||
fibers.push_back(MakeFiber([listener, timeout, issuer, &successful]() {
|
||||
successful &= listener->AwaitCurrentDispatches(timeout, issuer);
|
||||
}));
|
||||
}
|
||||
|
||||
for (auto& fb : fibers)
|
||||
fb.JoinIfNeeded();
|
||||
|
||||
return successful;
|
||||
}
|
||||
|
||||
string GetPassword() {
|
||||
string flag = GetFlag(FLAGS_requirepass);
|
||||
if (!flag.empty()) {
|
||||
|
@ -1306,31 +1290,28 @@ void ServerFamily::ClientPause(CmdArgList args, ConnectionContext* cntx) {
|
|||
return (*cntx)->SendError(err->MakeReply());
|
||||
}
|
||||
|
||||
// Pause dispatch commands before updating client puase state, and enable dispatch after updating
|
||||
// pause state. This will unsure that when we after changing the state all running commands will
|
||||
// read the new pause state, and we will not pause client in the middle of a transaction.
|
||||
service_.proactor_pool().Await([](util::ProactorBase* pb) {
|
||||
ServerState& etl = *ServerState::tlocal();
|
||||
etl.SetPauseDispatch(true);
|
||||
// Set global pause state and track commands that are running when the pause state is flipped.
|
||||
// Exlude already paused commands from the busy count.
|
||||
DispatchTracker tracker{GetListeners(), cntx->conn(), true /* ignore paused commands */};
|
||||
service_.proactor_pool().Await([&tracker, pause_state](util::ProactorBase* pb) {
|
||||
// Commands don't suspend before checking the pause state, so
|
||||
// it's impossible to deadlock on waiting for a command that will be paused.
|
||||
tracker.TrackOnThread();
|
||||
ServerState::tlocal()->SetPauseState(pause_state, true);
|
||||
});
|
||||
|
||||
// TODO handle blocking commands
|
||||
// Wait for all busy commands to finish running before replying to guarantee
|
||||
// that no more (write) operations will occur.
|
||||
const absl::Duration kDispatchTimeout = absl::Seconds(1);
|
||||
if (!AwaitCurrentDispatches(kDispatchTimeout, cntx->conn())) {
|
||||
LOG(WARNING) << "Couldn't wait for commands to finish dispatching. " << kDispatchTimeout;
|
||||
service_.proactor_pool().Await([](util::ProactorBase* pb) {
|
||||
ServerState& etl = *ServerState::tlocal();
|
||||
etl.SetPauseDispatch(false);
|
||||
if (!tracker.Wait(kDispatchTimeout)) {
|
||||
LOG(WARNING) << "Couldn't wait for commands to finish dispatching in " << kDispatchTimeout;
|
||||
service_.proactor_pool().Await([pause_state](util::ProactorBase* pb) {
|
||||
ServerState::tlocal()->SetPauseState(pause_state, false);
|
||||
});
|
||||
return (*cntx)->SendError("Failed to pause all running clients");
|
||||
}
|
||||
|
||||
service_.proactor_pool().AwaitFiberOnAll([pause_state](util::ProactorBase* pb) {
|
||||
ServerState& etl = *ServerState::tlocal();
|
||||
etl.SetPauseState(pause_state, true);
|
||||
etl.SetPauseDispatch(false);
|
||||
});
|
||||
|
||||
// We should not expire/evict keys while clients are puased.
|
||||
shard_set->RunBriefInParallel(
|
||||
[](EngineShard* shard) { shard->db_slice().SetExpireAllowed(false); });
|
||||
|
|
|
@ -184,7 +184,7 @@ class ServerFamily {
|
|||
return dfly_cmd_.get();
|
||||
}
|
||||
|
||||
const std::vector<facade::Listener*>& GetListeners() const {
|
||||
absl::Span<facade::Listener* const> GetListeners() const {
|
||||
return listeners_;
|
||||
}
|
||||
|
||||
|
@ -198,9 +198,6 @@ class ServerFamily {
|
|||
|
||||
void CancelBlockingCommands();
|
||||
|
||||
// Wait until all current dispatches finish, returns true on success, false if timeout was reached
|
||||
bool AwaitCurrentDispatches(absl::Duration timeout, util::Connection* issuer);
|
||||
|
||||
// Sets the server to replicate another instance. Does not flush the database beforehand!
|
||||
void Replicate(std::string_view host, std::string_view port);
|
||||
|
||||
|
|
|
@ -121,30 +121,13 @@ void ServerState::SetPauseState(ClientPause state, bool start) {
|
|||
|
||||
void ServerState::AwaitPauseState(bool is_write) {
|
||||
client_pause_ec_.await([is_write, this]() {
|
||||
if (client_pauses_[int(ClientPause::ALL)]) {
|
||||
return false;
|
||||
}
|
||||
if (is_write && client_pauses_[int(ClientPause::WRITE)]) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
return client_pauses_[int(ClientPause::ALL)] == 0 &&
|
||||
(!is_write || client_pauses_[int(ClientPause::WRITE)] == 0);
|
||||
});
|
||||
}
|
||||
|
||||
void ServerState::AwaitOnPauseDispatch() {
|
||||
pause_dispatch_ec_.await([this]() {
|
||||
if (pause_dispatch_) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
void ServerState::SetPauseDispatch(bool pause) {
|
||||
pause_dispatch_ = pause;
|
||||
if (!pause_dispatch_) {
|
||||
pause_dispatch_ec_.notifyAll();
|
||||
}
|
||||
bool ServerState::IsPaused() const {
|
||||
return (client_pauses_[0] + client_pauses_[1]) > 0;
|
||||
}
|
||||
|
||||
Interpreter* ServerState::BorrowInterpreter() {
|
||||
|
|
|
@ -231,11 +231,7 @@ class ServerState { // public struct - to allow initialization.
|
|||
// @is_write controls whether the command is a write command or not.
|
||||
void AwaitPauseState(bool is_write);
|
||||
|
||||
// Toggle a boolean indicating whether the server should temporarily pause or allow dispatching
|
||||
// new commands.
|
||||
void SetPauseDispatch(bool pause);
|
||||
// Awaits until dispatching new commands is allowed as determinded by SetPauseDispatch function
|
||||
void AwaitOnPauseDispatch();
|
||||
bool IsPaused() const;
|
||||
|
||||
SlowLogShard& GetSlowLog() {
|
||||
return slow_log_shard_;
|
||||
|
@ -260,8 +256,6 @@ class ServerState { // public struct - to allow initialization.
|
|||
// notified when the break is over.
|
||||
int client_pauses_[2] = {};
|
||||
EventCount client_pause_ec_;
|
||||
bool pause_dispatch_ = false;
|
||||
EventCount pause_dispatch_ec_;
|
||||
|
||||
using Counter = util::SlidingCounter<7>;
|
||||
Counter qps_;
|
||||
|
|
|
@ -1763,6 +1763,7 @@ async def test_search(df_local_factory):
|
|||
].id == "k0"
|
||||
|
||||
|
||||
# @pytest.mark.slow
|
||||
@pytest.mark.asyncio
|
||||
async def test_client_pause_with_replica(df_local_factory, df_seeder_factory):
|
||||
master = df_local_factory.create(proactor_threads=4)
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue