// Copyright 2022, DragonflyDB authors. All rights reserved. // See LICENSE for licensing terms. // #include "server/server_family.h" #include #include // for master_replid_ generation. #include #include #include #include #include // cron::cronexpr #include #include #include #include #include #include #include "absl/strings/ascii.h" #include "facade/error.h" #include "slowlog.h" #include "util/fibers/synchronization.h" extern "C" { #include "redis/redis_aux.h" } #include "base/flags.h" #include "base/logging.h" #include "core/compact_object.h" #include "facade/cmd_arg_parser.h" #include "facade/dragonfly_connection.h" #include "facade/reply_builder.h" #include "io/file_util.h" #include "io/proc_reader.h" #include "search/doc_index.h" #include "server/acl/acl_commands_def.h" #include "server/command_registry.h" #include "server/conn_context.h" #include "server/debugcmd.h" #include "server/detail/save_stages_controller.h" #include "server/detail/snapshot_storage.h" #include "server/dflycmd.h" #include "server/engine_shard_set.h" #include "server/error.h" #include "server/generic_family.h" #include "server/journal/journal.h" #include "server/main_service.h" #include "server/memory_cmd.h" #include "server/multi_command_squasher.h" #include "server/protocol_client.h" #include "server/rdb_load.h" #include "server/rdb_save.h" #include "server/script_mgr.h" #include "server/server_state.h" #include "server/snapshot.h" #include "server/tiered_storage.h" #include "server/transaction.h" #include "server/version.h" #include "strings/human_readable.h" #include "util/accept_server.h" #include "util/aws/aws.h" using namespace std; struct ReplicaOfFlag { string host; string port; bool has_value() const { return !host.empty() && !port.empty(); } }; static bool AbslParseFlag(std::string_view in, ReplicaOfFlag* flag, std::string* err); static std::string AbslUnparseFlag(const ReplicaOfFlag& flag); struct CronExprFlag { static constexpr std::string_view kCronPrefix = "0 "sv; std::optional cron_expr; }; static bool AbslParseFlag(std::string_view in, CronExprFlag* flag, std::string* err); static std::string AbslUnparseFlag(const CronExprFlag& flag); ABSL_FLAG(string, dir, "", "working directory"); ABSL_FLAG(string, dbfilename, "dump-{timestamp}", "the filename to save/load the DB, instead of/with {timestamp} can be used {Y}, {m}, and " "{d} macros"); ABSL_FLAG(string, requirepass, "", "password for AUTH authentication. " "If empty can also be set with DFLY_PASSWORD environment variable."); ABSL_FLAG(uint32_t, maxclients, 64000, "Maximum number of concurrent clients allowed."); ABSL_FLAG(string, save_schedule, "", "the flag is deprecated, please use snapshot_cron instead"); ABSL_FLAG(CronExprFlag, snapshot_cron, {}, "cron expression for the time to save a snapshot, crontab style"); ABSL_FLAG(bool, df_snapshot_format, true, "if true, save in dragonfly-specific snapshotting format"); ABSL_FLAG(int, epoll_file_threads, 0, "thread size for file workers when running in epoll mode, default is hardware concurrent " "threads"); ABSL_FLAG(ReplicaOfFlag, replicaof, ReplicaOfFlag{}, "Specifies a host and port which point to a target master " "to replicate. " "Format should be : or host: or []:"); ABSL_FLAG(int32_t, slowlog_log_slower_than, 10000, "Add commands slower than this threshold to slow log. The value is expressed in " "microseconds and if it's negative - disables the slowlog."); ABSL_FLAG(uint32_t, slowlog_max_len, 20, "Slow log maximum length."); ABSL_FLAG(string, s3_endpoint, "", "endpoint for s3 snapshots, default uses aws regional endpoint"); ABSL_FLAG(bool, s3_use_https, true, "whether to use https for s3 endpoints"); // Disable EC2 metadata by default, or if a users credentials are invalid the // AWS client will spent 30s trying to connect to inaccessable EC2 endpoints // to load the credentials. ABSL_FLAG(bool, s3_ec2_metadata, false, "whether to load credentials and configuration from EC2 metadata"); // Enables S3 payload signing over HTTP. This reduces the latency and resource // usage when writing snapshots to S3, at the expense of security. ABSL_FLAG(bool, s3_sign_payload, true, "whether to sign the s3 request payload when uploading snapshots"); ABSL_FLAG(bool, info_replication_valkey_compatible, true, "when true - output valkey compatible values for info-replication"); ABSL_FLAG(bool, managed_service_info, false, "Hides some implementation details from users when true (i.e. in managed service env)"); ABSL_FLAG(string, availability_zone, "", "server availability zone, used by clients to read from local-zone replicas"); ABSL_DECLARE_FLAG(int32_t, port); ABSL_DECLARE_FLAG(bool, cache_mode); ABSL_DECLARE_FLAG(uint32_t, hz); ABSL_DECLARE_FLAG(bool, tls); ABSL_DECLARE_FLAG(string, tls_ca_cert_file); ABSL_DECLARE_FLAG(string, tls_ca_cert_dir); ABSL_DECLARE_FLAG(int, replica_priority); ABSL_DECLARE_FLAG(double, rss_oom_deny_ratio); ABSL_DECLARE_FLAG(uint32_t, memcached_port); bool AbslParseFlag(std::string_view in, ReplicaOfFlag* flag, std::string* err) { #define RETURN_ON_ERROR(cond, m) \ do { \ if ((cond)) { \ *err = m; \ LOG(WARNING) << "Error in parsing arguments for --replicaof: " << m; \ return false; \ } \ } while (0) if (in.empty()) { // on empty flag "parse" nothing. If we return false then DF exists. *flag = ReplicaOfFlag{}; return true; } auto pos = in.find_last_of(':'); RETURN_ON_ERROR(pos == string::npos, "missing ':'."); string_view ip = in.substr(0, pos); flag->port = in.substr(pos + 1); RETURN_ON_ERROR(ip.empty() || flag->port.empty(), "IP/host or port are empty."); // For IPv6: ip1.front == '[' AND ip1.back == ']' // For IPv4: ip1.front != '[' AND ip1.back != ']' // Together, this ip1.front == '[' iff ip1.back == ']', which can be implemented as XNOR (NOT XOR) RETURN_ON_ERROR(((ip.front() == '[') ^ (ip.back() == ']')), "unclosed brackets."); if (ip.front() == '[') { // shortest possible IPv6 is '::1' (loopback) RETURN_ON_ERROR(ip.length() <= 2, "IPv6 host name is too short"); flag->host = ip.substr(1, ip.length() - 2); } else { flag->host = ip; } VLOG(1) << "--replicaof: Received " << flag->host << " : " << flag->port; return true; #undef RETURN_ON_ERROR } std::string AbslUnparseFlag(const ReplicaOfFlag& flag) { return (flag.has_value()) ? absl::StrCat(flag.host, ":", flag.port) : ""; } bool AbslParseFlag(std::string_view in, CronExprFlag* flag, std::string* err) { if (in.empty()) { flag->cron_expr = std::nullopt; return true; } if (absl::StartsWith(in, "\"")) { *err = absl::StrCat("Could it be that you put quotes in the flagfile?"); return false; } std::string raw_cron_expr = absl::StrCat(CronExprFlag::kCronPrefix, in); try { VLOG(1) << "creating cron from: '" << raw_cron_expr << "'"; flag->cron_expr = cron::make_cron(raw_cron_expr); return true; } catch (const cron::bad_cronexpr& ex) { *err = ex.what(); } return false; } std::string AbslUnparseFlag(const CronExprFlag& flag) { if (flag.cron_expr) { auto str_expr = to_cronstr(*flag.cron_expr); DCHECK(absl::StartsWith(str_expr, CronExprFlag::kCronPrefix)); return str_expr.substr(CronExprFlag::kCronPrefix.size()); } return ""; } namespace dfly { namespace fs = std::filesystem; using absl::GetFlag; using absl::StrCat; using namespace facade; using namespace util; using detail::SaveStagesController; using http::StringResponse; using strings::HumanReadableNumBytes; namespace { const auto kRedisVersion = "7.4.0"; using EngineFunc = void (ServerFamily::*)(CmdArgList args, const CommandContext&); inline CommandId::Handler3 HandlerFunc(ServerFamily* se, EngineFunc f) { return [=](CmdArgList args, const CommandContext& cntx) { return (se->*f)(args, cntx); }; } using CI = CommandId; struct CmdArgListFormatter { void operator()(std::string* out, MutableSlice arg) const { out->append(absl::StrCat("`", std::string_view(arg.data(), arg.size()), "`")); } }; string UnknownCmd(string cmd, CmdArgList args) { return absl::StrCat("unknown command '", cmd, "' with args beginning with: ", absl::StrJoin(args.begin(), args.end(), ", ", CmdArgListFormatter())); } std::shared_ptr CreateCloudSnapshotStorage(std::string_view uri) { if (detail::IsS3Path(uri)) { #ifdef WITH_AWS shard_set->pool()->GetNextProactor()->Await([&] { util::aws::Init(); }); return std::make_shared( absl::GetFlag(FLAGS_s3_endpoint), absl::GetFlag(FLAGS_s3_use_https), absl::GetFlag(FLAGS_s3_ec2_metadata), absl::GetFlag(FLAGS_s3_sign_payload)); #else LOG(ERROR) << "Compiled without AWS support"; exit(1); #endif } else if (detail::IsGCSPath(uri)) { auto gcs = std::make_shared(); auto ec = shard_set->pool()->GetNextProactor()->Await([&] { return gcs->Init(3000); }); if (ec) { LOG(ERROR) << "Failed to initialize GCS snapshot storage: " << ec.message(); exit(1); } return gcs; } else { LOG(ERROR) << "Uknown cloud storage " << uri; exit(1); } } // Check that if TLS is used at least one form of client authentication is // enabled. That means either using a password or giving a root // certificate for authenticating client certificates which will // be required. bool ValidateServerTlsFlags() { if (!absl::GetFlag(FLAGS_tls)) { return true; } bool has_auth = false; if (!dfly::GetPassword().empty()) { has_auth = true; } if (!(absl::GetFlag(FLAGS_tls_ca_cert_file).empty() && absl::GetFlag(FLAGS_tls_ca_cert_dir).empty())) { has_auth = true; } // Allow TLS without authentication for memcached protocol // We check if memcached_port is enabled, as this is a static check during startup if (GetFlag(FLAGS_memcached_port) > 0) { has_auth = true; } if (!has_auth) { LOG(ERROR) << "TLS configured but no authentication method is used!"; return false; } return true; } template void UpdateMax(T* maxv, T current) { *maxv = std::max(*maxv, current); } void SetMasterFlagOnAllThreads(bool is_master) { auto cb = [is_master](unsigned, auto*) { ServerState::tlocal()->is_master = is_master; }; shard_set->pool()->AwaitBrief(cb); } std::optional InferSnapshotCronExpr() { string save_time = GetFlag(FLAGS_save_schedule); auto cron_expr = GetFlag(FLAGS_snapshot_cron); if (!save_time.empty()) { LOG(ERROR) << "save_schedule flag is deprecated, please use snapshot_cron instead"; exit(1); } if (cron_expr.cron_expr) { return std::move(cron_expr.cron_expr); } return std::nullopt; } void ClientSetName(CmdArgList args, SinkReplyBuilder* builder, ConnectionContext* cntx) { if (args.size() == 1) { cntx->conn()->SetName(string{ArgS(args, 0)}); return builder->SendOk(); } else { return builder->SendError(facade::kSyntaxErr); } } void ClientGetName(CmdArgList args, SinkReplyBuilder* builder, ConnectionContext* cntx) { if (!args.empty()) { return builder->SendError(facade::kSyntaxErr); } auto* rb = static_cast(builder); if (auto name = cntx->conn()->GetName(); !name.empty()) { return rb->SendBulkString(name); } else { return rb->SendNull(); } } void ClientList(CmdArgList args, absl::Span listeners, SinkReplyBuilder* builder, ConnectionContext* cntx) { if (!args.empty()) { return builder->SendError(facade::kSyntaxErr); } vector client_info; absl::base_internal::SpinLock mu; // we can not preempt the connection traversal, so we need to use a spinlock. // alternatively we could lock when mutating the connection list, but it seems not important. auto cb = [&](unsigned thread_index, util::Connection* conn) { facade::Connection* dcon = static_cast(conn); string info = dcon->GetClientInfo(thread_index); absl::base_internal::SpinLockHolder l(&mu); client_info.push_back(std::move(info)); }; for (auto* listener : listeners) { listener->TraverseConnections(cb); } string result = absl::StrJoin(client_info, "\n"); result.append("\n"); auto* rb = static_cast(builder); return rb->SendVerbatimString(result); } void ClientTracking(CmdArgList args, SinkReplyBuilder* builder, ConnectionContext* cntx) { auto* rb = static_cast(builder); if (!rb->IsResp3()) return builder->SendError( "Client tracking is currently not supported for RESP2. Please use RESP3."); CmdArgParser parser{args}; if (!parser.HasAtLeast(1) || args.size() > 3) return builder->SendError(kSyntaxErr); bool is_on = false; using Tracking = ConnectionState::ClientTracking; Tracking::Options option = Tracking::NONE; if (parser.Check("ON")) { is_on = true; } else if (!parser.Check("OFF")) { return builder->SendError(kSyntaxErr); } bool noloop = false; if (parser.HasNext()) { if (parser.Check("OPTIN")) { option = Tracking::OPTIN; } else if (parser.Check("OPTOUT")) { option = Tracking::OPTOUT; } else if (parser.Check("NOLOOP")) { noloop = true; } else { return builder->SendError(kSyntaxErr); } } if (parser.HasNext()) { if (!noloop && parser.Check("NOLOOP")) { noloop = true; } else { return builder->SendError(kSyntaxErr); } } if (is_on) { ++cntx->subscriptions; } cntx->conn_state.tracking_info_.SetClientTracking(is_on); cntx->conn_state.tracking_info_.SetOption(option); cntx->conn_state.tracking_info_.SetNoLoop(noloop); return builder->SendOk(); } void ClientCaching(CmdArgList args, SinkReplyBuilder* builder, Transaction* tx, ConnectionContext* cntx) { auto* rb = static_cast(builder); if (!rb->IsResp3()) return builder->SendError( "Client caching is currently not supported for RESP2. Please use RESP3."); if (args.size() != 1) { return builder->SendError(kSyntaxErr); } using Tracking = ConnectionState::ClientTracking; CmdArgParser parser{args}; if (parser.Check("YES")) { if (!cntx->conn_state.tracking_info_.HasOption(Tracking::OPTIN)) { return builder->SendError( "ERR CLIENT CACHING YES is only valid when tracking is enabled in OPTIN mode"); } } else if (parser.Check("NO")) { if (!cntx->conn_state.tracking_info_.HasOption(Tracking::OPTOUT)) { return builder->SendError( "ERR CLIENT CACHING NO is only valid when tracking is enabled in OPTOUT mode"); } cntx->conn_state.tracking_info_.ResetCachingSequenceNumber(); } else { return builder->SendError(kSyntaxErr); } bool is_multi = tx && tx->IsMulti(); cntx->conn_state.tracking_info_.SetCachingSequenceNumber(is_multi); builder->SendOk(); } void ClientSetInfo(CmdArgList args, SinkReplyBuilder* builder, ConnectionContext* cntx) { if (args.size() != 2) { return builder->SendError(kSyntaxErr); } auto* conn = cntx->conn(); if (conn == nullptr) { return builder->SendError("No connection"); } string type = absl::AsciiStrToUpper(ArgS(args, 0)); string_view val = ArgS(args, 1); if (type == "LIB-NAME") { conn->SetLibName(string(val)); } else if (type == "LIB-VER") { conn->SetLibVersion(string(val)); } else { return builder->SendError(kSyntaxErr); } builder->SendOk(); } void ClientId(CmdArgList args, SinkReplyBuilder* builder, ConnectionContext* cntx) { if (args.size() != 0) { return builder->SendError(kSyntaxErr); } return builder->SendLong(cntx->conn()->GetClientId()); } void ClientKill(CmdArgList args, absl::Span listeners, SinkReplyBuilder* builder, ConnectionContext* cntx) { std::function evaluator; if (args.size() == 1) { string_view ip_port = ArgS(args, 0); if (ip_port.find(':') != ip_port.npos) { evaluator = [ip_port](facade::Connection* conn) { return conn->RemoteEndpointStr() == ip_port; }; } } else if (args.size() == 2) { string filter_type = absl::AsciiStrToUpper(ArgS(args, 0)); string_view filter_value = ArgS(args, 1); if (filter_type == "ADDR") { evaluator = [filter_value](facade::Connection* conn) { return conn->RemoteEndpointStr() == filter_value; }; } else if (filter_type == "LADDR") { evaluator = [filter_value](facade::Connection* conn) { return conn->LocalBindStr() == filter_value; }; } else if (filter_type == "ID") { uint32_t id; if (absl::SimpleAtoi(filter_value, &id)) { evaluator = [id](facade::Connection* conn) { return conn->GetClientId() == id; }; } } // TODO: Add support for KILL USER/TYPE/SKIPME } if (!evaluator) { return builder->SendError(kSyntaxErr); } const bool is_admin_request = cntx->conn()->IsPrivileged(); atomic killed_connections = 0; atomic kill_errors = 0; auto cb = [&](unsigned thread_index, util::Connection* conn) { facade::Connection* dconn = static_cast(conn); if (evaluator(dconn)) { if (is_admin_request || !dconn->IsPrivileged()) { dconn->ShutdownSelf(); killed_connections.fetch_add(1); } else { kill_errors.fetch_add(1); } } }; for (auto* listener : listeners) { listener->TraverseConnections(cb); } if (kill_errors.load() == 0) { return builder->SendLong(killed_connections.load()); } else { return builder->SendError(absl::StrCat("Killed ", killed_connections.load(), " client(s), but unable to kill ", kill_errors.load(), " admin client(s).")); } } std::string_view GetOSString() { // Call uname() only once since it can be expensive. Cache the final result in a static string. static string os_string = []() { utsname os_name; uname(&os_name); return StrCat(os_name.sysname, " ", os_name.release, " ", os_name.machine); }(); return os_string; } string_view GetRedisMode() { return IsClusterEnabledOrEmulated() ? "cluster"sv : "standalone"sv; } struct ReplicaOfArgs { string host; uint16_t port; std::optional slot_range; static optional FromCmdArgs(CmdArgList args, SinkReplyBuilder* builder); bool IsReplicaOfNoOne() const { return port == 0; } friend std::ostream& operator<<(std::ostream& os, const ReplicaOfArgs& args) { if (args.IsReplicaOfNoOne()) { return os << "NO ONE"; } os << args.host << ":" << args.port; if (args.slot_range.has_value()) { os << " SLOTS [" << args.slot_range.value().start << "-" << args.slot_range.value().end << "]"; } return os; } }; optional ReplicaOfArgs::FromCmdArgs(CmdArgList args, SinkReplyBuilder* builder) { ReplicaOfArgs replicaof_args; CmdArgParser parser(args); if (parser.Check("NO")) { parser.ExpectTag("ONE"); replicaof_args.port = 0; } else { replicaof_args.host = parser.Next(); replicaof_args.port = parser.Next(); if (auto err = parser.Error(); err || replicaof_args.port < 1) { builder->SendError("port is out of range"); return nullopt; } if (parser.HasNext()) { auto [slot_start, slot_end] = parser.Next(); replicaof_args.slot_range = cluster::SlotRange{slot_start, slot_end}; if (auto err = parser.Error(); err || !replicaof_args.slot_range->IsValid()) { builder->SendError("Invalid slot range"); return nullopt; } } } if (auto err = parser.Error(); err) { builder->SendError(err->MakeReply()); return nullopt; } return replicaof_args; } uint64_t GetDelayMs(uint64_t ts) { uint64_t now_ns = fb2::ProactorBase::GetMonotonicTimeNs(); uint64_t delay_ns = 0; if (ts < now_ns - 1000000) { // if more than 1ms has passed between ts and now_ns delay_ns = (now_ns - ts) / 1000000; } return delay_ns; } } // namespace void SlowLogGet(dfly::CmdArgList args, std::string_view sub_cmd, util::ProactorPool* pp, SinkReplyBuilder* builder) { size_t requested_slow_log_length = UINT32_MAX; size_t argc = args.size(); if (argc >= 3) { builder->SendError(facade::UnknownSubCmd(sub_cmd, "SLOWLOG"), facade::kSyntaxErrType); return; } else if (argc == 2) { string_view length = facade::ArgS(args, 1); int64_t num; if ((!absl::SimpleAtoi(length, &num)) || (num < -1)) { builder->SendError("count should be greater than or equal to -1"); return; } if (num >= 0) { requested_slow_log_length = num; } } // gather all the individual slowlogs from all the fibers and sort them by their timestamp std::vector> entries(pp->size()); pp->AwaitFiberOnAll([&](auto index, auto* context) { auto shard_entries = ServerState::tlocal()->GetSlowLog().Entries(); entries[index] = shard_entries; }); std::vector> merged_slow_log; for (size_t i = 0; i < entries.size(); ++i) { for (const auto& log_item : entries[i]) { merged_slow_log.emplace_back(log_item, i); } } std::sort(merged_slow_log.begin(), merged_slow_log.end(), [](const auto& e1, const auto& e2) { return e1.first.unix_ts_usec > e2.first.unix_ts_usec; }); requested_slow_log_length = std::min(merged_slow_log.size(), requested_slow_log_length); auto* rb = static_cast(builder); rb->StartArray(requested_slow_log_length); for (size_t i = 0; i < requested_slow_log_length; ++i) { const auto& entry = merged_slow_log[i].first; const auto& args = entry.cmd_args; rb->StartArray(6); rb->SendLong(entry.entry_id * pp->size() + merged_slow_log[i].second); rb->SendLong(entry.unix_ts_usec / 1000000); rb->SendLong(entry.exec_time_usec); // if we truncated the args, there is one pseudo-element containing the number of truncated // args that we must add, so the result length is increased by 1 size_t len = args.size() + int(args.size() < entry.original_length); rb->StartArray(len); for (const auto& arg : args) { if (arg.second > 0) { auto suffix = absl::StrCat("... (", arg.second, " more bytes)"); auto cmd_arg = arg.first.substr(0, kMaximumSlowlogArgLength - suffix.length()); rb->SendBulkString(absl::StrCat(cmd_arg, suffix)); } else { rb->SendBulkString(arg.first); } } // if we truncated arguments - add a special string to indicate that. if (args.size() < entry.original_length) { rb->SendBulkString( absl::StrCat("... (", entry.original_length - args.size(), " more arguments)")); } rb->SendBulkString(entry.client_ip); rb->SendBulkString(entry.client_name); } } std::optional Pause(std::vector listeners, Namespace* ns, facade::Connection* conn, ClientPause pause_state, std::function is_pause_in_progress, std::function maybe_cleanup) { // Track connections and set pause state to be able to wait untill all running transactions read // the new pause state. Exlude already paused commands from the busy count. Exlude tracking // blocked connections because: a) If the connection is blocked it is puased. b) We read pause // state after waking from blocking so if the trasaction was waken by another running // command that did not pause on the new state yet we will pause after waking up. DispatchTracker tracker{std::move(listeners), conn, true /* ignore paused commands */, true /*ignore blocking*/}; shard_set->pool()->AwaitFiberOnAll([&tracker, pause_state](unsigned, util::ProactorBase*) { // 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); }); // 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 (!tracker.Wait(kDispatchTimeout)) { LOG(WARNING) << "Couldn't wait for commands to finish dispatching in " << kDispatchTimeout; shard_set->pool()->AwaitBrief([pause_state](unsigned, util::ProactorBase*) { ServerState::tlocal()->SetPauseState(pause_state, false); }); return std::nullopt; } // We should not expire/evict keys while clients are paused. shard_set->RunBriefInParallel( [ns](EngineShard* shard) { ns->GetDbSlice(shard->shard_id()).SetExpireAllowed(false); }); return fb2::Fiber("client_pause", [is_pause_in_progress, pause_state, ns, maybe_cleanup]() mutable { // On server shutdown we sleep 10ms to make sure all running task finish, // therefore 10ms steps ensure this fiber will not left hanging . constexpr auto step = 10ms; while (is_pause_in_progress()) { ThisFiber::SleepFor(step); } ServerState& etl = *ServerState::tlocal(); if (etl.gstate() != GlobalState::SHUTTING_DOWN) { shard_set->pool()->AwaitFiberOnAll([pause_state](util::ProactorBase* pb) { ServerState::tlocal()->SetPauseState(pause_state, false); }); shard_set->RunBriefInParallel([ns](EngineShard* shard) { ns->GetDbSlice(shard->shard_id()).SetExpireAllowed(true); }); } if (maybe_cleanup) { maybe_cleanup(); } }); } ServerFamily::ServerFamily(Service* service) : service_(*service) { start_time_ = time(NULL); last_save_info_.save_time = start_time_; script_mgr_.reset(new ScriptMgr()); journal_.reset(new journal::Journal()); { absl::InsecureBitGen eng; master_replid_ = GetRandomHex(eng, CONFIG_RUN_ID_SIZE); DCHECK_EQ(CONFIG_RUN_ID_SIZE, master_replid_.size()); } if (auto ec = detail::ValidateFilename(GetFlag(FLAGS_dbfilename), GetFlag(FLAGS_df_snapshot_format)); ec) { LOG(ERROR) << ec.Format(); exit(1); } if (!ValidateServerTlsFlags()) { exit(1); } ValidateClientTlsFlags(); dfly_cmd_ = make_unique(this); } ServerFamily::~ServerFamily() { } void SetMaxClients(std::vector& listeners, uint32_t maxclients) { for (auto* listener : listeners) { if (!listener->IsPrivilegedInterface()) { listener->socket()->proactor()->Await( [listener, maxclients]() { listener->SetMaxClients(maxclients); }); } } } void SetSlowLogMaxLen(util::ProactorPool& pool, uint32_t val) { pool.AwaitFiberOnAll( [&val](auto index, auto* context) { ServerState::tlocal()->GetSlowLog().ChangeLength(val); }); } void SetSlowLogThreshold(util::ProactorPool& pool, int32_t val) { pool.AwaitFiberOnAll([val](auto index, auto* context) { ServerState::tlocal()->log_slower_than_usec = val < 0 ? UINT32_MAX : uint32_t(val); }); } void ServerFamily::Init(util::AcceptServer* acceptor, std::vector listeners) { CHECK(acceptor_ == nullptr); acceptor_ = acceptor; listeners_ = std::move(listeners); auto os_string = GetOSString(); LOG_FIRST_N(INFO, 1) << "Host OS: " << os_string << " with " << shard_set->pool()->size() << " threads"; SetMaxClients(listeners_, absl::GetFlag(FLAGS_maxclients)); config_registry.RegisterSetter( "maxclients", [this](uint32_t val) { SetMaxClients(listeners_, val); }); SetSlowLogThreshold(service_.proactor_pool(), absl::GetFlag(FLAGS_slowlog_log_slower_than)); config_registry.RegisterMutable("slowlog_log_slower_than", [this](const absl::CommandLineFlag& flag) { auto res = flag.TryGet(); if (res.has_value()) SetSlowLogThreshold(service_.proactor_pool(), res.value()); return res.has_value(); }); SetSlowLogMaxLen(service_.proactor_pool(), absl::GetFlag(FLAGS_slowlog_max_len)); config_registry.RegisterSetter( "slowlog_max_len", [this](uint32_t val) { SetSlowLogMaxLen(service_.proactor_pool(), val); }); // We only reconfigure TLS when the 'tls' config key changes. Therefore to // update TLS certs, first update tls_cert_file, then set 'tls true'. config_registry.RegisterMutable("tls", [this](const absl::CommandLineFlag& flag) { if (!ValidateServerTlsFlags()) { return false; } for (facade::Listener* l : listeners_) { // Must reconfigure in the listener proactor to avoid a race. if (!l->socket()->proactor()->Await([l] { return l->ReconfigureTLS(); })) { return false; } } return true; }); config_registry.RegisterMutable("tls_cert_file"); config_registry.RegisterMutable("tls_key_file"); config_registry.RegisterMutable("tls_ca_cert_file"); config_registry.RegisterMutable("tls_ca_cert_dir"); config_registry.RegisterMutable("replica_priority"); config_registry.RegisterMutable("lua_undeclared_keys_shas"); pb_task_ = shard_set->pool()->GetNextProactor(); if (pb_task_->GetKind() == ProactorBase::EPOLL) { fq_threadpool_.reset(new fb2::FiberQueueThreadPool(absl::GetFlag(FLAGS_epoll_file_threads))); } string flag_dir = GetFlag(FLAGS_dir); if (detail::IsCloudPath(flag_dir)) { snapshot_storage_ = CreateCloudSnapshotStorage(flag_dir); } else if (fq_threadpool_) { snapshot_storage_ = std::make_shared(fq_threadpool_.get()); } else { snapshot_storage_ = std::make_shared(nullptr); } // check for '--replicaof' before loading anything if (ReplicaOfFlag flag = GetFlag(FLAGS_replicaof); flag.has_value()) { service_.proactor_pool().GetNextProactor()->Await( [this, &flag]() { this->Replicate(flag.host, flag.port); }); } else { // load from snapshot only if --replicaof is empty LoadFromSnapshot(); } const auto create_snapshot_schedule_fb = [this] { snapshot_schedule_fb_ = service_.proactor_pool().GetNextProactor()->LaunchFiber([this] { SnapshotScheduling(); }); }; config_registry.RegisterMutable( "snapshot_cron", [this, create_snapshot_schedule_fb](const absl::CommandLineFlag& flag) { JoinSnapshotSchedule(); create_snapshot_schedule_fb(); return true; }); create_snapshot_schedule_fb(); } void ServerFamily::LoadFromSnapshot() { { util::fb2::LockGuard lk{loading_stats_mu_}; loading_stats_.restore_count++; } const auto load_path_result = snapshot_storage_->LoadPath(GetFlag(FLAGS_dir), GetFlag(FLAGS_dbfilename)); if (load_path_result) { const std::string load_path = *load_path_result; if (!load_path.empty()) { auto future = Load(load_path, LoadExistingKeys::kFail); load_fiber_ = service_.proactor_pool().GetNextProactor()->LaunchFiber([future]() mutable { // Wait for load to finish in a dedicated fiber. // Failure to load on start causes Dragonfly to exit with an error code. if (!future.has_value() || future->Get()) { // Error was already printed to log at this point. exit(1); } }); } } else { if (std::error_code(load_path_result.error()) == std::errc::no_such_file_or_directory) { LOG(WARNING) << "Load snapshot: No snapshot found"; } else { loading_stats_mu_.lock(); loading_stats_.failed_restore_count++; loading_stats_mu_.unlock(); LOG(ERROR) << "Failed to load snapshot with error: " << load_path_result.error().Format(); exit(1); } } } void ServerFamily::JoinSnapshotSchedule() { schedule_done_.Notify(); snapshot_schedule_fb_.JoinIfNeeded(); schedule_done_.Reset(); } void ServerFamily::Shutdown() { VLOG(1) << "ServerFamily::Shutdown"; load_fiber_.JoinIfNeeded(); JoinSnapshotSchedule(); bg_save_fb_.JoinIfNeeded(); if (save_on_shutdown_ && !absl::GetFlag(FLAGS_dbfilename).empty()) { shard_set->pool()->GetNextProactor()->Await([this]() ABSL_LOCKS_EXCLUDED(loading_stats_mu_) { GenericError ec = DoSave(); util::fb2::LockGuard lk{loading_stats_mu_}; loading_stats_.backup_count++; if (ec) { loading_stats_.failed_backup_count++; LOG(WARNING) << "Failed to perform snapshot " << ec.Format(); } }); } client_pause_ec_.await([this] { return active_pauses_.load() == 0; }); pb_task_->Await([this] { auto ec = journal_->Close(); LOG_IF(ERROR, ec) << "Error closing journal " << ec; util::fb2::LockGuard lk(replicaof_mu_); if (replica_) { replica_->Stop(); } StopAllClusterReplicas(); dfly_cmd_->Shutdown(); DebugCmd::Shutdown(); }); } bool ServerFamily::HasPrivilegedInterface() { for (auto* listener : listeners_) { if (listener->IsPrivilegedInterface()) { return true; } } return false; } void ServerFamily::UpdateMemoryGlobalStats() { ShardId sid = EngineShard::tlocal()->shard_id(); if (sid != 0) { // This function is executed periodicaly on all shards. To ensure the logic // bellow runs only on one shard we return is the shard is not 0. return; } uint64_t mem_current = used_mem_current.load(std::memory_order_relaxed); if (mem_current > used_mem_peak.load(memory_order_relaxed)) { used_mem_peak.store(mem_current, memory_order_relaxed); } io::Result sdata_res = io::ReadStatusInfo(); if (sdata_res) { size_t total_rss = FetchRssMemory(sdata_res.value()); rss_mem_current.store(total_rss, memory_order_relaxed); if (rss_mem_peak.load(memory_order_relaxed) < total_rss) { rss_mem_peak.store(total_rss, memory_order_relaxed); } double rss_oom_deny_ratio = ServerState::tlocal()->rss_oom_deny_ratio; if (rss_oom_deny_ratio > 0) { size_t memory_limit = max_memory_limit * rss_oom_deny_ratio; if (total_rss > memory_limit && accepting_connections_ && HasPrivilegedInterface()) { for (auto* listener : listeners_) { if (!listener->IsPrivilegedInterface()) { listener->socket()->proactor()->Await([listener]() { listener->pause_accepting(); }); } } accepting_connections_ = false; } else if (total_rss < memory_limit && !accepting_connections_) { for (auto* listener : listeners_) { if (!listener->IsPrivilegedInterface()) { listener->socket()->proactor()->Await([listener]() { listener->resume_accepting(); }); } } accepting_connections_ = true; } } } } struct AggregateLoadResult { AggregateError first_error; std::atomic keys_read; }; void ServerFamily::FlushAll(Namespace* ns) { const CommandId* cid = service_.FindCmd("FLUSHALL"); boost::intrusive_ptr flush_trans(new Transaction{cid}); flush_trans->InitByArgs(ns, 0, {}); VLOG(1) << "Performing flush"; error_code ec = Drakarys(flush_trans.get(), DbSlice::kDbAll); if (ec) { LOG(ERROR) << "Error flushing db " << ec.message(); } } // Load starts as many fibers as there are files to load each one separately. // It starts one more fiber that waits for all load fibers to finish and returns the first // error (if any occured) with a future. std::optional> ServerFamily::Load(string_view load_path, LoadExistingKeys existing_keys) { std::string path(load_path); if (load_path.empty()) { fs::path dir_path(GetFlag(FLAGS_dir)); string filename = GetFlag(FLAGS_dbfilename); dir_path.append(filename); path = dir_path.generic_string(); } DCHECK_GT(shard_count(), 0u); // TODO: to move it to helio. auto immediate = [](auto val) { fb2::Future future; future.Resolve(val); return future; }; if (ServerState::tlocal() && !ServerState::tlocal()->is_master) { return immediate(string("Replica cannot load data")); } auto expand_result = snapshot_storage_->ExpandSnapshot(path); if (!expand_result) { LOG(ERROR) << "Failed to load snapshot: " << expand_result.error().Format(); return immediate(expand_result.error()); } auto new_state = service_.SwitchState(GlobalState::ACTIVE, GlobalState::LOADING); if (new_state != GlobalState::LOADING) { LOG(WARNING) << new_state << " in progress, ignored"; return {}; } auto& pool = service_.proactor_pool(); const vector& paths = *expand_result; LOG(INFO) << "Loading " << path; vector load_fibers; load_fibers.reserve(paths.size()); auto aggregated_result = std::make_shared(); for (auto& path : paths) { // For single file, choose thread that does not handle shards if possible. // This will balance out the CPU during the load. ProactorBase* proactor; if (paths.size() == 1 && shard_count() < pool.size()) { proactor = pool.at(shard_count()); } else { proactor = pool.GetNextProactor(); } auto load_func = [this, aggregated_result, existing_keys, path = std::move(path)]() { auto load_result = LoadRdb(path, existing_keys); if (load_result.has_value()) aggregated_result->keys_read.fetch_add(*load_result); else aggregated_result->first_error = load_result.error(); }; load_fibers.push_back(proactor->LaunchFiber(std::move(load_func))); } fb2::Future future; // Run fiber that empties the channel and sets ec_promise. auto load_join_func = [this, aggregated_result, load_fibers = std::move(load_fibers), future]() mutable { for (auto& fiber : load_fibers) { fiber.Join(); } if (aggregated_result->first_error) { LOG(ERROR) << "Rdb load failed: " << (*aggregated_result->first_error).message(); } else { RdbLoader::PerformPostLoad(&service_); LOG(INFO) << "Load finished, num keys read: " << aggregated_result->keys_read; } service_.SwitchState(GlobalState::LOADING, GlobalState::ACTIVE); future.Resolve(*(aggregated_result->first_error)); }; pool.GetNextProactor()->Dispatch(std::move(load_join_func)); return future; } void ServerFamily::SnapshotScheduling() { const std::optional cron_expr = InferSnapshotCronExpr(); if (!cron_expr) { return; } ServerState* ss = ServerState::tlocal(); do { if (schedule_done_.WaitFor(100ms)) { return; } } while (ss->gstate() == GlobalState::LOADING); while (true) { const std::chrono::time_point now = std::chrono::system_clock::now(); const std::chrono::time_point next = cron::cron_next(cron_expr.value(), now); if (schedule_done_.WaitFor(next - now)) { break; }; GenericError ec = DoSave(); util::fb2::LockGuard lk{loading_stats_mu_}; loading_stats_.backup_count++; if (ec) { loading_stats_.failed_backup_count++; LOG(WARNING) << "Failed to perform snapshot " << ec.Format(); } } } io::Result ServerFamily::LoadRdb(const std::string& rdb_file, LoadExistingKeys existing_keys) { VLOG(1) << "Loading data from " << rdb_file; CHECK(fb2::ProactorBase::IsProactorThread()) << "must be called from proactor thread"; io::Result result; ProactorBase* proactor = fb2::ProactorBase::me(); auto fb = proactor->LaunchFiber([&] { io::ReadonlyFileOrError res = snapshot_storage_->OpenReadFile(rdb_file); if (!res) { result = nonstd::make_unexpected(res.error()); return; } io::FileSource fs(*res); RdbLoader loader{&service_}; if (existing_keys == LoadExistingKeys::kOverride) { loader.SetOverrideExistingKeys(true); } auto ec = loader.Load(&fs); if (ec) { result = nonstd::make_unexpected(ec); } else { VLOG(1) << "Done loading RDB from " << rdb_file << ", keys loaded: " << loader.keys_loaded(); VLOG(1) << "Loading finished after " << strings::HumanReadableElapsedTime(loader.load_time()); result = loader.keys_loaded(); } }); fb.Join(); return result; } enum MetricType { COUNTER, GAUGE, SUMMARY, HISTOGRAM }; const char* MetricTypeName(MetricType type) { switch (type) { case MetricType::COUNTER: return "counter"; case MetricType::GAUGE: return "gauge"; case MetricType::SUMMARY: return "summary"; case MetricType::HISTOGRAM: return "histogram"; } return "unknown"; } inline string GetMetricFullName(string_view metric_name) { return StrCat("dragonfly_", metric_name); } void AppendMetricHeader(string_view metric_name, string_view metric_help, MetricType type, string* dest) { const auto full_metric_name = GetMetricFullName(metric_name); absl::StrAppend(dest, "# HELP ", full_metric_name, " ", metric_help, "\n"); absl::StrAppend(dest, "# TYPE ", full_metric_name, " ", MetricTypeName(type), "\n"); } void AppendLabelTupple(absl::Span label_names, absl::Span label_values, string* dest) { if (label_names.empty()) return; absl::StrAppend(dest, "{"); for (size_t i = 0; i < label_names.size(); ++i) { if (i > 0) { absl::StrAppend(dest, ", "); } absl::StrAppend(dest, label_names[i], "=\"", label_values[i], "\""); } absl::StrAppend(dest, "}"); } void AppendMetricValue(string_view metric_name, const absl::AlphaNum& value, absl::Span label_names, absl::Span label_values, string* dest) { absl::StrAppend(dest, GetMetricFullName(metric_name)); AppendLabelTupple(label_names, label_values, dest); absl::StrAppend(dest, " ", value, "\n"); } void AppendMetricWithoutLabels(string_view name, string_view help, const absl::AlphaNum& value, MetricType type, string* dest) { AppendMetricHeader(name, help, type, dest); AppendMetricValue(name, value, {}, {}, dest); } void PrintPrometheusMetrics(uint64_t uptime, const Metrics& m, DflyCmd* dfly_cmd, StringResponse* resp) { // Server metrics AppendMetricHeader("version", "", MetricType::GAUGE, &resp->body()); AppendMetricValue("version", 1, {"version"}, {GetVersion()}, &resp->body()); bool is_master = ServerState::tlocal()->is_master; AppendMetricWithoutLabels("master", "1 if master 0 if replica", is_master ? 1 : 0, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("uptime_in_seconds", "", uptime, MetricType::COUNTER, &resp->body()); // Clients metrics const auto& conn_stats = m.facade_stats.conn_stats; AppendMetricWithoutLabels("max_clients", "Maximal number of clients", GetFlag(FLAGS_maxclients), MetricType::GAUGE, &resp->body()); AppendMetricHeader("connected_clients", "", MetricType::GAUGE, &resp->body()); AppendMetricValue("connected_clients", conn_stats.num_conns_main, {"listener"}, {"main"}, &resp->body()); AppendMetricValue("connected_clients", conn_stats.num_conns_other, {"listener"}, {"other"}, &resp->body()); AppendMetricWithoutLabels("client_read_buffer_bytes", "", conn_stats.read_buf_capacity, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("blocked_clients", "", conn_stats.num_blocked_clients, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("dispatch_queue_bytes", "", conn_stats.dispatch_queue_bytes, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("pipeline_queue_length", "", conn_stats.dispatch_queue_entries, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("send_delay_seconds", "", double(GetDelayMs(m.oldest_pending_send_ts)) / 1000.0, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("pipeline_throttle_total", "", conn_stats.pipeline_throttle_count, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("pipeline_cmd_cache_bytes", "", conn_stats.pipeline_cmd_cache_bytes, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("pipeline_commands_total", "", conn_stats.pipelined_cmd_cnt, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("pipeline_commands_duration_seconds", "", conn_stats.pipelined_cmd_latency * 1e-6, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("cmd_squash_hop_total", "", m.coordinator_stats.multi_squash_executions, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("cmd_squash_commands_total", "", m.coordinator_stats.squashed_commands, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("cmd_squash_hop_duration_seconds", "", m.coordinator_stats.multi_squash_exec_hop_usec * 1e-6, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("cmd_squash_hop_reply_seconds", "", m.coordinator_stats.multi_squash_exec_reply_usec * 1e-6, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("commands_squashing_replies_bytes", "", MultiCommandSquasher::GetRepliesMemSize(), MetricType::GAUGE, &resp->body()); string connections_libs; AppendMetricHeader("connections_libs", "Total number of connections by libname:ver", MetricType::GAUGE, &connections_libs); for (const auto& [lib, count] : m.connections_lib_name_ver_map) { AppendMetricValue("connections_libs", count, {"lib"}, {lib}, &connections_libs); } absl::StrAppend(&resp->body(), connections_libs); // Memory metrics auto sdata_res = io::ReadStatusInfo(); AppendMetricWithoutLabels("memory_used_bytes", "", m.heap_used_bytes, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("memory_used_peak_bytes", "", used_mem_peak.load(memory_order_relaxed), MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("memory_fiberstack_vms_bytes", "virtual memory size used by all the fibers", m.worker_fiber_stack_size, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("fibers_count", "", m.worker_fiber_count, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("blocked_tasks", "", m.blocked_tasks, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("memory_max_bytes", "", max_memory_limit, MetricType::GAUGE, &resp->body()); if (m.events.insertion_rejections | m.coordinator_stats.oom_error_cmd_cnt) { AppendMetricHeader("oom_errors_total", "Rejected requests due to out of memory errors", MetricType::COUNTER, &resp->body()); AppendMetricValue("oom_errors_total", m.events.insertion_rejections, {"type"}, {"insert"}, &resp->body()); AppendMetricValue("oom_errors_total", m.coordinator_stats.oom_error_cmd_cnt, {"type"}, {"cmd"}, &resp->body()); } if (sdata_res.has_value()) { size_t rss = FetchRssMemory(sdata_res.value()); AppendMetricWithoutLabels("used_memory_rss_bytes", "", rss, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("swap_memory_bytes", "", sdata_res->vm_swap, MetricType::GAUGE, &resp->body()); } else { LOG_FIRST_N(ERROR, 10) << "Error fetching /proc/self/status stats. error " << sdata_res.error().message(); } AppendMetricWithoutLabels("tls_bytes", "", m.tls_bytes, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("snapshot_serialization_bytes", "", m.serialization_bytes, MetricType::GAUGE, &resp->body()); DbStats total; for (const auto& db_stats : m.db_stats) { total += db_stats; } { string type_used_memory_metric; bool added = false; AppendMetricHeader("type_used_memory", "Memory used per type", MetricType::GAUGE, &type_used_memory_metric); for (unsigned type = 0; type < total.memory_usage_by_type.size(); type++) { size_t mem = total.memory_usage_by_type[type]; if (mem > 0) { AppendMetricValue("type_used_memory", mem, {"type"}, {ObjTypeToString(type)}, &type_used_memory_metric); added = true; } } if (added) absl::StrAppend(&resp->body(), type_used_memory_metric); } // Stats metrics AppendMetricWithoutLabels("connections_received_total", "", conn_stats.conn_received_cnt, MetricType::COUNTER, &resp->body()); AppendMetricHeader("commands_processed_total", "", MetricType::COUNTER, &resp->body()); AppendMetricValue("commands_processed_total", conn_stats.command_cnt_main, {"listener"}, {"main"}, &resp->body()); AppendMetricValue("commands_processed_total", conn_stats.command_cnt_other, {"listener"}, {"other"}, &resp->body()); AppendMetricWithoutLabels("keyspace_hits_total", "", m.events.hits, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("keyspace_misses_total", "", m.events.misses, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("keyspace_mutations_total", "", m.events.mutations, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("lua_interpreter_cnt", "", m.lua_stats.interpreter_cnt, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("used_memory_lua", "", m.lua_stats.used_bytes, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("lua_blocked_total", "", m.lua_stats.blocked_cnt, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("backups_total", "", m.loading_stats.backup_count, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("failed_backups_total", "", m.loading_stats.failed_backup_count, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("restores_total", "", m.loading_stats.restore_count, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("failed_restores_total", "", m.loading_stats.failed_restore_count, MetricType::COUNTER, &resp->body()); // Net metrics AppendMetricWithoutLabels("net_input_bytes_total", "", conn_stats.io_read_bytes, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("net_output_bytes_total", "", m.facade_stats.reply_stats.io_write_bytes, MetricType::COUNTER, &resp->body()); { AppendMetricWithoutLabels("reply_duration_seconds", "", m.facade_stats.reply_stats.send_stats.total_duration * 1e-6, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("reply_total", "", m.facade_stats.reply_stats.send_stats.count, MetricType::COUNTER, &resp->body()); } AppendMetricWithoutLabels("script_error_total", "", m.facade_stats.reply_stats.script_error_count, MetricType::COUNTER, &resp->body()); AppendMetricHeader("listener_accept_error_total", "Listener accept errors", MetricType::COUNTER, &resp->body()); AppendMetricValue("listener_accept_error_total", m.refused_conn_max_clients_reached_count, {"reason"}, {"limit_reached"}, &resp->body()); // DB stats AppendMetricWithoutLabels("expired_keys_total", "", m.events.expired_keys, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("evicted_keys_total", "", m.events.evicted_keys, MetricType::COUNTER, &resp->body()); // Command stats if (!m.cmd_stats_map.empty()) { string command_metrics; AppendMetricHeader("commands_total", "Total number of commands executed", MetricType::COUNTER, &command_metrics); for (const auto& [name, stat] : m.cmd_stats_map) { const auto calls = stat.first; AppendMetricValue("commands_total", calls, {"cmd"}, {name}, &command_metrics); } AppendMetricHeader("commands_duration_seconds", "Duration of commands in seconds", MetricType::HISTOGRAM, &command_metrics); for (const auto& [name, stat] : m.cmd_stats_map) { const double duration_seconds = stat.second * 1e-6; AppendMetricValue("commands_duration_seconds", duration_seconds, {"cmd"}, {name}, &command_metrics); } absl::StrAppend(&resp->body(), command_metrics); } if (m.replica_side_info) { // slave side auto& replica_info = *m.replica_side_info; AppendMetricWithoutLabels("replica_reconnect_count", "Number of replica reconnects", replica_info.reconnect_count, MetricType::COUNTER, &resp->body()); } else { // Master side string replication_lag_metrics; vector replicas_info = dfly_cmd->GetReplicasRoleInfo(); ReplicationMemoryStats repl_mem; dfly_cmd->GetReplicationMemoryStats(&repl_mem); AppendMetricWithoutLabels( "replication_streaming_bytes", "Stable sync replication memory usage", repl_mem.streamer_buf_capacity_bytes, MetricType::GAUGE, &resp->body()); AppendMetricWithoutLabels("replication_full_sync_bytes", "Full sync memory usage", repl_mem.full_sync_buf_bytes, MetricType::GAUGE, &resp->body()); AppendMetricHeader("connected_replica_lag_records", "Lag in records of a connected replica.", MetricType::GAUGE, &replication_lag_metrics); for (const auto& replica : replicas_info) { AppendMetricValue("connected_replica_lag_records", replica.lsn_lag, {"replica_ip", "replica_port", "replica_state"}, {replica.address, absl::StrCat(replica.listening_port), replica.state}, &replication_lag_metrics); } absl::StrAppend(&resp->body(), replication_lag_metrics); } AppendMetricWithoutLabels("fiber_switch_total", "", m.fiber_switch_cnt, MetricType::COUNTER, &resp->body()); double delay_seconds = m.fiber_switch_delay_usec * 1e-6; AppendMetricWithoutLabels("fiber_switch_delay_seconds_total", "", delay_seconds, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("fiber_longrun_total", "", m.fiber_longrun_cnt, MetricType::COUNTER, &resp->body()); double longrun_seconds = m.fiber_longrun_usec * 1e-6; AppendMetricWithoutLabels("fiber_longrun_seconds", "", longrun_seconds, MetricType::COUNTER, &resp->body()); AppendMetricWithoutLabels("tx_queue_len", "", m.tx_queue_len, MetricType::GAUGE, &resp->body()); { bool added = false; string str; AppendMetricHeader("transaction_widths_total", "Transaction counts by their widths", MetricType::COUNTER, &str); for (unsigned width = 0; width < shard_set->size(); ++width) { uint64_t count = m.coordinator_stats.tx_width_freq_arr[width]; if (count > 0) { AppendMetricValue("transaction_widths_total", count, {"width"}, {StrCat("w", width + 1)}, &str); added = true; } } if (added) absl::StrAppend(&resp->body(), str); } if (IsClusterEnabled()) { string migration_errors_str; AppendMetricHeader("migration_errors_total", "Total error numbers of current migrations", MetricType::GAUGE, &migration_errors_str); AppendMetricValue("migration_errors_total", m.migration_errors_total, {"num"}, {"migration errors"}, &migration_errors_str); absl::StrAppend(&resp->body(), migration_errors_str); string moved_errors_str; uint64_t moved_total_errors = 0; if (m.facade_stats.reply_stats.err_count.contains("MOVED")) { moved_total_errors = m.facade_stats.reply_stats.err_count.at("MOVED"); } AppendMetricHeader("moved_errors_total", "Total number of moved slot errors", MetricType::COUNTER, &moved_errors_str); AppendMetricValue("moved_errors_total", moved_total_errors, {"num"}, {"moved errors"}, &moved_errors_str); absl::StrAppend(&resp->body(), moved_errors_str); } string db_key_metrics; string db_key_expire_metrics; AppendMetricHeader("db_keys", "Total number of keys by DB", MetricType::GAUGE, &db_key_metrics); AppendMetricHeader("db_capacity", "Table capacity by DB", MetricType::GAUGE, &db_key_metrics); AppendMetricHeader("db_keys_expiring", "Total number of expiring keys by DB", MetricType::GAUGE, &db_key_expire_metrics); for (size_t i = 0; i < m.db_stats.size(); ++i) { AppendMetricValue("db_keys", m.db_stats[i].key_count, {"db"}, {StrCat("db", i)}, &db_key_metrics); AppendMetricValue("db_capacity", m.db_stats[i].prime_capacity, {"db", "type"}, {StrCat("db", i), "prime"}, &db_key_metrics); AppendMetricValue("db_capacity", m.db_stats[i].expire_capacity, {"db", "type"}, {StrCat("db", i), "expire"}, &db_key_metrics); AppendMetricValue("db_keys_expiring", m.db_stats[i].expire_count, {"db"}, {StrCat("db", i)}, &db_key_expire_metrics); } absl::StrAppend(&resp->body(), db_key_metrics); absl::StrAppend(&resp->body(), db_key_expire_metrics); } void ServerFamily::ConfigureMetrics(util::HttpListenerBase* http_base) { // The naming of the metrics should be compatible with redis_exporter, see // https://github.com/oliver006/redis_exporter/blob/master/exporter/exporter.go#L111 auto cb = [this](const util::http::QueryArgs& args, util::HttpContext* send) { StringResponse resp = util::http::MakeStringResponse(boost::beast::http::status::ok); util::http::SetMime(util::http::kTextMime, &resp); uint64_t uptime = time(NULL) - start_time_; PrintPrometheusMetrics(uptime, this->GetMetrics(&namespaces->GetDefaultNamespace()), this->dfly_cmd_.get(), &resp); return send->Invoke(std::move(resp)); }; http_base->RegisterCb("/metrics", cb); } void ServerFamily::PauseReplication(bool pause) { util::fb2::LockGuard lk(replicaof_mu_); // Switch to primary mode. if (!ServerState::tlocal()->is_master) { auto repl_ptr = replica_; CHECK(repl_ptr); repl_ptr->Pause(pause); } } std::optional ServerFamily::GetReplicaOffsetInfo() { util::fb2::LockGuard lk(replicaof_mu_); // Switch to primary mode. if (!ServerState::tlocal()->is_master) { auto repl_ptr = replica_; CHECK(repl_ptr); return ReplicaOffsetInfo{repl_ptr->GetSyncId(), repl_ptr->GetReplicaOffset()}; } return nullopt; } vector ServerFamily::GetNonPriviligedListeners() const { std::vector listeners; listeners.reserve(listeners.size()); for (facade::Listener* listener : listeners_) { if (!listener->IsPrivilegedInterface()) { listeners.push_back(listener); } } return listeners; } optional ServerFamily::GetReplicaSummary() const { util::fb2::LockGuard lk(replicaof_mu_); if (replica_ == nullptr) { return nullopt; } else { return replica_->GetSummary(); } } void ServerFamily::OnClose(ConnectionContext* cntx) { dfly_cmd_->OnClose(cntx->conn_state.replication_info.repl_session_id); } void ServerFamily::StatsMC(std::string_view section, SinkReplyBuilder* builder) { if (!section.empty()) { return builder->SendError(""); } string info; #define ADD_LINE(name, val) absl::StrAppend(&info, "STAT " #name " ", val, "\r\n") time_t now = time(NULL); struct rusage ru; getrusage(RUSAGE_SELF, &ru); auto dbl_time = [](const timeval& tv) -> double { return tv.tv_sec + double(tv.tv_usec) / 1000000.0; }; double utime = dbl_time(ru.ru_utime); double systime = dbl_time(ru.ru_stime); auto kind = ProactorBase::me()->GetKind(); const char* multiplex_api = (kind == ProactorBase::IOURING) ? "iouring" : "epoll"; Metrics m = GetMetrics(&namespaces->GetDefaultNamespace()); uint64_t uptime = time(NULL) - start_time_; const uint32_t total_conns = m.facade_stats.conn_stats.num_conns_main + m.facade_stats.conn_stats.num_conns_other; ADD_LINE(pid, getpid()); ADD_LINE(uptime, uptime); ADD_LINE(time, now); ADD_LINE(version, kGitTag); ADD_LINE(libevent, multiplex_api); ADD_LINE(pointer_size, sizeof(void*)); ADD_LINE(rusage_user, utime); ADD_LINE(rusage_system, systime); ADD_LINE(max_connections, -1); ADD_LINE(curr_connections, total_conns); ADD_LINE(total_connections, -1); ADD_LINE(rejected_connections, -1); ADD_LINE(bytes_read, m.facade_stats.conn_stats.io_read_bytes); ADD_LINE(bytes_written, m.facade_stats.reply_stats.io_write_bytes); ADD_LINE(limit_maxbytes, -1); absl::StrAppend(&info, "END\r\n"); MCReplyBuilder* mc_builder = static_cast(builder); mc_builder->SendRaw(info); #undef ADD_LINE } GenericError ServerFamily::DoSave(bool ignore_state) { const CommandId* cid = service().FindCmd("SAVE"); CHECK_NOTNULL(cid); boost::intrusive_ptr trans(new Transaction{cid}); trans->InitByArgs(&namespaces->GetDefaultNamespace(), 0, {}); return DoSave(SaveCmdOptions{absl::GetFlag(FLAGS_df_snapshot_format), {}, {}}, trans.get(), ignore_state); } GenericError ServerFamily::DoSaveCheckAndStart(const SaveCmdOptions& save_cmd_opts, Transaction* trans, DoSaveCheckAndStartOpts opts) { auto [ignore_state, bg_save] = opts; auto state = ServerState::tlocal()->gstate(); // In some cases we want to create a snapshot even if server is not active, f.e in takeover if (!ignore_state && (state != GlobalState::ACTIVE && state != GlobalState::SHUTTING_DOWN)) { return GenericError{make_error_code(errc::operation_in_progress), StrCat(GlobalStateName(state), " - can not save database")}; } { util::fb2::LockGuard lk(save_mu_); if (save_controller_) { return GenericError{make_error_code(errc::operation_in_progress), "SAVING - can not save database"}; } auto snapshot_storage = save_cmd_opts.cloud_uri.empty() ? snapshot_storage_ : CreateCloudSnapshotStorage(save_cmd_opts.cloud_uri); save_controller_ = make_unique(detail::SaveStagesInputs{ save_cmd_opts.new_version, save_cmd_opts.cloud_uri, save_cmd_opts.basename, trans, &service_, fq_threadpool_.get(), snapshot_storage, opts.bg_save}); auto res = save_controller_->InitResourcesAndStart(); if (res) { DCHECK_EQ(res->error, true); last_save_info_.SetLastSaveError(*res); save_controller_.reset(); if (bg_save) { last_save_info_.last_bgsave_status = false; } return res->error; } last_save_info_.bgsave_in_progress = bg_save; } return {}; } GenericError ServerFamily::WaitUntilSaveFinished(Transaction* trans, bool ignore_state) { save_controller_->WaitAllSnapshots(); detail::SaveInfo save_info; VLOG(1) << "Before WaitUntilSaveFinished::Finalize"; { util::fb2::LockGuard lk(save_mu_); save_info = save_controller_->Finalize(); if (save_controller_->IsBgSave()) { last_save_info_.bgsave_in_progress = false; last_save_info_.last_bgsave_status = !save_info.error; } if (save_info.error) { last_save_info_.SetLastSaveError(save_info); } else { last_save_info_.save_time = save_info.save_time; last_save_info_.success_duration_sec = save_info.duration_sec; last_save_info_.file_name = save_info.file_name; last_save_info_.freq_map = save_info.freq_map; } save_controller_.reset(); } return save_info.error; } GenericError ServerFamily::DoSave(const SaveCmdOptions& save_cmd_opts, Transaction* trans, bool ignore_state) { DoSaveCheckAndStartOpts opts{.ignore_state = ignore_state}; if (auto ec = DoSaveCheckAndStart(save_cmd_opts, trans, opts); ec) { return ec; } return WaitUntilSaveFinished(trans, ignore_state); } bool ServerFamily::TEST_IsSaving() const { std::atomic_bool is_saving{false}; shard_set->pool()->AwaitFiberOnAll([&](auto*) { if (SliceSnapshot::IsSnaphotInProgress()) is_saving.store(true, std::memory_order_relaxed); }); return is_saving.load(std::memory_order_relaxed); } error_code ServerFamily::Drakarys(Transaction* transaction, DbIndex db_ind) { VLOG(1) << "Drakarys"; transaction->Execute( [db_ind](Transaction* t, EngineShard* shard) { t->GetDbSlice(shard->shard_id()).FlushDb(db_ind); return OpStatus::OK; }, true); return error_code{}; } LastSaveInfo ServerFamily::GetLastSaveInfo() const { util::fb2::LockGuard lk(save_mu_); return last_save_info_; } void ServerFamily::DbSize(CmdArgList args, const CommandContext& cmd_cntx) { atomic_ulong num_keys{0}; shard_set->RunBriefInParallel( [&](EngineShard* shard) { auto db_size = cmd_cntx.conn_cntx->ns->GetDbSlice(shard->shard_id()) .DbSize(cmd_cntx.conn_cntx->conn_state.db_index); num_keys.fetch_add(db_size, memory_order_relaxed); }, [](ShardId) { return true; }); return cmd_cntx.rb->SendLong(num_keys.load(memory_order_relaxed)); } void ServerFamily::CancelBlockingOnThread(std::function status_cb) { auto cb = [status_cb](unsigned thread_index, util::Connection* conn) { if (auto fcntx = static_cast(conn)->cntx(); fcntx) { auto* cntx = static_cast(fcntx); if (cntx->transaction) { cntx->transaction->CancelBlocking(status_cb); } } }; for (auto* listener : listeners_) { listener->TraverseConnectionsOnThread(cb, UINT32_MAX, nullptr); } } string GetPassword() { string flag = GetFlag(FLAGS_requirepass); if (!flag.empty()) { return flag; } const char* env_var = getenv("DFLY_PASSWORD"); if (env_var) { return env_var; } return ""; } void ServerFamily::SendInvalidationMessages() const { // send invalidation message (caused by flushdb) to all the clients which // turned on client tracking auto cb = [](unsigned thread_index, util::Connection* conn) { facade::ConnectionContext* fc = static_cast(conn)->cntx(); if (fc) { ConnectionContext* cntx = static_cast(fc); if (cntx->conn_state.tracking_info_.IsTrackingOn()) { facade::Connection::InvalidationMessage x; x.invalidate_due_to_flush = true; cntx->conn()->SendInvalidationMessageAsync(x); } } }; for (auto* listener : listeners_) { listener->TraverseConnections(cb); } } void ServerFamily::FlushDb(CmdArgList args, const CommandContext& cmd_cntx) { DCHECK(cmd_cntx.tx); Drakarys(cmd_cntx.tx, cmd_cntx.tx->GetDbIndex()); SendInvalidationMessages(); cmd_cntx.rb->SendOk(); } void ServerFamily::FlushAll(CmdArgList args, const CommandContext& cmd_cntx) { if (args.size() > 1) { cmd_cntx.rb->SendError(kSyntaxErr); return; } DCHECK(cmd_cntx.tx); Drakarys(cmd_cntx.tx, DbSlice::kDbAll); SendInvalidationMessages(); cmd_cntx.rb->SendOk(); } bool ServerFamily::DoAuth(ConnectionContext* cntx, std::string_view username, std::string_view password) { const auto* registry = ServerState::tlocal()->user_registry; CHECK(registry); const bool is_authorized = registry->AuthUser(username, password); if (is_authorized) { cntx->authed_username = username; auto cred = registry->GetCredentials(username); cntx->acl_commands = cred.acl_commands; cntx->keys = std::move(cred.keys); cntx->pub_sub = std::move(cred.pub_sub); cntx->ns = &namespaces->GetOrInsert(cred.ns); cntx->authenticated = true; cntx->acl_db_idx = cred.db; if (cred.db == std::numeric_limits::max()) { cntx->conn_state.db_index = 0; } else { auto cb = [ns = cntx->ns, index = cred.db](EngineShard* shard) { auto& db_slice = ns->GetDbSlice(shard->shard_id()); db_slice.ActivateDb(index); return OpStatus::OK; }; shard_set->RunBriefInParallel(std::move(cb)); cntx->conn_state.db_index = cred.db; } } return is_authorized; } void ServerFamily::Auth(CmdArgList args, const CommandContext& cmd_cntx) { if (args.size() > 2) { return cmd_cntx.rb->SendError(kSyntaxErr); } ConnectionContext* cntx = cmd_cntx.conn_cntx; // non admin port auth if (!cntx->conn()->IsPrivileged()) { const bool one_arg = args.size() == 1; std::string_view username = one_arg ? "default" : facade::ToSV(args[0]); const size_t index = one_arg ? 0 : 1; std::string_view password = facade::ToSV(args[index]); if (DoAuth(cntx, username, password)) { return cmd_cntx.rb->SendOk(); } auto& log = ServerState::tlocal()->acl_log; using Reason = acl::AclLog::Reason; log.Add(*cntx, "AUTH", Reason::AUTH, std::string(username)); return cmd_cntx.rb->SendError(facade::kAuthRejected); } if (!cntx->req_auth) { return cmd_cntx.rb->SendError( "AUTH called without any password configured for " "admin port. Are you sure your configuration is correct?"); } string_view pass = ArgS(args, 0); if (pass == GetPassword()) { cntx->authenticated = true; cmd_cntx.rb->SendOk(); } else { cmd_cntx.rb->SendError(facade::kAuthRejected); } } void ServerFamily::ClientUnPauseCmd(CmdArgList args, SinkReplyBuilder* builder) { if (!args.empty()) { builder->SendError(facade::kSyntaxErr); return; } is_c_pause_in_progress_.store(false, std::memory_order_relaxed); builder->SendOk(); } void ClientHelp(SinkReplyBuilder* builder) { string_view help_arr[] = { "CLIENT [ [value] [opt] ...]. Subcommands are:", "CACHING (YES|NO)", " Enable/disable tracking of the keys for next command in OPTIN/OPTOUT modes.", "GETNAME", " Return the name of the current connection.", "ID", " Return the ID of the current connection.", "KILL ", " Kill connection made from .", "KILL