fix(cluster): Support FLUSHALL while slot migration is in progress (#3173)

* fix(cluster): Support `FLUSHALL` while slot migration is in progress

Fixes #3132

Also do a small refactor to move cancellation logic into
`RestoreStreamer`.
This commit is contained in:
Shahar Mike 2024-06-20 11:40:23 +03:00 committed by GitHub
parent 5f4fb57df4
commit f66ee5f47d
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
7 changed files with 111 additions and 46 deletions

View file

@ -791,15 +791,11 @@ bool RemoveIncomingMigrationImpl(std::vector<std::shared_ptr<IncomingSlotMigrati
// TODO make it outside in one run with other slots that should be flushed // TODO make it outside in one run with other slots that should be flushed
if (!removed.Empty()) { if (!removed.Empty()) {
auto removed_ranges = make_shared<SlotRanges>(removed.ToSlotRanges()); auto removed_ranges = removed.ToSlotRanges();
LOG_IF(WARNING, migration->GetState() == MigrationState::C_FINISHED) LOG_IF(WARNING, migration->GetState() == MigrationState::C_FINISHED)
<< "Flushing slots of removed FINISHED migration " << migration->GetSourceID() << "Flushing slots of removed FINISHED migration " << migration->GetSourceID()
<< ", slots: " << SlotRange::ToString(*removed_ranges); << ", slots: " << SlotRange::ToString(removed_ranges);
shard_set->pool()->DispatchOnAll([removed_ranges](unsigned, ProactorBase*) { DeleteSlots(removed_ranges);
if (EngineShard* shard = EngineShard::tlocal(); shard) {
shard->db_slice().FlushSlots(*removed_ranges);
}
});
} }
return true; return true;
@ -844,7 +840,7 @@ void ClusterFamily::InitMigration(CmdArgList args, ConnectionContext* cntx) {
lock_guard lk(migration_mu_); lock_guard lk(migration_mu_);
auto was_removed = RemoveIncomingMigrationImpl(incoming_migrations_jobs_, source_id); auto was_removed = RemoveIncomingMigrationImpl(incoming_migrations_jobs_, source_id);
LOG_IF(WARNING, was_removed) << "Reinit was happen for migration from:" << source_id; LOG_IF(WARNING, was_removed) << "Reinit issued for migration from:" << source_id;
incoming_migrations_jobs_.emplace_back(make_shared<IncomingSlotMigration>( incoming_migrations_jobs_.emplace_back(make_shared<IncomingSlotMigration>(
std::move(source_id), &server_family_->service(), std::move(slots), flows_num)); std::move(source_id), &server_family_->service(), std::move(slots), flows_num));

View file

@ -58,17 +58,11 @@ class OutgoingMigration::SliceSlotMigration : private ProtocolClient {
return; return;
} }
// Check if migration was cancelled while we yielded so far.
if (cancelled_) {
return;
}
streamer_.Start(Sock()); streamer_.Start(Sock());
} }
void Cancel() { void Cancel() {
streamer_.Cancel(); streamer_.Cancel();
cancelled_ = true;
} }
void Finalize() { void Finalize() {
@ -81,7 +75,6 @@ class OutgoingMigration::SliceSlotMigration : private ProtocolClient {
private: private:
RestoreStreamer streamer_; RestoreStreamer streamer_;
bool cancelled_ = false;
}; };
OutgoingMigration::OutgoingMigration(MigrationInfo info, ClusterFamily* cf, ServerFamily* sf) OutgoingMigration::OutgoingMigration(MigrationInfo info, ClusterFamily* cf, ServerFamily* sf)
@ -94,6 +87,13 @@ OutgoingMigration::OutgoingMigration(MigrationInfo info, ClusterFamily* cf, Serv
OutgoingMigration::~OutgoingMigration() { OutgoingMigration::~OutgoingMigration() {
main_sync_fb_.JoinIfNeeded(); main_sync_fb_.JoinIfNeeded();
// Destroy each flow in its dedicated thread, because we could be the last owner of the db tables
shard_set->pool()->AwaitFiberOnAll([this](util::ProactorBase* pb) {
if (const auto* shard = EngineShard::tlocal(); shard) {
slot_migrations_[shard->shard_id()].reset();
}
});
} }
bool OutgoingMigration::ChangeState(MigrationState new_state) { bool OutgoingMigration::ChangeState(MigrationState new_state) {

View file

@ -165,6 +165,7 @@ void JournalSlice::AddLogRecord(const Entry& entry, bool await) {
item = &dummy; item = &dummy;
item->opcode = entry.opcode; item->opcode = entry.opcode;
item->lsn = lsn_++; item->lsn = lsn_++;
item->cmd = entry.payload.cmd;
item->slot = entry.slot; item->slot = entry.slot;
io::BufSink buf_sink{&ring_serialize_buf_}; io::BufSink buf_sink{&ring_serialize_buf_};

View file

@ -34,7 +34,7 @@ uint32_t replication_stream_output_limit_cached = 64_KB;
} // namespace } // namespace
JournalStreamer::JournalStreamer(journal::Journal* journal, Context* cntx) JournalStreamer::JournalStreamer(journal::Journal* journal, Context* cntx)
: journal_(journal), cntx_(cntx) { : cntx_(cntx), journal_(journal) {
// cache the flag to avoid accessing it later. // cache the flag to avoid accessing it later.
replication_stream_output_limit_cached = absl::GetFlag(FLAGS_replication_stream_output_limit); replication_stream_output_limit_cached = absl::GetFlag(FLAGS_replication_stream_output_limit);
} }
@ -44,7 +44,7 @@ JournalStreamer::~JournalStreamer() {
VLOG(1) << "~JournalStreamer"; VLOG(1) << "~JournalStreamer";
} }
void JournalStreamer::Start(io::AsyncSink* dest, bool send_lsn) { void JournalStreamer::Start(util::FiberSocketBase* dest, bool send_lsn) {
CHECK(dest_ == nullptr && dest != nullptr); CHECK(dest_ == nullptr && dest != nullptr);
dest_ = dest; dest_ = dest;
journal_cb_id_ = journal_cb_id_ =
@ -188,9 +188,13 @@ RestoreStreamer::RestoreStreamer(DbSlice* slice, cluster::SlotSet slots, journal
Context* cntx) Context* cntx)
: JournalStreamer(journal, cntx), db_slice_(slice), my_slots_(std::move(slots)) { : JournalStreamer(journal, cntx), db_slice_(slice), my_slots_(std::move(slots)) {
DCHECK(slice != nullptr); DCHECK(slice != nullptr);
db_array_ = slice->databases(); // Inc ref to make sure DB isn't deleted while we use it
} }
void RestoreStreamer::Start(io::AsyncSink* dest, bool send_lsn) { void RestoreStreamer::Start(util::FiberSocketBase* dest, bool send_lsn) {
if (fiber_cancelled_)
return;
VLOG(1) << "RestoreStreamer start"; VLOG(1) << "RestoreStreamer start";
auto db_cb = absl::bind_front(&RestoreStreamer::OnDbChange, this); auto db_cb = absl::bind_front(&RestoreStreamer::OnDbChange, this);
snapshot_version_ = db_slice_->RegisterOnChange(std::move(db_cb)); snapshot_version_ = db_slice_->RegisterOnChange(std::move(db_cb));
@ -199,7 +203,7 @@ void RestoreStreamer::Start(io::AsyncSink* dest, bool send_lsn) {
PrimeTable::Cursor cursor; PrimeTable::Cursor cursor;
uint64_t last_yield = 0; uint64_t last_yield = 0;
PrimeTable* pt = &db_slice_->databases()[0]->prime; PrimeTable* pt = &db_array_[0]->prime;
do { do {
if (fiber_cancelled_) if (fiber_cancelled_)
@ -244,14 +248,22 @@ RestoreStreamer::~RestoreStreamer() {
void RestoreStreamer::Cancel() { void RestoreStreamer::Cancel() {
auto sver = snapshot_version_; auto sver = snapshot_version_;
snapshot_version_ = 0; // to prevent double cancel in another fiber snapshot_version_ = 0; // to prevent double cancel in another fiber
if (sver != 0) {
fiber_cancelled_ = true; fiber_cancelled_ = true;
if (sver != 0) {
db_slice_->UnregisterOnChange(sver); db_slice_->UnregisterOnChange(sver);
JournalStreamer::Cancel(); JournalStreamer::Cancel();
} }
} }
bool RestoreStreamer::ShouldWrite(const journal::JournalItem& item) const { bool RestoreStreamer::ShouldWrite(const journal::JournalItem& item) const {
if (item.cmd == "FLUSHALL" || item.cmd == "FLUSHDB") {
// On FLUSH* we restart the migration
CHECK(dest_ != nullptr);
cntx_->ReportError("FLUSH command during migration");
dest_->Shutdown(SHUT_RDWR);
return false;
}
if (!item.slot.has_value()) { if (!item.slot.has_value()) {
return false; return false;
} }

View file

@ -23,7 +23,7 @@ class JournalStreamer {
JournalStreamer(JournalStreamer&& other) = delete; JournalStreamer(JournalStreamer&& other) = delete;
// Register journal listener and start writer in fiber. // Register journal listener and start writer in fiber.
virtual void Start(io::AsyncSink* dest, bool send_lsn); virtual void Start(util::FiberSocketBase* dest, bool send_lsn);
// Must be called on context cancellation for unblocking // Must be called on context cancellation for unblocking
// and manual cleanup. // and manual cleanup.
@ -48,6 +48,9 @@ class JournalStreamer {
void WaitForInflightToComplete(); void WaitForInflightToComplete();
util::FiberSocketBase* dest_ = nullptr;
Context* cntx_;
private: private:
void OnCompletion(std::error_code ec, size_t len); void OnCompletion(std::error_code ec, size_t len);
@ -58,8 +61,6 @@ class JournalStreamer {
bool IsStalled() const; bool IsStalled() const;
journal::Journal* journal_; journal::Journal* journal_;
Context* cntx_;
io::AsyncSink* dest_ = nullptr;
std::vector<uint8_t> pending_buf_; std::vector<uint8_t> pending_buf_;
size_t in_flight_bytes_ = 0; size_t in_flight_bytes_ = 0;
time_t last_lsn_time_ = 0; time_t last_lsn_time_ = 0;
@ -74,7 +75,7 @@ class RestoreStreamer : public JournalStreamer {
RestoreStreamer(DbSlice* slice, cluster::SlotSet slots, journal::Journal* journal, Context* cntx); RestoreStreamer(DbSlice* slice, cluster::SlotSet slots, journal::Journal* journal, Context* cntx);
~RestoreStreamer() override; ~RestoreStreamer() override;
void Start(io::AsyncSink* dest, bool send_lsn = false) override; void Start(util::FiberSocketBase* dest, bool send_lsn = false) override;
// Cancel() must be called if Start() is called // Cancel() must be called if Start() is called
void Cancel() override; void Cancel() override;
@ -96,6 +97,7 @@ class RestoreStreamer : public JournalStreamer {
void WriteCommand(journal::Entry::Payload cmd_payload); void WriteCommand(journal::Entry::Payload cmd_payload);
DbSlice* db_slice_; DbSlice* db_slice_;
DbTableArray db_array_;
uint64_t snapshot_version_ = 0; uint64_t snapshot_version_ = 0;
cluster::SlotSet my_slots_; cluster::SlotSet my_slots_;
bool fiber_cancelled_ = false; bool fiber_cancelled_ = false;

View file

@ -95,6 +95,7 @@ struct JournalItem {
LSN lsn; LSN lsn;
Op opcode; Op opcode;
std::string data; std::string data;
std::string_view cmd;
std::optional<cluster::SlotId> slot; std::optional<cluster::SlotId> slot;
}; };

View file

@ -19,6 +19,16 @@ from . import dfly_args
BASE_PORT = 30001 BASE_PORT = 30001
async def assert_eventually(e):
iterations = 0
while True:
if await e():
return
iterations += 1
assert iterations < 500
await asyncio.sleep(0.1)
class RedisClusterNode: class RedisClusterNode:
def __init__(self, port): def __init__(self, port):
self.port = port self.port = port
@ -1026,6 +1036,59 @@ async def test_config_consistency(df_local_factory: DflyInstanceFactory):
await close_clients(*[node.client for node in nodes], *[node.admin_client for node in nodes]) await close_clients(*[node.client for node in nodes], *[node.admin_client for node in nodes])
@dfly_args({"proactor_threads": 4, "cluster_mode": "yes"})
async def test_cluster_flushall_during_migration(
df_local_factory: DflyInstanceFactory, df_seeder_factory
):
# Check data migration from one node to another
instances = [
df_local_factory.create(
port=BASE_PORT + i,
admin_port=BASE_PORT + i + 1000,
vmodule="cluster_family=9,cluster_slot_migration=9,outgoing_slot_migration=9",
logtostdout=True,
)
for i in range(2)
]
df_local_factory.start_all(instances)
nodes = [(await create_node_info(instance)) for instance in instances]
nodes[0].slots = [(0, 16383)]
nodes[1].slots = []
await push_config(json.dumps(generate_config(nodes)), [node.admin_client for node in nodes])
seeder = df_seeder_factory.create(keys=10_000, port=nodes[0].instance.port, cluster_mode=True)
await seeder.run(target_deviation=0.1)
nodes[0].migrations.append(
MigrationInfo("127.0.0.1", nodes[1].instance.admin_port, [(0, 16383)], nodes[1].id)
)
logging.debug("Start migration")
await push_config(json.dumps(generate_config(nodes)), [node.admin_client for node in nodes])
await nodes[0].client.execute_command("flushall")
assert "FINISHED" not in await nodes[1].admin_client.execute_command(
"DFLYCLUSTER", "SLOT-MIGRATION-STATUS", nodes[0].id
), "Weak test case - finished migration too early"
await wait_for_status(nodes[0].admin_client, nodes[1].id, "FINISHED")
logging.debug("Finalizing migration")
nodes[0].migrations = []
nodes[0].slots = []
nodes[1].slots = [(0, 16383)]
await push_config(json.dumps(generate_config(nodes)), [node.admin_client for node in nodes])
logging.debug("Migration finalized")
assert await nodes[0].client.dbsize() == 0
await close_clients(*[node.client for node in nodes], *[node.admin_client for node in nodes])
@dfly_args({"proactor_threads": 4, "cluster_mode": "yes"}) @dfly_args({"proactor_threads": 4, "cluster_mode": "yes"})
async def test_cluster_data_migration(df_local_factory: DflyInstanceFactory): async def test_cluster_data_migration(df_local_factory: DflyInstanceFactory):
# Check data migration from one node to another # Check data migration from one node to another
@ -1065,12 +1128,12 @@ async def test_cluster_data_migration(df_local_factory: DflyInstanceFactory):
await nodes[0].admin_client.execute_command( await nodes[0].admin_client.execute_command(
"DFLYCLUSTER", "SLOT-MIGRATION-STATUS", nodes[1].id "DFLYCLUSTER", "SLOT-MIGRATION-STATUS", nodes[1].id
) )
).startswith(f"""out {nodes[1].id} FINISHED keys:7""") ).startswith(f"out {nodes[1].id} FINISHED keys:7")
assert ( assert (
await nodes[1].admin_client.execute_command( await nodes[1].admin_client.execute_command(
"DFLYCLUSTER", "SLOT-MIGRATION-STATUS", nodes[0].id "DFLYCLUSTER", "SLOT-MIGRATION-STATUS", nodes[0].id
) )
).startswith(f"""in {nodes[0].id} FINISHED keys:7""") ).startswith(f"in {nodes[0].id} FINISHED keys:7")
nodes[0].migrations = [] nodes[0].migrations = []
nodes[0].slots = [(0, 2999)] nodes[0].slots = [(0, 2999)]
@ -1232,23 +1295,15 @@ async def test_cluster_fuzzymigration(
logging.debug("start migrations") logging.debug("start migrations")
await push_config(json.dumps(generate_config(nodes)), [node.admin_client for node in nodes]) await push_config(json.dumps(generate_config(nodes)), [node.admin_client for node in nodes])
iterations = 0 async def all_finished():
while True:
is_all_finished = True
for node in nodes: for node in nodes:
states = await node.admin_client.execute_command("DFLYCLUSTER", "SLOT-MIGRATION-STATUS") states = await node.admin_client.execute_command("DFLYCLUSTER", "SLOT-MIGRATION-STATUS")
logging.debug(states) logging.debug(states)
is_all_finished = is_all_finished and ( if not all("FINISHED" in s for s in states) or states == "NO_STATE":
all("FINISHED" in s for s in states) or states == "NO_STATE" return False
) return True
if is_all_finished: await assert_eventually(all_finished)
break
iterations += 1
assert iterations < 500
await asyncio.sleep(0.1)
for counter in counters: for counter in counters:
counter.cancel() counter.cancel()
@ -1360,13 +1415,11 @@ async def test_cluster_migration_cancel(df_local_factory: DflyInstanceFactory):
nodes[0].migrations = [] nodes[0].migrations = []
await push_config(json.dumps(generate_config(nodes)), [node.admin_client for node in nodes]) await push_config(json.dumps(generate_config(nodes)), [node.admin_client for node in nodes])
assert SIZE == await nodes[0].client.dbsize() assert SIZE == await nodes[0].client.dbsize()
while True:
db_size = await nodes[1].client.dbsize() async def node1size0():
if 0 == db_size: return await nodes[1].client.dbsize() == 0
break
logging.debug(f"target dbsize is {db_size}") await assert_eventually(node1size0)
logging.debug(await nodes[1].client.execute_command("KEYS", "*"))
await asyncio.sleep(0.1)
logging.debug("Reissuing migration") logging.debug("Reissuing migration")
nodes[0].migrations.append( nodes[0].migrations.append(