mirror of
https://github.com/dragonflydb/dragonfly.git
synced 2025-05-10 18:05:44 +02:00
chore: remove DflyVersion::VER0 (#3593)
Stop supporting DflyVersion::VER0 from more than a year ago. In addition, rename Metrics fields to make them more clear General improvements and fix the reconnect metric. Signed-off-by: Roman Gershman <roman@dragonflydb.io>
This commit is contained in:
parent
24b8d3294b
commit
0ee52c9d35
7 changed files with 98 additions and 94 deletions
|
@ -133,7 +133,7 @@ struct ConnectionState {
|
||||||
uint32_t repl_flow_id = UINT32_MAX;
|
uint32_t repl_flow_id = UINT32_MAX;
|
||||||
std::string repl_ip_address;
|
std::string repl_ip_address;
|
||||||
uint32_t repl_listening_port = 0;
|
uint32_t repl_listening_port = 0;
|
||||||
DflyVersion repl_version = DflyVersion::VER0;
|
DflyVersion repl_version = DflyVersion::VER1;
|
||||||
};
|
};
|
||||||
|
|
||||||
struct SquashingInfo {
|
struct SquashingInfo {
|
||||||
|
|
|
@ -43,7 +43,7 @@ struct FlowInfo {
|
||||||
std::unique_ptr<JournalStreamer> streamer; // Streamer for stable sync phase
|
std::unique_ptr<JournalStreamer> streamer; // Streamer for stable sync phase
|
||||||
std::string eof_token;
|
std::string eof_token;
|
||||||
|
|
||||||
DflyVersion version = DflyVersion::VER0;
|
DflyVersion version = DflyVersion::VER1;
|
||||||
|
|
||||||
std::optional<LSN> start_partial_sync_at;
|
std::optional<LSN> start_partial_sync_at;
|
||||||
uint64_t last_acked_lsn = 0;
|
uint64_t last_acked_lsn = 0;
|
||||||
|
@ -128,7 +128,7 @@ class DflyCmd {
|
||||||
std::string id;
|
std::string id;
|
||||||
std::string address;
|
std::string address;
|
||||||
uint32_t listening_port;
|
uint32_t listening_port;
|
||||||
DflyVersion version = DflyVersion::VER0;
|
DflyVersion version = DflyVersion::VER1;
|
||||||
|
|
||||||
// Flows describe the state of shard-local flow.
|
// Flows describe the state of shard-local flow.
|
||||||
// They are always indexed by the shard index on the master.
|
// They are always indexed by the shard index on the master.
|
||||||
|
@ -153,6 +153,7 @@ class DflyCmd {
|
||||||
// Master side acces method to replication info of that connection.
|
// Master side acces method to replication info of that connection.
|
||||||
std::shared_ptr<ReplicaInfo> GetReplicaInfoFromConnection(ConnectionContext* cntx);
|
std::shared_ptr<ReplicaInfo> GetReplicaInfoFromConnection(ConnectionContext* cntx);
|
||||||
|
|
||||||
|
// Master-side command. Provides Replica info.
|
||||||
std::vector<ReplicaRoleInfo> GetReplicasRoleInfo() const ABSL_LOCKS_EXCLUDED(mu_);
|
std::vector<ReplicaRoleInfo> GetReplicasRoleInfo() const ABSL_LOCKS_EXCLUDED(mu_);
|
||||||
|
|
||||||
void GetReplicationMemoryStats(ReplicationMemoryStats* out) const ABSL_LOCKS_EXCLUDED(mu_);
|
void GetReplicationMemoryStats(ReplicationMemoryStats* out) const ABSL_LOCKS_EXCLUDED(mu_);
|
||||||
|
|
|
@ -162,15 +162,15 @@ void Replica::Pause(bool pause) {
|
||||||
VLOG(1) << "Pausing replication";
|
VLOG(1) << "Pausing replication";
|
||||||
Proactor()->Await([&] {
|
Proactor()->Await([&] {
|
||||||
is_paused_ = pause;
|
is_paused_ = pause;
|
||||||
if (num_df_flows_ > 0) {
|
if (shard_flows_.empty())
|
||||||
auto partition = Partition(num_df_flows_);
|
return;
|
||||||
|
|
||||||
auto cb = [&](unsigned index, auto*) {
|
auto cb = [&](unsigned index, auto*) {
|
||||||
for (auto id : partition[index]) {
|
for (auto id : thread_flow_map_[index]) {
|
||||||
shard_flows_[id]->Pause(pause);
|
shard_flows_[id]->Pause(pause);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
shard_set->pool()->AwaitBrief(cb);
|
shard_set->pool()->AwaitBrief(cb);
|
||||||
}
|
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -347,14 +347,15 @@ std::error_code Replica::HandleCapaDflyResp() {
|
||||||
}
|
}
|
||||||
master_context_.master_repl_id = master_repl_id;
|
master_context_.master_repl_id = master_repl_id;
|
||||||
master_context_.dfly_session_id = ToSV(LastResponseArgs()[1].GetBuf());
|
master_context_.dfly_session_id = ToSV(LastResponseArgs()[1].GetBuf());
|
||||||
num_df_flows_ = param_num_flows;
|
master_context_.num_flows = param_num_flows;
|
||||||
|
|
||||||
if (LastResponseArgs().size() >= 4) {
|
if (LastResponseArgs().size() >= 4) {
|
||||||
PC_RETURN_ON_BAD_RESPONSE(LastResponseArgs()[3].type == RespExpr::INT64);
|
PC_RETURN_ON_BAD_RESPONSE(LastResponseArgs()[3].type == RespExpr::INT64);
|
||||||
master_context_.version = DflyVersion(get<int64_t>(LastResponseArgs()[3].u));
|
master_context_.version = DflyVersion(get<int64_t>(LastResponseArgs()[3].u));
|
||||||
}
|
}
|
||||||
VLOG(1) << "Master id: " << master_context_.master_repl_id
|
VLOG(1) << "Master id: " << master_context_.master_repl_id
|
||||||
<< ", sync id: " << master_context_.dfly_session_id << ", num journals: " << num_df_flows_
|
<< ", sync id: " << master_context_.dfly_session_id
|
||||||
|
<< ", num journals: " << param_num_flows
|
||||||
<< ", version: " << unsigned(master_context_.version);
|
<< ", version: " << unsigned(master_context_.version);
|
||||||
|
|
||||||
return error_code{};
|
return error_code{};
|
||||||
|
@ -369,12 +370,9 @@ std::error_code Replica::ConfigureDflyMaster() {
|
||||||
LOG(WARNING) << "Bad REPLCONF CLIENT-ID response";
|
LOG(WARNING) << "Bad REPLCONF CLIENT-ID response";
|
||||||
}
|
}
|
||||||
|
|
||||||
// Tell the master our version if it supports REPLCONF CLIENT-VERSION
|
|
||||||
if (master_context_.version > DflyVersion::VER0) {
|
|
||||||
RETURN_ON_ERR(
|
RETURN_ON_ERR(
|
||||||
SendCommandAndReadResponse(StrCat("REPLCONF CLIENT-VERSION ", DflyVersion::CURRENT_VER)));
|
SendCommandAndReadResponse(StrCat("REPLCONF CLIENT-VERSION ", DflyVersion::CURRENT_VER)));
|
||||||
PC_RETURN_ON_BAD_RESPONSE(CheckRespIsSimpleReply("OK"));
|
PC_RETURN_ON_BAD_RESPONSE(CheckRespIsSimpleReply("OK"));
|
||||||
}
|
|
||||||
|
|
||||||
return error_code{};
|
return error_code{};
|
||||||
}
|
}
|
||||||
|
@ -476,14 +474,16 @@ error_code Replica::InitiateDflySync() {
|
||||||
multi_shard_exe_.reset(new MultiShardExecution());
|
multi_shard_exe_.reset(new MultiShardExecution());
|
||||||
|
|
||||||
// Initialize shard flows.
|
// Initialize shard flows.
|
||||||
shard_flows_.resize(num_df_flows_);
|
shard_flows_.resize(master_context_.num_flows);
|
||||||
for (unsigned i = 0; i < num_df_flows_; ++i) {
|
DCHECK(!shard_flows_.empty());
|
||||||
|
for (unsigned i = 0; i < shard_flows_.size(); ++i) {
|
||||||
shard_flows_[i].reset(
|
shard_flows_[i].reset(
|
||||||
new DflyShardReplica(server(), master_context_, i, &service_, multi_shard_exe_));
|
new DflyShardReplica(server(), master_context_, i, &service_, multi_shard_exe_));
|
||||||
}
|
}
|
||||||
|
thread_flow_map_ = Partition(shard_flows_.size());
|
||||||
|
|
||||||
// Blocked on until all flows got full sync cut.
|
// Blocked on until all flows got full sync cut.
|
||||||
BlockingCounter sync_block{num_df_flows_};
|
BlockingCounter sync_block{unsigned(shard_flows_.size())};
|
||||||
|
|
||||||
// Switch to new error handler that closes flow sockets.
|
// Switch to new error handler that closes flow sockets.
|
||||||
auto err_handler = [this, sync_block](const auto& ge) mutable {
|
auto err_handler = [this, sync_block](const auto& ge) mutable {
|
||||||
|
@ -516,12 +516,12 @@ error_code Replica::InitiateDflySync() {
|
||||||
|
|
||||||
std::string_view sync_type = "full";
|
std::string_view sync_type = "full";
|
||||||
{
|
{
|
||||||
|
unsigned num_df_flows = shard_flows_.size();
|
||||||
// Going out of the way to avoid using std::vector<bool>...
|
// Going out of the way to avoid using std::vector<bool>...
|
||||||
auto is_full_sync = std::make_unique<bool[]>(num_df_flows_);
|
auto is_full_sync = std::make_unique<bool[]>(num_df_flows);
|
||||||
auto partition = Partition(num_df_flows_);
|
DCHECK(!last_journal_LSNs_ || last_journal_LSNs_->size() == num_df_flows);
|
||||||
CHECK(!last_journal_LSNs_ || last_journal_LSNs_->size() == shard_flows_.size());
|
|
||||||
auto shard_cb = [&](unsigned index, auto*) {
|
auto shard_cb = [&](unsigned index, auto*) {
|
||||||
for (auto id : partition[index]) {
|
for (auto id : thread_flow_map_[index]) {
|
||||||
auto ec = shard_flows_[id]->StartSyncFlow(sync_block, &cntx_,
|
auto ec = shard_flows_[id]->StartSyncFlow(sync_block, &cntx_,
|
||||||
last_journal_LSNs_.has_value()
|
last_journal_LSNs_.has_value()
|
||||||
? std::optional((*last_journal_LSNs_)[id])
|
? std::optional((*last_journal_LSNs_)[id])
|
||||||
|
@ -538,9 +538,9 @@ error_code Replica::InitiateDflySync() {
|
||||||
shard_set->pool()->AwaitFiberOnAll(std::move(shard_cb));
|
shard_set->pool()->AwaitFiberOnAll(std::move(shard_cb));
|
||||||
|
|
||||||
size_t num_full_flows =
|
size_t num_full_flows =
|
||||||
std::accumulate(is_full_sync.get(), is_full_sync.get() + num_df_flows_, 0);
|
std::accumulate(is_full_sync.get(), is_full_sync.get() + num_df_flows, 0);
|
||||||
|
|
||||||
if (num_full_flows == num_df_flows_) {
|
if (num_full_flows == num_df_flows) {
|
||||||
if (slot_range_.has_value()) {
|
if (slot_range_.has_value()) {
|
||||||
JournalExecutor{&service_}.FlushSlots(slot_range_.value());
|
JournalExecutor{&service_}.FlushSlots(slot_range_.value());
|
||||||
} else {
|
} else {
|
||||||
|
@ -668,9 +668,8 @@ error_code Replica::ConsumeDflyStream() {
|
||||||
LOG(INFO) << "Transitioned into stable sync";
|
LOG(INFO) << "Transitioned into stable sync";
|
||||||
// Transition flows into stable sync.
|
// Transition flows into stable sync.
|
||||||
{
|
{
|
||||||
auto partition = Partition(num_df_flows_);
|
|
||||||
auto shard_cb = [&](unsigned index, auto*) {
|
auto shard_cb = [&](unsigned index, auto*) {
|
||||||
const auto& local_ids = partition[index];
|
const auto& local_ids = thread_flow_map_[index];
|
||||||
for (unsigned id : local_ids) {
|
for (unsigned id : local_ids) {
|
||||||
auto ec = shard_flows_[id]->StartStableSyncFlow(&cntx_);
|
auto ec = shard_flows_[id]->StartStableSyncFlow(&cntx_);
|
||||||
if (ec)
|
if (ec)
|
||||||
|
@ -723,6 +722,7 @@ io::Result<bool> DflyShardReplica::StartSyncFlow(BlockingCounter sb, Context* cn
|
||||||
std::optional<LSN> lsn) {
|
std::optional<LSN> lsn) {
|
||||||
using nonstd::make_unexpected;
|
using nonstd::make_unexpected;
|
||||||
DCHECK(!master_context_.master_repl_id.empty() && !master_context_.dfly_session_id.empty());
|
DCHECK(!master_context_.master_repl_id.empty() && !master_context_.dfly_session_id.empty());
|
||||||
|
proactor_index_ = ProactorBase::me()->GetPoolIndex();
|
||||||
|
|
||||||
RETURN_ON_ERR_T(make_unexpected,
|
RETURN_ON_ERR_T(make_unexpected,
|
||||||
ConnectAndAuth(absl::GetFlag(FLAGS_master_connect_timeout_ms) * 1ms, &cntx_));
|
ConnectAndAuth(absl::GetFlag(FLAGS_master_connect_timeout_ms) * 1ms, &cntx_));
|
||||||
|
@ -826,7 +826,6 @@ void DflyShardReplica::FullSyncDflyFb(std::string eof_token, BlockingCounter bc,
|
||||||
if (auto jo = rdb_loader_->journal_offset(); jo.has_value()) {
|
if (auto jo = rdb_loader_->journal_offset(); jo.has_value()) {
|
||||||
this->journal_rec_executed_.store(*jo);
|
this->journal_rec_executed_.store(*jo);
|
||||||
} else {
|
} else {
|
||||||
if (master_context_.version > DflyVersion::VER0)
|
|
||||||
cntx->ReportError(std::make_error_code(errc::protocol_error),
|
cntx->ReportError(std::make_error_code(errc::protocol_error),
|
||||||
"Error finding journal offset in stream");
|
"Error finding journal offset in stream");
|
||||||
}
|
}
|
||||||
|
@ -834,6 +833,8 @@ void DflyShardReplica::FullSyncDflyFb(std::string eof_token, BlockingCounter bc,
|
||||||
}
|
}
|
||||||
|
|
||||||
void DflyShardReplica::StableSyncDflyReadFb(Context* cntx) {
|
void DflyShardReplica::StableSyncDflyReadFb(Context* cntx) {
|
||||||
|
DCHECK_EQ(proactor_index_, ProactorBase::me()->GetPoolIndex());
|
||||||
|
|
||||||
// Check leftover from full sync.
|
// Check leftover from full sync.
|
||||||
io::Bytes prefix{};
|
io::Bytes prefix{};
|
||||||
if (leftover_buf_ && leftover_buf_->InputLen() > 0) {
|
if (leftover_buf_ && leftover_buf_->InputLen() > 0) {
|
||||||
|
@ -846,15 +847,15 @@ void DflyShardReplica::StableSyncDflyReadFb(Context* cntx) {
|
||||||
DCHECK_GE(journal_rec_executed_, 1u);
|
DCHECK_GE(journal_rec_executed_, 1u);
|
||||||
TransactionReader tx_reader{journal_rec_executed_.load(std::memory_order_relaxed) - 1};
|
TransactionReader tx_reader{journal_rec_executed_.load(std::memory_order_relaxed) - 1};
|
||||||
|
|
||||||
if (master_context_.version > DflyVersion::VER0) {
|
|
||||||
acks_fb_ = fb2::Fiber("shard_acks", &DflyShardReplica::StableSyncDflyAcksFb, this, cntx);
|
acks_fb_ = fb2::Fiber("shard_acks", &DflyShardReplica::StableSyncDflyAcksFb, this, cntx);
|
||||||
}
|
|
||||||
|
|
||||||
while (!cntx->IsCancelled()) {
|
while (!cntx->IsCancelled()) {
|
||||||
auto tx_data = tx_reader.NextTxData(&reader, cntx);
|
auto tx_data = tx_reader.NextTxData(&reader, cntx);
|
||||||
if (!tx_data)
|
if (!tx_data)
|
||||||
break;
|
break;
|
||||||
|
|
||||||
|
DVLOG(3) << "Lsn: " << tx_data->lsn;
|
||||||
|
|
||||||
last_io_time_ = Proactor()->GetMonotonicTimeNs();
|
last_io_time_ = Proactor()->GetMonotonicTimeNs();
|
||||||
if (tx_data->opcode == journal::Op::LSN) {
|
if (tx_data->opcode == journal::Op::LSN) {
|
||||||
// Do nothing
|
// Do nothing
|
||||||
|
@ -895,6 +896,8 @@ void Replica::RedisStreamAcksFb() {
|
||||||
}
|
}
|
||||||
|
|
||||||
void DflyShardReplica::StableSyncDflyAcksFb(Context* cntx) {
|
void DflyShardReplica::StableSyncDflyAcksFb(Context* cntx) {
|
||||||
|
DCHECK_EQ(proactor_index_, ProactorBase::me()->GetPoolIndex());
|
||||||
|
|
||||||
constexpr size_t kAckRecordMaxInterval = 1024;
|
constexpr size_t kAckRecordMaxInterval = 1024;
|
||||||
std::chrono::duration ack_time_max_interval =
|
std::chrono::duration ack_time_max_interval =
|
||||||
1ms * absl::GetFlag(FLAGS_replication_acks_interval);
|
1ms * absl::GetFlag(FLAGS_replication_acks_interval);
|
||||||
|
@ -1077,6 +1080,9 @@ error_code Replica::ParseReplicationHeader(base::IoBuf* io_buf, PSyncResponse* d
|
||||||
auto Replica::GetSummary() const -> Summary {
|
auto Replica::GetSummary() const -> Summary {
|
||||||
auto f = [this]() {
|
auto f = [this]() {
|
||||||
auto last_io_time = LastIoTime();
|
auto last_io_time = LastIoTime();
|
||||||
|
|
||||||
|
// Note: we access LastIoTime from foreigh thread in unsafe manner. However, specifically here
|
||||||
|
// it's unlikely to cause a real bug.
|
||||||
for (const auto& flow : shard_flows_) { // Get last io time from all sub flows.
|
for (const auto& flow : shard_flows_) { // Get last io time from all sub flows.
|
||||||
last_io_time = std::max(last_io_time, flow->LastIoTime());
|
last_io_time = std::max(last_io_time, flow->LastIoTime());
|
||||||
}
|
}
|
||||||
|
@ -1095,7 +1101,7 @@ auto Replica::GetSummary() const -> Summary {
|
||||||
|
|
||||||
if (Sock())
|
if (Sock())
|
||||||
return Proactor()->AwaitBrief(f);
|
return Proactor()->AwaitBrief(f);
|
||||||
else {
|
|
||||||
/**
|
/**
|
||||||
* when this branch happens: there is a very short grace period
|
* when this branch happens: there is a very short grace period
|
||||||
* where Sock() is not initialized, yet the server can
|
* where Sock() is not initialized, yet the server can
|
||||||
|
@ -1106,7 +1112,6 @@ auto Replica::GetSummary() const -> Summary {
|
||||||
* In that instance, we have to run f() on the current fiber.
|
* In that instance, we have to run f() on the current fiber.
|
||||||
*/
|
*/
|
||||||
return f();
|
return f();
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
std::vector<uint64_t> Replica::GetReplicaOffset() const {
|
std::vector<uint64_t> Replica::GetReplicaOffset() const {
|
||||||
|
@ -1129,10 +1134,6 @@ uint32_t DflyShardReplica::FlowId() const {
|
||||||
return flow_id_;
|
return flow_id_;
|
||||||
}
|
}
|
||||||
|
|
||||||
uint64_t DflyShardReplica::JournalExecutedCount() const {
|
|
||||||
return journal_rec_executed_.load(std::memory_order_relaxed);
|
|
||||||
}
|
|
||||||
|
|
||||||
void DflyShardReplica::Pause(bool pause) {
|
void DflyShardReplica::Pause(bool pause) {
|
||||||
if (rdb_loader_) {
|
if (rdb_loader_) {
|
||||||
rdb_loader_->Pause(pause);
|
rdb_loader_->Pause(pause);
|
||||||
|
|
|
@ -36,7 +36,8 @@ class DflyShardReplica;
|
||||||
struct MasterContext {
|
struct MasterContext {
|
||||||
std::string master_repl_id;
|
std::string master_repl_id;
|
||||||
std::string dfly_session_id; // Sync session id for dfly sync.
|
std::string dfly_session_id; // Sync session id for dfly sync.
|
||||||
DflyVersion version = DflyVersion::VER0;
|
unsigned num_flows = 0;
|
||||||
|
DflyVersion version = DflyVersion::VER1;
|
||||||
};
|
};
|
||||||
|
|
||||||
// This class manages replication from both Dragonfly and Redis masters.
|
// This class manages replication from both Dragonfly and Redis masters.
|
||||||
|
@ -122,7 +123,7 @@ class Replica : ProtocolClient {
|
||||||
uint32_t reconnect_count;
|
uint32_t reconnect_count;
|
||||||
};
|
};
|
||||||
|
|
||||||
Summary GetSummary() const; // thread-safe, blocks fiber
|
Summary GetSummary() const; // thread-safe, blocks fiber, makes a hop.
|
||||||
|
|
||||||
bool HasDflyMaster() const {
|
bool HasDflyMaster() const {
|
||||||
return !master_context_.dfly_session_id.empty();
|
return !master_context_.dfly_session_id.empty();
|
||||||
|
@ -142,6 +143,8 @@ class Replica : ProtocolClient {
|
||||||
util::fb2::EventCount replica_waker_;
|
util::fb2::EventCount replica_waker_;
|
||||||
|
|
||||||
std::vector<std::unique_ptr<DflyShardReplica>> shard_flows_;
|
std::vector<std::unique_ptr<DflyShardReplica>> shard_flows_;
|
||||||
|
std::vector<std::vector<unsigned>> thread_flow_map_; // a map from proactor id to flow list.
|
||||||
|
|
||||||
// A vector of the last executer LSNs when a replication is interrupted.
|
// A vector of the last executer LSNs when a replication is interrupted.
|
||||||
// Allows partial sync on reconnects.
|
// Allows partial sync on reconnects.
|
||||||
std::optional<std::vector<LSN>> last_journal_LSNs_;
|
std::optional<std::vector<LSN>> last_journal_LSNs_;
|
||||||
|
@ -154,7 +157,6 @@ class Replica : ProtocolClient {
|
||||||
// ack_offs_ last acknowledged offset.
|
// ack_offs_ last acknowledged offset.
|
||||||
size_t repl_offs_ = 0, ack_offs_ = 0;
|
size_t repl_offs_ = 0, ack_offs_ = 0;
|
||||||
std::atomic<unsigned> state_mask_ = 0;
|
std::atomic<unsigned> state_mask_ = 0;
|
||||||
unsigned num_df_flows_ = 0;
|
|
||||||
|
|
||||||
bool is_paused_ = false;
|
bool is_paused_ = false;
|
||||||
std::string id_;
|
std::string id_;
|
||||||
|
@ -196,8 +198,11 @@ class DflyShardReplica : public ProtocolClient {
|
||||||
|
|
||||||
uint32_t FlowId() const;
|
uint32_t FlowId() const;
|
||||||
|
|
||||||
uint64_t JournalExecutedCount() const;
|
uint64_t JournalExecutedCount() const {
|
||||||
|
return journal_rec_executed_.load(std::memory_order_relaxed);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Can be called from any thread.
|
||||||
void Pause(bool pause);
|
void Pause(bool pause);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
@ -218,13 +223,12 @@ class DflyShardReplica : public ProtocolClient {
|
||||||
// Note: This is not 1-to-1 the LSN in the master, because this counts
|
// Note: This is not 1-to-1 the LSN in the master, because this counts
|
||||||
// **executed** records, which might be received interleaved when commands
|
// **executed** records, which might be received interleaved when commands
|
||||||
// run out-of-order on the master instance.
|
// run out-of-order on the master instance.
|
||||||
|
// Atomic, because JournalExecutedCount() can be called from any thread.
|
||||||
std::atomic_uint64_t journal_rec_executed_ = 0;
|
std::atomic_uint64_t journal_rec_executed_ = 0;
|
||||||
|
|
||||||
util::fb2::Fiber sync_fb_;
|
util::fb2::Fiber sync_fb_, acks_fb_;
|
||||||
|
|
||||||
util::fb2::Fiber acks_fb_;
|
|
||||||
size_t ack_offs_ = 0;
|
size_t ack_offs_ = 0;
|
||||||
|
int proactor_index_ = -1;
|
||||||
bool force_ping_ = false;
|
bool force_ping_ = false;
|
||||||
|
|
||||||
std::shared_ptr<MultiShardExecution> multi_shard_exe_;
|
std::shared_ptr<MultiShardExecution> multi_shard_exe_;
|
||||||
|
|
|
@ -1261,7 +1261,7 @@ void PrintPrometheusMetrics(const Metrics& m, DflyCmd* dfly_cmd, StringResponse*
|
||||||
if (added)
|
if (added)
|
||||||
absl::StrAppend(&resp->body(), type_used_memory_metric);
|
absl::StrAppend(&resp->body(), type_used_memory_metric);
|
||||||
}
|
}
|
||||||
if (!m.replication_metrics.empty()) {
|
if (!m.master_side_replicas_info.empty()) {
|
||||||
ReplicationMemoryStats repl_mem;
|
ReplicationMemoryStats repl_mem;
|
||||||
dfly_cmd->GetReplicationMemoryStats(&repl_mem);
|
dfly_cmd->GetReplicationMemoryStats(&repl_mem);
|
||||||
AppendMetricWithoutLabels(
|
AppendMetricWithoutLabels(
|
||||||
|
@ -1342,11 +1342,11 @@ void PrintPrometheusMetrics(const Metrics& m, DflyCmd* dfly_cmd, StringResponse*
|
||||||
absl::StrAppend(&resp->body(), command_metrics);
|
absl::StrAppend(&resp->body(), command_metrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!m.replication_metrics.empty()) {
|
if (!m.master_side_replicas_info.empty()) {
|
||||||
string replication_lag_metrics;
|
string replication_lag_metrics;
|
||||||
AppendMetricHeader("connected_replica_lag_records", "Lag in records of a connected replica.",
|
AppendMetricHeader("connected_replica_lag_records", "Lag in records of a connected replica.",
|
||||||
MetricType::GAUGE, &replication_lag_metrics);
|
MetricType::GAUGE, &replication_lag_metrics);
|
||||||
for (const auto& replica : m.replication_metrics) {
|
for (const auto& replica : m.master_side_replicas_info) {
|
||||||
AppendMetricValue("connected_replica_lag_records", replica.lsn_lag,
|
AppendMetricValue("connected_replica_lag_records", replica.lsn_lag,
|
||||||
{"replica_ip", "replica_port", "replica_state"},
|
{"replica_ip", "replica_port", "replica_state"},
|
||||||
{replica.address, absl::StrCat(replica.listening_port), replica.state},
|
{replica.address, absl::StrCat(replica.listening_port), replica.state},
|
||||||
|
@ -1355,14 +1355,10 @@ void PrintPrometheusMetrics(const Metrics& m, DflyCmd* dfly_cmd, StringResponse*
|
||||||
absl::StrAppend(&resp->body(), replication_lag_metrics);
|
absl::StrAppend(&resp->body(), replication_lag_metrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (m.replica_reconnections) {
|
if (m.replica_side_info) {
|
||||||
auto& replica_reconnections = m.replica_reconnections.value();
|
auto& replica_info = *m.replica_side_info;
|
||||||
AppendMetricHeader("replica_reconnect_count", "Number of replica reconnects",
|
AppendMetricWithoutLabels("replica_reconnect_count", "Number of replica reconnects",
|
||||||
MetricType::COUNTER, &resp->body());
|
replica_info.reconnect_count, MetricType::COUNTER, &resp->body());
|
||||||
AppendMetricValue("replica_reconnect_count", replica_reconnections.reconnect_count,
|
|
||||||
{"replica_host", "replica_port"},
|
|
||||||
{replica_reconnections.host, absl::StrCat(replica_reconnections.port)},
|
|
||||||
&resp->body());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
AppendMetricWithoutLabels("fiber_switch_total", "", m.fiber_switch_cnt, MetricType::COUNTER,
|
AppendMetricWithoutLabels("fiber_switch_total", "", m.fiber_switch_cnt, MetricType::COUNTER,
|
||||||
|
@ -2037,7 +2033,7 @@ Metrics ServerFamily::GetMetrics(Namespace* ns) const {
|
||||||
result.delete_ttl_per_sec += shard->GetMovingSum6(EngineShard::TTL_DELETE);
|
result.delete_ttl_per_sec += shard->GetMovingSum6(EngineShard::TTL_DELETE);
|
||||||
if (result.tx_queue_len < shard->txq()->size())
|
if (result.tx_queue_len < shard->txq()->size())
|
||||||
result.tx_queue_len = shard->txq()->size();
|
result.tx_queue_len = shard->txq()->size();
|
||||||
}
|
} // if (shard)
|
||||||
|
|
||||||
result.tls_bytes += Listener::TLSUsedMemoryThreadLocal();
|
result.tls_bytes += Listener::TLSUsedMemoryThreadLocal();
|
||||||
result.refused_conn_max_clients_reached_count += Listener::RefusedConnectionMaxClientsCount();
|
result.refused_conn_max_clients_reached_count += Listener::RefusedConnectionMaxClientsCount();
|
||||||
|
@ -2045,7 +2041,7 @@ Metrics ServerFamily::GetMetrics(Namespace* ns) const {
|
||||||
result.lua_stats += InterpreterManager::tl_stats();
|
result.lua_stats += InterpreterManager::tl_stats();
|
||||||
|
|
||||||
service_.mutable_registry()->MergeCallStats(index, cmd_stat_cb);
|
service_.mutable_registry()->MergeCallStats(index, cmd_stat_cb);
|
||||||
};
|
}; // cb
|
||||||
|
|
||||||
service_.proactor_pool().AwaitFiberOnAll(std::move(cb));
|
service_.proactor_pool().AwaitFiberOnAll(std::move(cb));
|
||||||
|
|
||||||
|
@ -2056,11 +2052,13 @@ Metrics ServerFamily::GetMetrics(Namespace* ns) const {
|
||||||
|
|
||||||
bool is_master = ServerState::tlocal() && ServerState::tlocal()->is_master;
|
bool is_master = ServerState::tlocal() && ServerState::tlocal()->is_master;
|
||||||
if (is_master) {
|
if (is_master) {
|
||||||
result.replication_metrics = dfly_cmd_->GetReplicasRoleInfo();
|
result.master_side_replicas_info = dfly_cmd_->GetReplicasRoleInfo();
|
||||||
} else {
|
} else {
|
||||||
auto info = GetReplicaSummary();
|
auto info = GetReplicaSummary();
|
||||||
if (info) {
|
if (info) {
|
||||||
result.replica_reconnections = {std::move(info->host), info->port, info->reconnect_count};
|
result.replica_side_info = {
|
||||||
|
.reconnect_count = info->reconnect_count,
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2197,7 +2195,7 @@ void ServerFamily::Info(CmdArgList args, ConnectionContext* cntx) {
|
||||||
append("maxmemory_policy", "noeviction");
|
append("maxmemory_policy", "noeviction");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!m.replication_metrics.empty()) {
|
if (!m.master_side_replicas_info.empty()) {
|
||||||
ReplicationMemoryStats repl_mem;
|
ReplicationMemoryStats repl_mem;
|
||||||
dfly_cmd_->GetReplicationMemoryStats(&repl_mem);
|
dfly_cmd_->GetReplicationMemoryStats(&repl_mem);
|
||||||
append("replication_streaming_buffer_bytes", repl_mem.streamer_buf_capacity_bytes);
|
append("replication_streaming_buffer_bytes", repl_mem.streamer_buf_capacity_bytes);
|
||||||
|
@ -2361,7 +2359,7 @@ void ServerFamily::Info(CmdArgList args, ConnectionContext* cntx) {
|
||||||
if (!replica_) {
|
if (!replica_) {
|
||||||
append("role", "master");
|
append("role", "master");
|
||||||
append("connected_slaves", m.facade_stats.conn_stats.num_replicas);
|
append("connected_slaves", m.facade_stats.conn_stats.num_replicas);
|
||||||
const auto& replicas = m.replication_metrics;
|
const auto& replicas = m.master_side_replicas_info;
|
||||||
for (size_t i = 0; i < replicas.size(); i++) {
|
for (size_t i = 0; i < replicas.size(); i++) {
|
||||||
auto& r = replicas[i];
|
auto& r = replicas[i];
|
||||||
// e.g. slave0:ip=172.19.0.3,port=6379,state=full_sync
|
// e.g. slave0:ip=172.19.0.3,port=6379,state=full_sync
|
||||||
|
@ -2372,7 +2370,7 @@ void ServerFamily::Info(CmdArgList args, ConnectionContext* cntx) {
|
||||||
} else {
|
} else {
|
||||||
append("role", GetFlag(FLAGS_info_replication_valkey_compatible) ? "slave" : "replica");
|
append("role", GetFlag(FLAGS_info_replication_valkey_compatible) ? "slave" : "replica");
|
||||||
|
|
||||||
auto replication_info_cb = [&](Replica::Summary rinfo) {
|
auto replication_info_cb = [&](const Replica::Summary& rinfo) {
|
||||||
append("master_host", rinfo.host);
|
append("master_host", rinfo.host);
|
||||||
append("master_port", rinfo.port);
|
append("master_port", rinfo.port);
|
||||||
|
|
||||||
|
@ -2385,6 +2383,8 @@ void ServerFamily::Info(CmdArgList args, ConnectionContext* cntx) {
|
||||||
append("slave_read_only", 1);
|
append("slave_read_only", 1);
|
||||||
};
|
};
|
||||||
replication_info_cb(replica_->GetSummary());
|
replication_info_cb(replica_->GetSummary());
|
||||||
|
|
||||||
|
// Special case, when multiple masters replicate to a single replica.
|
||||||
for (const auto& replica : cluster_replicas_) {
|
for (const auto& replica : cluster_replicas_) {
|
||||||
replication_info_cb(replica->GetSummary());
|
replication_info_cb(replica->GetSummary());
|
||||||
}
|
}
|
||||||
|
@ -2700,8 +2700,6 @@ void ServerFamily::Replicate(string_view host, string_view port) {
|
||||||
void ServerFamily::ReplTakeOver(CmdArgList args, ConnectionContext* cntx) {
|
void ServerFamily::ReplTakeOver(CmdArgList args, ConnectionContext* cntx) {
|
||||||
VLOG(1) << "ReplTakeOver start";
|
VLOG(1) << "ReplTakeOver start";
|
||||||
|
|
||||||
util::fb2::LockGuard lk(replicaof_mu_);
|
|
||||||
|
|
||||||
CmdArgParser parser{args};
|
CmdArgParser parser{args};
|
||||||
|
|
||||||
int timeout_sec = parser.Next<int>();
|
int timeout_sec = parser.Next<int>();
|
||||||
|
@ -2722,6 +2720,8 @@ void ServerFamily::ReplTakeOver(CmdArgList args, ConnectionContext* cntx) {
|
||||||
if (ServerState::tlocal()->is_master)
|
if (ServerState::tlocal()->is_master)
|
||||||
return cntx->SendOk();
|
return cntx->SendOk();
|
||||||
|
|
||||||
|
util::fb2::LockGuard lk(replicaof_mu_);
|
||||||
|
|
||||||
auto repl_ptr = replica_;
|
auto repl_ptr = replica_;
|
||||||
CHECK(repl_ptr);
|
CHECK(repl_ptr);
|
||||||
|
|
||||||
|
@ -2735,8 +2735,7 @@ void ServerFamily::ReplTakeOver(CmdArgList args, ConnectionContext* cntx) {
|
||||||
return cntx->SendError("Couldn't execute takeover");
|
return cntx->SendError("Couldn't execute takeover");
|
||||||
|
|
||||||
LOG(INFO) << "Takeover successful, promoting this instance to master.";
|
LOG(INFO) << "Takeover successful, promoting this instance to master.";
|
||||||
service_.proactor_pool().AwaitFiberOnAll(
|
SetMasterFlagOnAllThreads(true);
|
||||||
[&](util::ProactorBase* pb) { ServerState::tlocal()->is_master = true; });
|
|
||||||
replica_->Stop();
|
replica_->Stop();
|
||||||
replica_.reset();
|
replica_.reset();
|
||||||
return cntx->SendOk();
|
return cntx->SendOk();
|
||||||
|
|
|
@ -67,12 +67,6 @@ struct ReplicationMemoryStats {
|
||||||
size_t full_sync_buf_bytes = 0; // total bytes used for full sync buffers
|
size_t full_sync_buf_bytes = 0; // total bytes used for full sync buffers
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ReplicaReconnectionsInfo {
|
|
||||||
std::string host;
|
|
||||||
uint16_t port;
|
|
||||||
uint32_t reconnect_count;
|
|
||||||
};
|
|
||||||
|
|
||||||
struct LoadingStats {
|
struct LoadingStats {
|
||||||
size_t restore_count = 0;
|
size_t restore_count = 0;
|
||||||
size_t failed_restore_count = 0;
|
size_t failed_restore_count = 0;
|
||||||
|
@ -128,8 +122,16 @@ struct Metrics {
|
||||||
|
|
||||||
// command call frequencies (count, aggregated latency in usec).
|
// command call frequencies (count, aggregated latency in usec).
|
||||||
std::map<std::string, std::pair<uint64_t, uint64_t>> cmd_stats_map;
|
std::map<std::string, std::pair<uint64_t, uint64_t>> cmd_stats_map;
|
||||||
std::vector<ReplicaRoleInfo> replication_metrics;
|
|
||||||
std::optional<ReplicaReconnectionsInfo> replica_reconnections;
|
// Replica info on the master side.
|
||||||
|
std::vector<ReplicaRoleInfo> master_side_replicas_info;
|
||||||
|
|
||||||
|
struct ReplicaInfo {
|
||||||
|
uint32_t reconnect_count;
|
||||||
|
};
|
||||||
|
|
||||||
|
// Replica reconnect stats on the replica side. Undefined for master
|
||||||
|
std::optional<ReplicaInfo> replica_side_info;
|
||||||
|
|
||||||
LoadingStats loading_stats;
|
LoadingStats loading_stats;
|
||||||
};
|
};
|
||||||
|
|
|
@ -17,9 +17,6 @@ const char* GetVersion();
|
||||||
// Please document for each new entry what the behavior changes are
|
// Please document for each new entry what the behavior changes are
|
||||||
// and to which released versions this corresponds.
|
// and to which released versions this corresponds.
|
||||||
enum class DflyVersion {
|
enum class DflyVersion {
|
||||||
// ver <= 1.3
|
|
||||||
VER0,
|
|
||||||
|
|
||||||
// 1.4 <= ver <= 1.10
|
// 1.4 <= ver <= 1.10
|
||||||
// - Supports receiving ACKs from replicas
|
// - Supports receiving ACKs from replicas
|
||||||
// - Sends version back on REPLCONF capa dragonfly
|
// - Sends version back on REPLCONF capa dragonfly
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue