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:
Vladislav 2023-12-05 11:02:11 +03:00 committed by GitHub
parent f39eac5bca
commit 11ef6623dc
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
17 changed files with 165 additions and 143 deletions

View file

@ -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;

View file

@ -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}});
}

View file

@ -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_;

View file

@ -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

View file

@ -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

View file

@ -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;
}

View file

@ -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) {

View file

@ -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) {
}

View file

@ -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);

View file

@ -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;
}

View file

@ -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);
}

View file

@ -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;

View file

@ -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); });

View file

@ -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);

View file

@ -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() {

View file

@ -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_;

View file

@ -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)