mirror of
https://github.com/dragonflydb/dragonfly.git
synced 2025-05-11 02:15:45 +02:00
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:
parent
5f4fb57df4
commit
f66ee5f47d
7 changed files with 111 additions and 46 deletions
|
@ -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));
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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_};
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
|
@ -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(
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue