diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp index 8a89e6317733f8..e7229e6f12d6de 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -434,7 +434,8 @@ Status CloudCumulativeCompaction::modify_rowsets() { DBUG_BLOCK); auto status = _engine.meta_mgr().cloud_update_delete_bitmap_without_lock( *cloud_tablet(), pre_rowsets_delete_bitmap.get(), pre_rowset_to_versions, - _output_rowset->start_version(), _output_rowset->end_version()); + cloud_tablet()->table_id(), _output_rowset->start_version(), + _output_rowset->end_version()); if (!status.ok()) { LOG(WARNING) << "failed to agg pre rowsets delete bitmap to ms. tablet_id=" << _tablet->tablet_id() << ", pre rowset num=" << pre_rowsets.size() diff --git a/be/src/cloud/cloud_delete_task.cpp b/be/src/cloud/cloud_delete_task.cpp index dc3d991df58397..6ad6fd9d65ee7f 100644 --- a/be/src/cloud/cloud_delete_task.cpp +++ b/be/src/cloud/cloud_delete_task.cpp @@ -96,13 +96,14 @@ Status CloudDeleteTask::execute(CloudStorageEngine& engine, const TPushReq& requ RETURN_IF_ERROR(rowset_writer->build(rowset)); rowset->rowset_meta()->set_delete_predicate(std::move(del_pred)); - auto st = engine.meta_mgr().prepare_rowset(*rowset_writer->rowset_meta(), ""); + auto st = + engine.meta_mgr().prepare_rowset(*rowset_writer->rowset_meta(), "", tablet->table_id()); if (!st.ok()) { LOG(WARNING) << "failed to prepare rowset, status=" << st.to_string(); return st; } - st = engine.meta_mgr().commit_rowset(*rowset->rowset_meta(), ""); + st = engine.meta_mgr().commit_rowset(*rowset->rowset_meta(), "", tablet->table_id()); if (!st.ok()) { LOG(WARNING) << "failed to commit rowset, status=" << st.to_string(); return st; diff --git a/be/src/cloud/cloud_delta_writer.cpp b/be/src/cloud/cloud_delta_writer.cpp index d51d5d8b5763bc..c54d26921d6ae7 100644 --- a/be/src/cloud/cloud_delta_writer.cpp +++ b/be/src/cloud/cloud_delta_writer.cpp @@ -120,7 +120,8 @@ Status CloudDeltaWriter::commit_rowset() { } // Handle normal rowset with data - return _engine.meta_mgr().commit_rowset(*rowset_meta(), ""); + return _engine.meta_mgr().commit_rowset(*rowset_meta(), "", + _rowset_builder->tablet()->table_id()); } Status CloudDeltaWriter::_commit_empty_rowset() { @@ -138,7 +139,8 @@ Status CloudDeltaWriter::_commit_empty_rowset() { return Status::OK(); } // write a empty rowset kv to keep version continuous - return _engine.meta_mgr().commit_rowset(*rowset_meta(), ""); + return _engine.meta_mgr().commit_rowset(*rowset_meta(), "", + _rowset_builder->tablet()->table_id()); } Status CloudDeltaWriter::set_txn_related_info() { diff --git a/be/src/cloud/cloud_full_compaction.cpp b/be/src/cloud/cloud_full_compaction.cpp index 8dfb3da433f94e..56d81e98a9f029 100644 --- a/be/src/cloud/cloud_full_compaction.cpp +++ b/be/src/cloud/cloud_full_compaction.cpp @@ -379,7 +379,7 @@ Status CloudFullCompaction::_cloud_full_compaction_update_delete_bitmap(int64_t RETURN_IF_ERROR(_engine.meta_mgr().update_delete_bitmap( *cloud_tablet(), -1, initiator, delete_bitmap.get(), delete_bitmap.get(), _output_rowset->rowset_id().to_string(), storage_resource, - config::delete_bitmap_store_write_version)); + config::delete_bitmap_store_write_version, cloud_tablet()->table_id())); LOG_INFO("update delete bitmap in CloudFullCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), _input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()) diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index 7405930b408c73..d84a54cd1e9b2f 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -42,6 +42,8 @@ #include #include +#include "cloud/cloud_ms_backpressure_handler.h" +#include "cloud/cloud_ms_rpc_rate_limiters.h" #include "cloud/cloud_storage_engine.h" #include "cloud/cloud_tablet.h" #include "cloud/cloud_warm_up_manager.h" @@ -378,17 +380,85 @@ inline std::default_random_engine make_random_engine() { static_cast(std::chrono::steady_clock::now().time_since_epoch().count())); } +// Convert MetaServiceRPC to LoadRelatedRpc +// Returns LoadRelatedRpc::COUNT if the RPC is not a load-related RPC +LoadRelatedRpc to_load_related_rpc(MetaServiceRPC rpc) { + switch (rpc) { + case MetaServiceRPC::PREPARE_ROWSET: + return LoadRelatedRpc::PREPARE_ROWSET; + case MetaServiceRPC::COMMIT_ROWSET: + return LoadRelatedRpc::COMMIT_ROWSET; + case MetaServiceRPC::UPDATE_TMP_ROWSET: + return LoadRelatedRpc::UPDATE_TMP_ROWSET; + case MetaServiceRPC::UPDATE_PACKED_FILE_INFO: + return LoadRelatedRpc::UPDATE_PACKED_FILE_INFO; + case MetaServiceRPC::UPDATE_DELETE_BITMAP: + return LoadRelatedRpc::UPDATE_DELETE_BITMAP; + default: + return LoadRelatedRpc::COUNT; // Not a load-related RPC + } +} + template using MetaServiceMethod = void (MetaService_Stub::*)(::google::protobuf::RpcController*, const Request*, Response*, ::google::protobuf::Closure*); +// Rate limiting context for retry_rpc +struct RpcRateLimitCtx { + HostLevelMSRpcRateLimiters* host_limiters {nullptr}; + MSBackpressureHandler* backpressure_handler {nullptr}; + int64_t table_id {-1}; // For table-level backpressure, passed from caller +}; + +// Apply rate limiting before RPC (both host-level and table-level) +void apply_rate_limit(MetaServiceRPC rpc, const RpcRateLimitCtx& ctx) { + // Table-level rate limit (for load-related RPCs only) + if (ctx.backpressure_handler && ctx.table_id > 0) { + LoadRelatedRpc load_rpc = to_load_related_rpc(rpc); + if (load_rpc != LoadRelatedRpc::COUNT) { + auto wait_until = ctx.backpressure_handler->before_rpc(load_rpc, ctx.table_id); + auto now = std::chrono::steady_clock::now(); + if (wait_until > now) { + auto wait_us = + std::chrono::duration_cast(wait_until - now) + .count(); + if (wait_us > 0) { + if (auto* recorder = get_throttle_wait_recorder(load_rpc); + recorder != nullptr) { + *recorder << wait_us; + } + bthread_usleep(wait_us); + } + } + } + } + + // Host-level rate limit + if (ctx.host_limiters) { + ctx.host_limiters->limit(rpc); + } +} + +// Record RPC QPS statistics after RPC (for table-level tracking) +void record_rpc_qps(MetaServiceRPC rpc, const RpcRateLimitCtx& ctx) { + if (ctx.backpressure_handler && ctx.table_id > 0) { + LoadRelatedRpc load_rpc = to_load_related_rpc(rpc); + if (load_rpc != LoadRelatedRpc::COUNT) { + ctx.backpressure_handler->after_rpc(load_rpc, ctx.table_id); + } + } +} + template -Status retry_rpc(std::string_view op_name, const Request& req, Response* res, - MetaServiceMethod method) { +Status retry_rpc(MetaServiceRPC rpc, const Request& req, Response* res, + MetaServiceMethod method, + const RpcRateLimitCtx& rate_limit_ctx = {}) { static_assert(std::is_base_of_v<::google::protobuf::Message, Request>); static_assert(std::is_base_of_v<::google::protobuf::Message, Response>); + std::string_view op_name = meta_service_rpc_display_name(rpc); + // Applies only to the current file, and all req are non-const, but passed as const types. const_cast(req).set_request_ip(BackendOptions::get_be_endpoint()); @@ -400,11 +470,18 @@ Status retry_rpc(std::string_view op_name, const Request& req, Response* res, std::uniform_int_distribution u2(500, 1000); MetaServiceProxy* proxy; RETURN_IF_ERROR(MetaServiceProxy::get_proxy(&proxy)); + while (true) { std::shared_ptr stub; RETURN_IF_ERROR(proxy->get(&stub)); + + // Apply rate limiting (both host-level and table-level) + apply_rate_limit(rpc, rate_limit_ctx); + TEST_SYNC_POINT_CALLBACK("retry_rpc::after_rate_limit", &rpc); + brpc::Controller cntl; - if (op_name == "get delete bitmap" || op_name == "update delete bitmap") { + if (rpc == MetaServiceRPC::GET_DELETE_BITMAP || + rpc == MetaServiceRPC::UPDATE_DELETE_BITMAP) { cntl.set_timeout_ms(3 * config::meta_service_brpc_timeout_ms); } else { cntl.set_timeout_ms(config::meta_service_brpc_timeout_ms); @@ -413,6 +490,10 @@ Status retry_rpc(std::string_view op_name, const Request& req, Response* res, res->Clear(); int error_code = 0; (stub.get()->*method)(&cntl, &req, res, nullptr); + + // Record QPS statistics for all RPCs sent to MS (success or failure) + record_rpc_qps(rpc, rate_limit_ctx); + if (cntl.Failed()) [[unlikely]] { error_msg = cntl.ErrorText(); error_code = cntl.ErrorCode(); @@ -422,6 +503,11 @@ Status retry_rpc(std::string_view op_name, const Request& req, Response* res, } else if (res->status().code() == MetaServiceCode::INVALID_ARGUMENT) { return Status::Error("failed to {}: {}", op_name, res->status().msg()); + } else if (res->status().code() == MetaServiceCode::MS_TOO_BUSY) { + // MS_BUSY should also be retried + if (rate_limit_ctx.backpressure_handler) { + rate_limit_ctx.backpressure_handler->on_ms_busy(); + } } else if (res->status().code() != MetaServiceCode::KV_TXN_CONFLICT) { return Status::Error("failed to {}: {}", op_name, res->status().msg()); @@ -460,7 +546,12 @@ Status CloudMetaMgr::get_tablet_meta(int64_t tablet_id, TabletMetaSharedPtr* tab GetTabletResponse resp; req.set_cloud_unique_id(config::cloud_unique_id); req.set_tablet_id(tablet_id); - Status st = retry_rpc("get tablet meta", req, &resp, &MetaService_Stub::get_tablet); + Status st = + retry_rpc(MetaServiceRPC::GET_TABLET_META, req, &resp, &MetaService_Stub::get_tablet, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); if (!st.ok()) { if (resp.status().code() == MetaServiceCode::TABLET_NOT_FOUND) { return Status::NotFound("failed to get tablet meta: {}", resp.status().msg()); @@ -617,6 +708,12 @@ Status CloudMetaMgr::sync_tablet_rowsets_unlocked(CloudTablet* tablet, } req.set_end_version(-1); VLOG_DEBUG << "send GetRowsetRequest: " << req.ShortDebugString(); + + // Host-level rate limiting for get_rowset + if (host_level_ms_rpc_rate_limiters_) { + host_level_ms_rpc_rate_limiters_->limit(MetaServiceRPC::GET_ROWSET); + } + auto start = std::chrono::steady_clock::now(); stub->get_rowset(&cntl, &req, &resp, nullptr); auto end = std::chrono::steady_clock::now(); @@ -645,6 +742,25 @@ Status CloudMetaMgr::sync_tablet_rowsets_unlocked(CloudTablet* tablet, return Status::NotFound("failed to get rowset meta: {}, {}", resp.status().msg(), tablet_info); } + if (resp.status().code() == MetaServiceCode::MS_TOO_BUSY) { + // MS_BUSY should also be retried + if (ms_backpressure_handler_) { + ms_backpressure_handler_->on_ms_busy(); + } + if (tried++ < retry_times) { + auto rng = make_random_engine(); + std::uniform_int_distribution u(20, 200); + std::uniform_int_distribution u1(500, 1000); + uint32_t duration_ms = tried >= 100 ? u(rng) : u1(rng); + bthread_usleep(duration_ms * 1000); + LOG_INFO("meta service is too busy when getting rowset meta, " + tablet_info) + .tag("reason", resp.status().msg()) + .tag("tried", tried) + .tag("sleep", duration_ms); + continue; + } + return Status::RpcError("failed to get rowset meta: {}", resp.status().msg()); + } if (resp.status().code() != MetaServiceCode::OK) { LOG(WARNING) << " failed to get rowset meta, err=" << resp.status().msg() << " " << tablet_info; @@ -858,7 +974,12 @@ Status CloudMetaMgr::_get_delete_bitmap_from_ms(GetDeleteBitmapRequest& req, VLOG_DEBUG << "send GetDeleteBitmapRequest: " << req.ShortDebugString(); TEST_SYNC_POINT_CALLBACK("CloudMetaMgr::_get_delete_bitmap_from_ms", &req, &res); - auto st = retry_rpc("get delete bitmap", req, &res, &MetaService_Stub::get_delete_bitmap); + auto st = retry_rpc(MetaServiceRPC::GET_DELETE_BITMAP, req, &res, + &MetaService_Stub::get_delete_bitmap, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); if (st.code() == ErrorCode::THRIFT_RPC_ERROR) { return st; } @@ -1306,7 +1427,7 @@ Status CloudMetaMgr::_read_tablet_delete_bitmap_v2(CloudTablet* tablet, int64_t } Status CloudMetaMgr::prepare_rowset(const RowsetMeta& rs_meta, const std::string& job_id, - RowsetMetaSharedPtr* existed_rs_meta) { + int64_t table_id, RowsetMetaSharedPtr* existed_rs_meta) { VLOG_DEBUG << "prepare rowset, tablet_id: " << rs_meta.tablet_id() << ", rowset_id: " << rs_meta.rowset_id() << " txn_id: " << rs_meta.txn_id(); { @@ -1322,7 +1443,13 @@ Status CloudMetaMgr::prepare_rowset(const RowsetMeta& rs_meta, const std::string RowsetMetaPB doris_rs_meta = rs_meta.get_rowset_pb(/*skip_schema=*/true); doris_rowset_meta_to_cloud(req.mutable_rowset_meta(), std::move(doris_rs_meta)); - Status st = retry_rpc("prepare rowset", req, &resp, &MetaService_Stub::prepare_rowset); + Status st = + retry_rpc(MetaServiceRPC::PREPARE_ROWSET, req, &resp, &MetaService_Stub::prepare_rowset, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + .table_id = table_id, + }); if (!st.ok() && resp.status().code() == MetaServiceCode::ALREADY_EXISTED) { if (existed_rs_meta != nullptr && resp.has_existed_rowset_meta()) { RowsetMetaPB doris_rs_meta_tmp = @@ -1335,7 +1462,7 @@ Status CloudMetaMgr::prepare_rowset(const RowsetMeta& rs_meta, const std::string return st; } -Status CloudMetaMgr::commit_rowset(RowsetMeta& rs_meta, const std::string& job_id, +Status CloudMetaMgr::commit_rowset(RowsetMeta& rs_meta, const std::string& job_id, int64_t table_id, RowsetMetaSharedPtr* existed_rs_meta) { VLOG_DEBUG << "commit rowset, tablet_id: " << rs_meta.tablet_id() << ", rowset_id: " << rs_meta.rowset_id() << " txn_id: " << rs_meta.txn_id(); @@ -1352,7 +1479,13 @@ Status CloudMetaMgr::commit_rowset(RowsetMeta& rs_meta, const std::string& job_i RowsetMetaPB rs_meta_pb = rs_meta.get_rowset_pb(); doris_rowset_meta_to_cloud(req.mutable_rowset_meta(), std::move(rs_meta_pb)); - Status st = retry_rpc("commit rowset", req, &resp, &MetaService_Stub::commit_rowset); + Status st = + retry_rpc(MetaServiceRPC::COMMIT_ROWSET, req, &resp, &MetaService_Stub::commit_rowset, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + .table_id = table_id, + }); if (!st.ok() && resp.status().code() == MetaServiceCode::ALREADY_EXISTED) { if (existed_rs_meta != nullptr && resp.has_existed_rowset_meta()) { RowsetMetaPB doris_rs_meta = @@ -1394,7 +1527,7 @@ void CloudMetaMgr::cache_committed_rowset(RowsetMetaSharedPtr rs_meta, int64_t e txn_id, tablet_id, std::move(rs_meta), expiration_time); } -Status CloudMetaMgr::update_tmp_rowset(const RowsetMeta& rs_meta) { +Status CloudMetaMgr::update_tmp_rowset(const RowsetMeta& rs_meta, int64_t table_id) { VLOG_DEBUG << "update committed rowset, tablet_id: " << rs_meta.tablet_id() << ", rowset_id: " << rs_meta.rowset_id(); CreateRowsetRequest req; @@ -1408,8 +1541,13 @@ Status CloudMetaMgr::update_tmp_rowset(const RowsetMeta& rs_meta) { bool skip_schema = rs_meta.tablet_schema()->num_variant_columns() == 0; RowsetMetaPB rs_meta_pb = rs_meta.get_rowset_pb(skip_schema); doris_rowset_meta_to_cloud(req.mutable_rowset_meta(), std::move(rs_meta_pb)); - Status st = - retry_rpc("update committed rowset", req, &resp, &MetaService_Stub::update_tmp_rowset); + Status st = retry_rpc(MetaServiceRPC::UPDATE_TMP_ROWSET, req, &resp, + &MetaService_Stub::update_tmp_rowset, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + .table_id = table_id, + }); if (!st.ok() && resp.status().code() == MetaServiceCode::ROWSET_META_NOT_FOUND) { return Status::InternalError("failed to update committed rowset: {}", resp.status().msg()); } @@ -1489,7 +1627,11 @@ Status CloudMetaMgr::commit_txn(const StreamLoadContext& ctx, bool is_2pc) { req.set_txn_id(ctx.txn_id); req.set_is_2pc(is_2pc); req.set_enable_txn_lazy_commit(config::enable_cloud_txn_lazy_commit); - auto st = retry_rpc("commit txn", req, &res, &MetaService_Stub::commit_txn); + auto st = retry_rpc(MetaServiceRPC::COMMIT_TXN, req, &res, &MetaService_Stub::commit_txn, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); if (st.ok()) { std::vector tablet_ids; @@ -1523,7 +1665,11 @@ Status CloudMetaMgr::abort_txn(const StreamLoadContext& ctx) { << " txn_id=" << ctx.txn_id << " label=" << ctx.label; return Status::InternalError("failed to abort txn"); } - return retry_rpc("abort txn", req, &res, &MetaService_Stub::abort_txn); + return retry_rpc(MetaServiceRPC::ABORT_TXN, req, &res, &MetaService_Stub::abort_txn, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); } Status CloudMetaMgr::precommit_txn(const StreamLoadContext& ctx) { @@ -1538,7 +1684,11 @@ Status CloudMetaMgr::precommit_txn(const StreamLoadContext& ctx) { req.set_cloud_unique_id(config::cloud_unique_id); req.set_db_id(ctx.db_id); req.set_txn_id(ctx.txn_id); - return retry_rpc("precommit txn", req, &res, &MetaService_Stub::precommit_txn); + return retry_rpc(MetaServiceRPC::PRECOMMIT_TXN, req, &res, &MetaService_Stub::precommit_txn, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); } Status CloudMetaMgr::prepare_restore_job(const TabletMetaPB& tablet_meta) { @@ -1551,7 +1701,12 @@ Status CloudMetaMgr::prepare_restore_job(const TabletMetaPB& tablet_meta) { req.set_action(RestoreJobRequest::PREPARE); doris_tablet_meta_to_cloud(req.mutable_tablet_meta(), std::move(tablet_meta)); - return retry_rpc("prepare restore job", req, &resp, &MetaService_Stub::prepare_restore_job); + return retry_rpc(MetaServiceRPC::PREPARE_RESTORE_JOB, req, &resp, + &MetaService_Stub::prepare_restore_job, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); } Status CloudMetaMgr::commit_restore_job(const int64_t tablet_id) { @@ -1563,7 +1718,12 @@ Status CloudMetaMgr::commit_restore_job(const int64_t tablet_id) { req.set_action(RestoreJobRequest::COMMIT); req.set_store_version(config::delete_bitmap_store_write_version); - return retry_rpc("commit restore job", req, &resp, &MetaService_Stub::commit_restore_job); + return retry_rpc(MetaServiceRPC::COMMIT_RESTORE_JOB, req, &resp, + &MetaService_Stub::commit_restore_job, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); } Status CloudMetaMgr::finish_restore_job(const int64_t tablet_id, bool is_completed) { @@ -1575,15 +1735,24 @@ Status CloudMetaMgr::finish_restore_job(const int64_t tablet_id, bool is_complet req.set_tablet_id(tablet_id); req.set_action(is_completed ? RestoreJobRequest::COMPLETE : RestoreJobRequest::ABORT); - return retry_rpc("finish restore job", req, &resp, &MetaService_Stub::finish_restore_job); + return retry_rpc(MetaServiceRPC::FINISH_RESTORE_JOB, req, &resp, + &MetaService_Stub::finish_restore_job, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); } Status CloudMetaMgr::get_storage_vault_info(StorageVaultInfos* vault_infos, bool* is_vault_mode) { GetObjStoreInfoRequest req; GetObjStoreInfoResponse resp; req.set_cloud_unique_id(config::cloud_unique_id); - Status s = - retry_rpc("get storage vault info", req, &resp, &MetaService_Stub::get_obj_store_info); + Status s = retry_rpc(MetaServiceRPC::GET_OBJ_STORE_INFO, req, &resp, + &MetaService_Stub::get_obj_store_info, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); if (!s.ok()) { return s; } @@ -1636,7 +1805,12 @@ Status CloudMetaMgr::prepare_tablet_job(const TabletJobInfoPB& job, StartTabletJ StartTabletJobRequest req; req.mutable_job()->CopyFrom(job); req.set_cloud_unique_id(config::cloud_unique_id); - return retry_rpc("start tablet job", req, res, &MetaService_Stub::start_tablet_job); + return retry_rpc(MetaServiceRPC::START_TABLET_JOB, req, res, + &MetaService_Stub::start_tablet_job, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); } Status CloudMetaMgr::commit_tablet_job(const TabletJobInfoPB& job, FinishTabletJobResponse* res) { @@ -1650,7 +1824,12 @@ Status CloudMetaMgr::commit_tablet_job(const TabletJobInfoPB& job, FinishTabletJ req.mutable_job()->CopyFrom(job); req.set_action(FinishTabletJobRequest::COMMIT); req.set_cloud_unique_id(config::cloud_unique_id); - auto st = retry_rpc("commit tablet job", req, res, &MetaService_Stub::finish_tablet_job); + auto st = retry_rpc(MetaServiceRPC::FINISH_TABLET_JOB, req, res, + &MetaService_Stub::finish_tablet_job, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); if (res->status().code() == MetaServiceCode::KV_TXN_CONFLICT_RETRY_EXCEEDED_MAX_TIMES) { return Status::Error( "txn conflict when commit tablet job {}", job.ShortDebugString()); @@ -1673,7 +1852,12 @@ Status CloudMetaMgr::abort_tablet_job(const TabletJobInfoPB& job) { req.mutable_job()->CopyFrom(job); req.set_action(FinishTabletJobRequest::ABORT); req.set_cloud_unique_id(config::cloud_unique_id); - return retry_rpc("abort tablet job", req, &res, &MetaService_Stub::finish_tablet_job); + return retry_rpc(MetaServiceRPC::FINISH_TABLET_JOB, req, &res, + &MetaService_Stub::finish_tablet_job, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); } Status CloudMetaMgr::lease_tablet_job(const TabletJobInfoPB& job) { @@ -1683,7 +1867,12 @@ Status CloudMetaMgr::lease_tablet_job(const TabletJobInfoPB& job) { req.mutable_job()->CopyFrom(job); req.set_action(FinishTabletJobRequest::LEASE); req.set_cloud_unique_id(config::cloud_unique_id); - return retry_rpc("lease tablet job", req, &res, &MetaService_Stub::finish_tablet_job); + return retry_rpc(MetaServiceRPC::FINISH_TABLET_JOB, req, &res, + &MetaService_Stub::finish_tablet_job, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); } static void add_delete_bitmap(DeleteBitmapPB& delete_bitmap_pb, const DeleteBitmap::BitmapKey& key, @@ -1754,7 +1943,7 @@ Status CloudMetaMgr::update_delete_bitmap(const CloudTablet& tablet, int64_t loc int64_t initiator, DeleteBitmap* delete_bitmap, DeleteBitmap* delete_bitmap_v2, std::string rowset_id, std::optional storage_resource, - int64_t store_version, int64_t txn_id, + int64_t store_version, int64_t table_id, int64_t txn_id, bool is_explicit_txn, int64_t next_visible_version) { VLOG_DEBUG << "update_delete_bitmap , tablet_id: " << tablet.tablet_id(); if (config::enable_mow_verbose_log) { @@ -1872,7 +2061,13 @@ Status CloudMetaMgr::update_delete_bitmap(const CloudTablet& tablet, int64_t loc "test update delete bitmap failed, tablet_id: {}, lock_id: {}", tablet.tablet_id(), lock_id); }); - auto st = retry_rpc("update delete bitmap", req, &res, &MetaService_Stub::update_delete_bitmap); + auto st = retry_rpc(MetaServiceRPC::UPDATE_DELETE_BITMAP, req, &res, + &MetaService_Stub::update_delete_bitmap, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + .table_id = table_id, + }); if (config::enable_update_delete_bitmap_kv_check_core && res.status().code() == MetaServiceCode::UPDATE_OVERRIDE_EXISTING_KV) { auto& msg = res.status().msg(); @@ -1890,8 +2085,8 @@ Status CloudMetaMgr::update_delete_bitmap(const CloudTablet& tablet, int64_t loc Status CloudMetaMgr::cloud_update_delete_bitmap_without_lock( const CloudTablet& tablet, DeleteBitmap* delete_bitmap, - std::map& rowset_to_versions, int64_t pre_rowset_agg_start_version, - int64_t pre_rowset_agg_end_version) { + std::map& rowset_to_versions, int64_t table_id, + int64_t pre_rowset_agg_start_version, int64_t pre_rowset_agg_end_version) { if (config::delete_bitmap_store_write_version == 2) { VLOG_DEBUG << "no need to agg delete bitmap v1 in ms because use v2"; return Status::OK(); @@ -1930,7 +2125,13 @@ Status CloudMetaMgr::cloud_update_delete_bitmap_without_lock( req.set_pre_rowset_agg_start_version(pre_rowset_agg_start_version); req.set_pre_rowset_agg_end_version(pre_rowset_agg_end_version); } - return retry_rpc("update delete bitmap", req, &res, &MetaService_Stub::update_delete_bitmap); + return retry_rpc(MetaServiceRPC::UPDATE_DELETE_BITMAP, req, &res, + &MetaService_Stub::update_delete_bitmap, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + .table_id = table_id, + }); } Status CloudMetaMgr::get_delete_bitmap_update_lock(const CloudTablet& tablet, int64_t lock_id, @@ -1963,8 +2164,12 @@ Status CloudMetaMgr::get_delete_bitmap_update_lock(const CloudTablet& tablet, in uint64_t backoff_sleep_time_ms {0}; do { bool test_conflict = false; - st = retry_rpc("get delete bitmap update lock", req, &res, - &MetaService_Stub::get_delete_bitmap_update_lock); + st = retry_rpc(MetaServiceRPC::GET_DELETE_BITMAP_UPDATE_LOCK, req, &res, + &MetaService_Stub::get_delete_bitmap_update_lock, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); DBUG_EXECUTE_IF("CloudMetaMgr::test_get_delete_bitmap_update_lock_conflict", { test_conflict = true; }); if (!test_conflict && res.status().code() != MetaServiceCode::LOCK_CONFLICT) { @@ -2018,8 +2223,12 @@ void CloudMetaMgr::remove_delete_bitmap_update_lock(int64_t table_id, int64_t lo req.set_tablet_id(tablet_id); req.set_lock_id(lock_id); req.set_initiator(initiator); - auto st = retry_rpc("remove delete bitmap update lock", req, &res, - &MetaService_Stub::remove_delete_bitmap_update_lock); + auto st = retry_rpc(MetaServiceRPC::REMOVE_DELETE_BITMAP_UPDATE_LOCK, req, &res, + &MetaService_Stub::remove_delete_bitmap_update_lock, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + }); if (!st.ok()) { LOG(WARNING) << "remove delete bitmap update lock fail,table_id=" << table_id << ",tablet_id=" << tablet_id << ",lock_id=" << lock_id @@ -2298,7 +2507,12 @@ Status CloudMetaMgr::list_snapshot(std::vector& snapshots) { ListSnapshotResponse res; req.set_cloud_unique_id(config::cloud_unique_id); req.set_include_aborted(true); - RETURN_IF_ERROR(retry_rpc("list snapshot", req, &res, &MetaService_Stub::list_snapshot)); + RETURN_IF_ERROR(retry_rpc(MetaServiceRPC::LIST_SNAPSHOTS, req, &res, + &MetaService_Stub::list_snapshot, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + })); for (auto& snapshot : res.snapshots()) { snapshots.emplace_back(snapshot); } @@ -2311,8 +2525,12 @@ Status CloudMetaMgr::get_snapshot_properties(SnapshotSwitchStatus& switch_status GetInstanceRequest req; GetInstanceResponse res; req.set_cloud_unique_id(config::cloud_unique_id); - RETURN_IF_ERROR( - retry_rpc("get snapshot properties", req, &res, &MetaService_Stub::get_instance)); + RETURN_IF_ERROR(retry_rpc(MetaServiceRPC::GET_INSTANCE, req, &res, + &MetaService_Stub::get_instance, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + })); switch_status = res.instance().has_snapshot_switch_status() ? res.instance().snapshot_switch_status() : SnapshotSwitchStatus::SNAPSHOT_SWITCH_DISABLED; @@ -2325,7 +2543,8 @@ Status CloudMetaMgr::get_snapshot_properties(SnapshotSwitchStatus& switch_status } Status CloudMetaMgr::update_packed_file_info(const std::string& packed_file_path, - const cloud::PackedFileInfoPB& packed_file_info) { + const cloud::PackedFileInfoPB& packed_file_info, + int64_t table_id) { VLOG_DEBUG << "Updating meta service for packed file: " << packed_file_path << " with " << packed_file_info.total_slice_num() << " small files" << ", total bytes: " << packed_file_info.total_slice_bytes(); @@ -2340,8 +2559,13 @@ Status CloudMetaMgr::update_packed_file_info(const std::string& packed_file_path *req.mutable_packed_file_info() = packed_file_info; // Make RPC call using retry pattern - return retry_rpc("update packed file info", req, &resp, - &cloud::MetaService_Stub::update_packed_file_info); + return retry_rpc(MetaServiceRPC::UPDATE_PACKED_FILE_INFO, req, &resp, + &cloud::MetaService_Stub::update_packed_file_info, + { + .host_limiters = host_level_ms_rpc_rate_limiters_, + .backpressure_handler = ms_backpressure_handler_, + .table_id = table_id, + }); } Status CloudMetaMgr::get_cluster_status( @@ -2351,7 +2575,9 @@ Status CloudMetaMgr::get_cluster_status( GetClusterStatusResponse resp; req.add_cloud_unique_ids(config::cloud_unique_id); - Status s = retry_rpc("get cluster status", req, &resp, &MetaService_Stub::get_cluster_status); + Status s = retry_rpc(MetaServiceRPC::GET_CLUSTER_STATUS, req, &resp, + &MetaService_Stub::get_cluster_status, + {.host_limiters = host_level_ms_rpc_rate_limiters_}); if (!s.ok()) { return s; } diff --git a/be/src/cloud/cloud_meta_mgr.h b/be/src/cloud/cloud_meta_mgr.h index f181f5adf9b1d5..eb2a0f1ab9ced5 100644 --- a/be/src/cloud/cloud_meta_mgr.h +++ b/be/src/cloud/cloud_meta_mgr.h @@ -48,6 +48,8 @@ class StartTabletJobResponse; class TabletJobInfoPB; class TabletStatsPB; class TabletIndexPB; +class HostLevelMSRpcRateLimiters; +class MSBackpressureHandler; using StorageVaultInfos = std::vector< std::tuple, StorageVaultPB_PathFormat>>; @@ -77,17 +79,18 @@ class CloudMetaMgr { CloudTablet* tablet, std::unique_lock& lock /* _sync_meta_lock */, const SyncOptions& options = {}, SyncRowsetStats* sync_stats = nullptr); - Status prepare_rowset(const RowsetMeta& rs_meta, const std::string& job_id, + Status prepare_rowset(const RowsetMeta& rs_meta, const std::string& job_id, int64_t table_id, std::shared_ptr* existed_rs_meta = nullptr); - Status commit_rowset(RowsetMeta& rs_meta, const std::string& job_id, + Status commit_rowset(RowsetMeta& rs_meta, const std::string& job_id, int64_t table_id, std::shared_ptr* existed_rs_meta = nullptr); void cache_committed_rowset(RowsetMetaSharedPtr rs_meta, int64_t expiration_time); - Status update_tmp_rowset(const RowsetMeta& rs_meta); + Status update_tmp_rowset(const RowsetMeta& rs_meta, int64_t table_id); Status update_packed_file_info(const std::string& packed_file_path, - const cloud::PackedFileInfoPB& packed_file_info); + const cloud::PackedFileInfoPB& packed_file_info, + int64_t table_id); Status commit_txn(const StreamLoadContext& ctx, bool is_2pc); @@ -141,12 +144,12 @@ class CloudMetaMgr { DeleteBitmap* delete_bitmap, DeleteBitmap* delete_bitmap_v2, std::string rowset_id, std::optional storage_resource, - int64_t store_version, int64_t txn_id = -1, + int64_t store_version, int64_t table_id, int64_t txn_id = -1, bool is_explicit_txn = false, int64_t next_visible_version = -1); Status cloud_update_delete_bitmap_without_lock( const CloudTablet& tablet, DeleteBitmap* delete_bitmap, - std::map& rowset_to_versions, + std::map& rowset_to_versions, int64_t table_id, int64_t pre_rowset_agg_start_version = 0, int64_t pre_rowset_agg_end_version = 0); Status get_delete_bitmap_update_lock(const CloudTablet& tablet, int64_t lock_id, @@ -175,6 +178,14 @@ class CloudMetaMgr { Status get_cluster_status(std::unordered_map>* result, std::string* my_cluster_id = nullptr); + void set_host_level_ms_rpc_rate_limiters(HostLevelMSRpcRateLimiters* limiters) { + host_level_ms_rpc_rate_limiters_ = limiters; + } + + void set_ms_backpressure_handler(MSBackpressureHandler* handler) { + ms_backpressure_handler_ = handler; + } + private: bool sync_tablet_delete_bitmap_by_cache(CloudTablet* tablet, std::ranges::range auto&& rs_metas, DeleteBitmap* delete_bitmap); @@ -202,6 +213,9 @@ class CloudMetaMgr { void check_table_size_correctness(RowsetMeta& rs_meta); int64_t get_segment_file_size(RowsetMeta& rs_meta); int64_t get_inverted_index_file_size(RowsetMeta& rs_meta); + + HostLevelMSRpcRateLimiters* host_level_ms_rpc_rate_limiters_ {nullptr}; + MSBackpressureHandler* ms_backpressure_handler_ {nullptr}; }; } // namespace cloud diff --git a/be/src/cloud/cloud_ms_backpressure_handler.cpp b/be/src/cloud/cloud_ms_backpressure_handler.cpp new file mode 100644 index 00000000000000..e99cc281001f55 --- /dev/null +++ b/be/src/cloud/cloud_ms_backpressure_handler.cpp @@ -0,0 +1,521 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "cloud/cloud_ms_backpressure_handler.h" + +#include +#include + +#include +#include +#include +#include + +#include "cloud/config.h" +#include "common/status.h" +#include "util/thread.h" + +namespace doris::cloud { + +// Global bvar metrics +bvar::Adder g_backpressure_upgrade_count("ms_rpc_backpressure_upgrade_count"); +bvar::Window> g_backpressure_upgrade_60s("ms_rpc_backpressure_upgrade_60s", + &g_backpressure_upgrade_count, 60); +bvar::Adder g_backpressure_downgrade_count("ms_rpc_backpressure_downgrade_count"); +bvar::Window> g_backpressure_downgrade_60s( + "ms_rpc_backpressure_downgrade_60s", &g_backpressure_downgrade_count, 60); +bvar::LatencyRecorder g_throttle_wait_prepare_rowset( + "ms_rpc_backpressure_throttle_wait_prepare_rowset"); +bvar::LatencyRecorder g_throttle_wait_commit_rowset( + "ms_rpc_backpressure_throttle_wait_commit_rowset"); +bvar::LatencyRecorder g_throttle_wait_update_tmp_rowset( + "ms_rpc_backpressure_throttle_wait_update_tmp_rowset"); +bvar::LatencyRecorder g_throttle_wait_update_packed_file_info( + "ms_rpc_backpressure_throttle_wait_update_packed_file_info"); +bvar::LatencyRecorder g_throttle_wait_update_delete_bitmap( + "ms_rpc_backpressure_throttle_wait_update_delete_bitmap"); +bvar::Adder g_ms_busy_count("ms_rpc_backpressure_ms_busy_count"); +bvar::Window> g_ms_busy_60s("ms_rpc_backpressure_ms_busy_60s", + &g_ms_busy_count, 60); + +static bvar::LatencyRecorder* s_throttle_wait_recorders[] = { + &g_throttle_wait_prepare_rowset, &g_throttle_wait_commit_rowset, + &g_throttle_wait_update_tmp_rowset, &g_throttle_wait_update_packed_file_info, + &g_throttle_wait_update_delete_bitmap, +}; + +bvar::LatencyRecorder* get_throttle_wait_recorder(LoadRelatedRpc rpc) { + size_t idx = static_cast(rpc); + if (idx >= static_cast(LoadRelatedRpc::COUNT)) { + return nullptr; + } + return s_throttle_wait_recorders[idx]; +} + +// ============== StrictQpsLimiter ============== + +StrictQpsLimiter::StrictQpsLimiter(double qps) { + if (qps <= 0) { + qps = 1.0; + } + _interval_ns = static_cast(1e9 / qps); + _next_allowed_time = Clock::now(); +} + +StrictQpsLimiter::Clock::time_point StrictQpsLimiter::reserve() { + std::lock_guard lock(_mtx); + auto now = Clock::now(); + if (_next_allowed_time <= now) { + _next_allowed_time = now + std::chrono::nanoseconds(_interval_ns); + return now; + } + auto result = _next_allowed_time; + _next_allowed_time += std::chrono::nanoseconds(_interval_ns); + return result; +} + +void StrictQpsLimiter::update_qps(double new_qps) { + if (new_qps <= 0) { + new_qps = 1.0; + } + std::lock_guard lock(_mtx); + _interval_ns = static_cast(1e9 / new_qps); +} + +double StrictQpsLimiter::get_qps() const { + std::lock_guard lock(_mtx); + if (_interval_ns <= 0) { + return 0; + } + return 1e9 / _interval_ns; +} + +// ============== TableRpcQpsCounter ============== + +TableRpcQpsCounter::TableRpcQpsCounter(int64_t table_id, LoadRelatedRpc rpc_type, int window_sec) + : _table_id(table_id), _rpc_type(rpc_type) { + _counter = std::make_unique>(); + _counter->hide(); + _qps = std::make_unique>>(_counter.get(), window_sec); + _qps->hide(); +} + +void TableRpcQpsCounter::increment() { + (*_counter) << 1; +} + +double TableRpcQpsCounter::get_qps() const { + return _qps->get_value(); +} + +// ============== TableRpcQpsRegistry ============== + +TableRpcQpsRegistry::TableRpcQpsRegistry() = default; + +void TableRpcQpsRegistry::record(LoadRelatedRpc rpc_type, int64_t table_id) { + auto* counter = get_or_create_counter(rpc_type, table_id); + if (counter) { + counter->increment(); + } +} + +TableRpcQpsCounter* TableRpcQpsRegistry::get_or_create_counter(LoadRelatedRpc rpc_type, + int64_t table_id) { + size_t idx = static_cast(rpc_type); + if (idx >= static_cast(LoadRelatedRpc::COUNT)) { + return nullptr; + } + + { + std::shared_lock lock(_mutex); + auto it = _counters[idx].find(table_id); + if (it != _counters[idx].end()) { + return it->second.get(); + } + } + + std::unique_lock lock(_mutex); + // Double check after acquiring exclusive lock + auto it = _counters[idx].find(table_id); + if (it != _counters[idx].end()) { + return it->second.get(); + } + + auto counter = std::make_unique(table_id, rpc_type, + config::ms_rpc_table_qps_window_sec); + auto* ptr = counter.get(); + _counters[idx][table_id] = std::move(counter); + return ptr; +} + +std::vector> TableRpcQpsRegistry::get_top_k_tables( + LoadRelatedRpc rpc_type, int k) const { + size_t idx = static_cast(rpc_type); + if (idx >= static_cast(LoadRelatedRpc::COUNT) || k <= 0) { + return {}; + } + + // Use a min-heap of size k to find top-k without allocating a vector for all tables. + // The heap top is the smallest among the k largest elements seen so far. + using Entry = std::pair; // (table_id, qps) + auto min_cmp = [](const Entry& a, const Entry& b) { return a.second > b.second; }; + std::priority_queue, decltype(min_cmp)> min_heap(min_cmp); + + { + std::shared_lock lock(_mutex); + for (const auto& [table_id, counter] : _counters[idx]) { + double qps = counter->get_qps(); + if (qps > 0) { + if (static_cast(min_heap.size()) < k) { + min_heap.push({table_id, qps}); + } else if (qps > min_heap.top().second) { + min_heap.pop(); + min_heap.push({table_id, qps}); + } + } + } + } + + // Extract results from heap (comes out in ascending order, reverse to descending) + std::vector result; + result.reserve(min_heap.size()); + while (!min_heap.empty()) { + result.push_back(min_heap.top()); + min_heap.pop(); + } + std::reverse(result.begin(), result.end()); + + return result; +} + +double TableRpcQpsRegistry::get_qps(LoadRelatedRpc rpc_type, int64_t table_id) const { + size_t idx = static_cast(rpc_type); + if (idx >= static_cast(LoadRelatedRpc::COUNT)) { + return 0; + } + + std::shared_lock lock(_mutex); + auto it = _counters[idx].find(table_id); + if (it != _counters[idx].end()) { + return it->second->get_qps(); + } + return 0; +} + +void TableRpcQpsRegistry::cleanup_inactive_tables() { + std::unique_lock lock(_mutex); + + for (size_t idx = 0; idx < static_cast(LoadRelatedRpc::COUNT); ++idx) { + auto& counter_map = _counters[idx]; + for (auto it = counter_map.begin(); it != counter_map.end();) { + // Remove counters with zero QPS for a long time + if (it->second->get_qps() < 0.01) { + it = counter_map.erase(it); + } else { + ++it; + } + } + } +} + +// ============== TableRpcThrottler ============== + +TableRpcThrottler::TableRpcThrottler() { + // Initialize bvar for throttled table counts + for (size_t i = 0; i < static_cast(LoadRelatedRpc::COUNT); ++i) { + std::string bvar_name = fmt::format("ms_rpc_backpressure_throttled_tables_{}", + load_related_rpc_name(static_cast(i))); + _throttled_table_counts[i] = std::make_unique>(bvar_name, 0); + } +} + +std::chrono::steady_clock::time_point TableRpcThrottler::throttle(LoadRelatedRpc rpc_type, + int64_t table_id) { + std::shared_lock lock(_mutex); + auto it = _limiters.find({rpc_type, table_id}); + if (it == _limiters.end()) { + return std::chrono::steady_clock::now(); + } + return it->second->reserve(); +} + +void TableRpcThrottler::set_qps_limit(LoadRelatedRpc rpc_type, int64_t table_id, double qps_limit) { + if (qps_limit <= 0) { + return; + } + + std::unique_lock lock(_mutex); + auto key = std::make_pair(rpc_type, table_id); + auto it = _limiters.find(key); + if (it != _limiters.end()) { + it->second->update_qps(qps_limit); + } else { + _limiters[key] = std::make_unique(qps_limit); + // Update bvar count + size_t idx = static_cast(rpc_type); + if (idx < static_cast(LoadRelatedRpc::COUNT)) { + size_t count = 0; + for (const auto& [k, _] : _limiters) { + if (k.first == rpc_type) { + ++count; + } + } + _throttled_table_counts[idx]->set_value(count); + } + } + + LOG(INFO) << "[ms-throttle] set table QPS limit: rpc=" << load_related_rpc_name(rpc_type) + << ", table_id=" << table_id << ", qps_limit=" << qps_limit; +} + +void TableRpcThrottler::remove_qps_limit(LoadRelatedRpc rpc_type, int64_t table_id) { + std::unique_lock lock(_mutex); + auto key = std::make_pair(rpc_type, table_id); + auto it = _limiters.find(key); + if (it != _limiters.end()) { + _limiters.erase(it); + // Update bvar count + size_t idx = static_cast(rpc_type); + if (idx < static_cast(LoadRelatedRpc::COUNT)) { + size_t count = 0; + for (const auto& [k, _] : _limiters) { + if (k.first == rpc_type) { + ++count; + } + } + _throttled_table_counts[idx]->set_value(count); + } + + LOG(INFO) << "[ms-throttle] removed table QPS limit: rpc=" + << load_related_rpc_name(rpc_type) << ", table_id=" << table_id; + } +} + +double TableRpcThrottler::get_qps_limit(LoadRelatedRpc rpc_type, int64_t table_id) const { + std::shared_lock lock(_mutex); + auto it = _limiters.find({rpc_type, table_id}); + if (it != _limiters.end()) { + return it->second->get_qps(); + } + return 0; +} + +bool TableRpcThrottler::has_limit(LoadRelatedRpc rpc_type, int64_t table_id) const { + std::shared_lock lock(_mutex); + return _limiters.find({rpc_type, table_id}) != _limiters.end(); +} + +size_t TableRpcThrottler::get_throttled_table_count(LoadRelatedRpc rpc_type) const { + size_t idx = static_cast(rpc_type); + if (idx >= static_cast(LoadRelatedRpc::COUNT)) { + return 0; + } + return _throttled_table_counts[idx]->get_value(); +} + +std::vector TableRpcThrottler::get_all_throttled_entries() const { + std::shared_lock lock(_mutex); + std::vector entries; + entries.reserve(_limiters.size()); + for (const auto& [key, limiter] : _limiters) { + entries.push_back({key.first, key.second, limiter->get_qps()}); + } + return entries; +} + +// ============== MSBackpressureHandler ============== + +MSBackpressureHandler::MSBackpressureHandler(TableRpcQpsRegistry* qps_registry, + TableRpcThrottler* throttler) + : _qps_registry(qps_registry), + _throttler(throttler), + _stop_latch(1), + _last_ms_busy_time(std::chrono::steady_clock::time_point::min()) { + // Initialize state machine with config values + RpcThrottleParams throttle_params { + .top_k = config::ms_backpressure_upgrade_top_k, + .ratio = config::ms_backpressure_throttle_ratio, + .floor_qps = config::ms_rpc_table_qps_limit_floor, + }; + _state_machine = std::make_unique(throttle_params); + + // Initialize coordinator with config values + // Coordinator uses ticks where 1 tick = 1 millisecond (fixed unit) + // This allows tick_interval_ms to change at runtime without affecting correctness + ThrottleCoordinatorParams coordinator_params { + .upgrade_cooldown_ticks = config::ms_backpressure_upgrade_interval_ms, + .downgrade_after_ticks = config::ms_backpressure_downgrade_interval_ms, + }; + _coordinator = std::make_unique(coordinator_params); + + auto st = Thread::create( + "MSBackpressureHandler", "tick_thread", [this]() { this->_tick_thread_callback(); }, + &_tick_thread); + if (!st.ok()) { + LOG(WARNING) << "[ms-throttle] failed to create tick thread: " << st; + } else { + LOG(INFO) << "[ms-throttle] handler started: upgrade_cooldown=" + << config::ms_backpressure_upgrade_interval_ms + << "ms, downgrade_interval=" << config::ms_backpressure_downgrade_interval_ms + << "ms"; + } +} + +MSBackpressureHandler::~MSBackpressureHandler() { + _stop_latch.count_down(); + if (_tick_thread) { + _tick_thread->join(); + } +} + +void MSBackpressureHandler::_tick_thread_callback() { + // Fixed tick interval: 1 second. Since 1 tick = 1 ms, advance by 1000 ticks each iteration. + constexpr int kTickIntervalMs = 1000; + while (!_stop_latch.wait_for(std::chrono::milliseconds(kTickIntervalMs))) { + _advance_time(kTickIntervalMs); + } +} + +void MSBackpressureHandler::_advance_time(int ticks) { + if (!config::enable_ms_backpressure_handling) { + return; + } + + std::lock_guard lock(_transition_mutex); + + // Advance coordinator time; if downgrade is triggered, handle it + if (_coordinator->tick(ticks)) { + LOG(INFO) << "[ms-throttle] triggering downgrade, upgrade_level=" + << _state_machine->upgrade_level(); + + auto actions = _state_machine->on_downgrade(); + _apply_actions(actions); + _coordinator->set_has_pending_upgrades(_state_machine->upgrade_level() > 0); + + g_backpressure_downgrade_count << 1; + } +} + +bool MSBackpressureHandler::on_ms_busy() { + g_ms_busy_count << 1; + + if (!config::enable_ms_backpressure_handling) { + return false; + } + + { + std::lock_guard lock(_mutex); + _last_ms_busy_time = std::chrono::steady_clock::now(); + } + + std::lock_guard lock(_transition_mutex); + + // Check with coordinator if upgrade should be triggered + if (!_coordinator->report_ms_busy()) { + return false; + } + + LOG(INFO) << "[ms-throttle] received MS_BUSY, triggering upgrade"; + + auto snapshot = _build_qps_snapshot(); + auto actions = _state_machine->on_upgrade(snapshot); + _apply_actions(actions); + _coordinator->set_has_pending_upgrades(_state_machine->upgrade_level() > 0); + + g_backpressure_upgrade_count << 1; + return true; +} + +std::chrono::steady_clock::time_point MSBackpressureHandler::before_rpc(LoadRelatedRpc rpc_type, + int64_t table_id) { + if (!config::enable_ms_backpressure_handling) { + return std::chrono::steady_clock::now(); + } + + return _throttler->throttle(rpc_type, table_id); +} + +void MSBackpressureHandler::after_rpc(LoadRelatedRpc rpc_type, int64_t table_id) { + if (!config::enable_ms_backpressure_handling) { + return; + } + + _qps_registry->record(rpc_type, table_id); +} + +void MSBackpressureHandler::update_throttle_params(RpcThrottleParams params) { + _state_machine->update_params(params); +} + +void MSBackpressureHandler::update_coordinator_params(ThrottleCoordinatorParams params) { + _coordinator->update_params(params); +} + +int64_t MSBackpressureHandler::seconds_since_last_ms_busy() const { + std::lock_guard lock(_mutex); + if (_last_ms_busy_time == std::chrono::steady_clock::time_point::min()) { + return -1; // Never received MS_BUSY + } + return std::chrono::duration_cast(std::chrono::steady_clock::now() - + _last_ms_busy_time) + .count(); +} + +size_t MSBackpressureHandler::upgrade_level() const { + return _state_machine->upgrade_level(); +} + +int MSBackpressureHandler::ticks_since_last_ms_busy() const { + return _coordinator->ticks_since_last_ms_busy(); +} + +int MSBackpressureHandler::ticks_since_last_upgrade() const { + return _coordinator->ticks_since_last_upgrade(); +} + +void MSBackpressureHandler::_apply_actions(const std::vector& actions) { + for (const auto& action : actions) { + switch (action.type) { + case RpcThrottleAction::Type::SET_LIMIT: + _throttler->set_qps_limit(action.rpc_type, action.table_id, action.qps_limit); + break; + case RpcThrottleAction::Type::REMOVE_LIMIT: + _throttler->remove_qps_limit(action.rpc_type, action.table_id); + break; + } + } +} + +std::vector MSBackpressureHandler::_build_qps_snapshot() const { + std::vector snapshot; + + // For each RPC type, get top-k tables + int top_k = _state_machine->get_params().top_k; + + for (size_t i = 0; i < static_cast(LoadRelatedRpc::COUNT); ++i) { + LoadRelatedRpc rpc_type = static_cast(i); + auto top_tables = _qps_registry->get_top_k_tables(rpc_type, top_k); + + for (const auto& [table_id, qps] : top_tables) { + snapshot.push_back({rpc_type, table_id, qps}); + } + } + + return snapshot; +} + +} // namespace doris::cloud diff --git a/be/src/cloud/cloud_ms_backpressure_handler.h b/be/src/cloud/cloud_ms_backpressure_handler.h new file mode 100644 index 00000000000000..f32100c2c4695a --- /dev/null +++ b/be/src/cloud/cloud_ms_backpressure_handler.h @@ -0,0 +1,231 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "cloud/cloud_throttle_state_machine.h" +#include "util/countdown_latch.h" +#include "util/thread.h" + +namespace doris::cloud { + +// Strict QPS limiter that doesn't allow burst +// Unlike token bucket, it strictly enforces fixed intervals between requests +class StrictQpsLimiter { +public: + using Clock = std::chrono::steady_clock; + + explicit StrictQpsLimiter(double qps); + + // Returns the time point when the request is allowed to execute + // Caller should sleep until this time point + Clock::time_point reserve(); + + // Dynamically update the QPS limit + void update_qps(double new_qps); + + // Get current QPS limit + double get_qps() const; + +private: + mutable std::mutex _mtx; + int64_t _interval_ns; + Clock::time_point _next_allowed_time; +}; + +// QPS counter for a single (table, RPC type) pair using bvar +class TableRpcQpsCounter { +public: + TableRpcQpsCounter(int64_t table_id, LoadRelatedRpc rpc_type, int window_sec); + ~TableRpcQpsCounter() = default; + + // Record one RPC call + void increment(); + + // Get current QPS (average over the configured time window) + double get_qps() const; + + int64_t table_id() const { return _table_id; } + LoadRelatedRpc rpc_type() const { return _rpc_type; } + +private: + int64_t _table_id; + LoadRelatedRpc _rpc_type; + + std::unique_ptr> _counter; + std::unique_ptr>> _qps; +}; + +// Registry managing QPS counters for all tables +class TableRpcQpsRegistry { +public: + TableRpcQpsRegistry(); + ~TableRpcQpsRegistry() = default; + + // Record one RPC call for the given table + void record(LoadRelatedRpc rpc_type, int64_t table_id); + + // Get the top-k tables with highest QPS for the given RPC type + // Returns: [(table_id, qps), ...] sorted by qps in descending order + std::vector> get_top_k_tables(LoadRelatedRpc rpc_type, int k) const; + + // Get QPS for a specific table on a specific RPC type + double get_qps(LoadRelatedRpc rpc_type, int64_t table_id) const; + + // Clean up counters for tables that have been inactive for a long time + void cleanup_inactive_tables(); + +private: + // Get or create counter for (rpc_type, table_id) + TableRpcQpsCounter* get_or_create_counter(LoadRelatedRpc rpc_type, int64_t table_id); + + mutable std::shared_mutex _mutex; + + // rpc_type -> (table_id -> counter) + std::array>, + static_cast(LoadRelatedRpc::COUNT)> + _counters; +}; + +// Table-level throttler managing StrictQpsLimiter for each (RPC type, table) pair +class TableRpcThrottler { +public: + TableRpcThrottler(); + ~TableRpcThrottler() = default; + + // Called before RPC execution, returns the time point when execution is allowed + // Returns now if no limit is set + std::chrono::steady_clock::time_point throttle(LoadRelatedRpc rpc_type, int64_t table_id); + + // Set or update the QPS limit for a table + void set_qps_limit(LoadRelatedRpc rpc_type, int64_t table_id, double qps_limit); + + // Remove the QPS limit for a table + void remove_qps_limit(LoadRelatedRpc rpc_type, int64_t table_id); + + // Get current QPS limit (returns 0 if not set) + double get_qps_limit(LoadRelatedRpc rpc_type, int64_t table_id) const; + + // Check if a limit exists for the given (rpc_type, table_id) + bool has_limit(LoadRelatedRpc rpc_type, int64_t table_id) const; + + // Get the number of throttled tables for a given RPC type + size_t get_throttled_table_count(LoadRelatedRpc rpc_type) const; + + // Get all currently throttled entries: (rpc_type, table_id, qps_limit) + struct ThrottleEntry { + LoadRelatedRpc rpc_type; + int64_t table_id; + double qps_limit; + }; + std::vector get_all_throttled_entries() const; + +private: + mutable std::shared_mutex _mutex; + // (rpc_type, table_id) -> StrictQpsLimiter + std::map, std::unique_ptr> _limiters; + + // bvar: current throttled table count per RPC type + std::array>, static_cast(LoadRelatedRpc::COUNT)> + _throttled_table_counts; +}; + +// MS backpressure handler that coordinates QPS statistics, throttle upgrade and downgrade +// Uses state machine for decisions, providing better testability +class MSBackpressureHandler { +public: + MSBackpressureHandler(TableRpcQpsRegistry* qps_registry, TableRpcThrottler* throttler); + ~MSBackpressureHandler(); + + // Called when receiving MS_BUSY response + // Returns true if throttle upgrade was triggered + bool on_ms_busy(); + + // Called before RPC execution, performs throttle wait + // Returns the time point to wait until + std::chrono::steady_clock::time_point before_rpc(LoadRelatedRpc rpc_type, int64_t table_id); + + // Called after RPC execution, records QPS statistics + void after_rpc(LoadRelatedRpc rpc_type, int64_t table_id); + + // Runtime update parameters + void update_throttle_params(RpcThrottleParams params); + void update_coordinator_params(ThrottleCoordinatorParams params); + + // Get seconds since last MS_BUSY (for monitoring) + int64_t seconds_since_last_ms_busy() const; + + // Query current state + size_t upgrade_level() const; + int ticks_since_last_ms_busy() const; + int ticks_since_last_upgrade() const; + +private: + // Background thread that periodically advances time + void _tick_thread_callback(); + + // Advance time by specified ticks, handle any triggered events (e.g., downgrade) + void _advance_time(int ticks); + + // Apply actions to the throttler + void _apply_actions(const std::vector& actions); + + // Build QPS snapshot from registry + std::vector _build_qps_snapshot() const; + + TableRpcQpsRegistry* _qps_registry; + TableRpcThrottler* _throttler; + mutable std::mutex _transition_mutex; + + // State machine components + std::unique_ptr _state_machine; + std::unique_ptr _coordinator; + + // Background thread for periodic tick + std::shared_ptr _tick_thread; + CountDownLatch _stop_latch; + + // For bvar compatibility only - track approximate seconds since last MS_BUSY + mutable std::mutex _mutex; + std::chrono::steady_clock::time_point _last_ms_busy_time; +}; + +// Global bvar metrics for backpressure handling +extern bvar::Adder g_backpressure_upgrade_count; +extern bvar::Window> g_backpressure_upgrade_60s; +extern bvar::Adder g_backpressure_downgrade_count; +extern bvar::Window> g_backpressure_downgrade_60s; +extern bvar::Adder g_ms_busy_count; +extern bvar::Window> g_ms_busy_60s; + +// Per-RPC-type throttle wait latency recorders +bvar::LatencyRecorder* get_throttle_wait_recorder(LoadRelatedRpc rpc); + +} // namespace doris::cloud diff --git a/be/src/cloud/cloud_ms_rpc_rate_limit_services.cpp b/be/src/cloud/cloud_ms_rpc_rate_limit_services.cpp new file mode 100644 index 00000000000000..360864208c445e --- /dev/null +++ b/be/src/cloud/cloud_ms_rpc_rate_limit_services.cpp @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "cloud/cloud_ms_rpc_rate_limit_services.h" + +#include "cloud/cloud_ms_backpressure_handler.h" +#include "cloud/cloud_ms_rpc_rate_limiters.h" + +namespace doris { + +MSRpcRateLimitServices::MSRpcRateLimitServices() + : _host_level_ms_rpc_rate_limiters(std::make_unique()), + _table_rpc_qps_registry(std::make_unique()), + _table_rpc_throttler(std::make_unique()), + _ms_backpressure_handler(std::make_unique( + _table_rpc_qps_registry.get(), _table_rpc_throttler.get())) {} + +MSRpcRateLimitServices::~MSRpcRateLimitServices() = default; + +void MSRpcRateLimitServices::reset_host_level_rate_limiters() { + _host_level_ms_rpc_rate_limiters->reset_all(); +} + +void MSRpcRateLimitServices::update_backpressure_throttle_params(int top_k, double ratio, + double floor_qps) { + _ms_backpressure_handler->update_throttle_params({ + .top_k = top_k, + .ratio = ratio, + .floor_qps = floor_qps, + }); +} + +void MSRpcRateLimitServices::update_backpressure_coordinator_params(int upgrade_cooldown_ticks, + int downgrade_after_ticks) { + _ms_backpressure_handler->update_coordinator_params({ + .upgrade_cooldown_ticks = upgrade_cooldown_ticks, + .downgrade_after_ticks = downgrade_after_ticks, + }); +} + +} // namespace doris diff --git a/be/src/cloud/cloud_ms_rpc_rate_limit_services.h b/be/src/cloud/cloud_ms_rpc_rate_limit_services.h new file mode 100644 index 00000000000000..81a5990850a799 --- /dev/null +++ b/be/src/cloud/cloud_ms_rpc_rate_limit_services.h @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +namespace doris { +namespace cloud { +class HostLevelMSRpcRateLimiters; +class TableRpcQpsRegistry; +class TableRpcThrottler; +class MSBackpressureHandler; +} // namespace cloud + +class MSRpcRateLimitServices { +public: + MSRpcRateLimitServices(); + ~MSRpcRateLimitServices(); + + cloud::HostLevelMSRpcRateLimiters* host_level_ms_rpc_rate_limiters() { + return _host_level_ms_rpc_rate_limiters.get(); + } + + cloud::TableRpcQpsRegistry* table_rpc_qps_registry() { return _table_rpc_qps_registry.get(); } + + cloud::TableRpcThrottler* table_rpc_throttler() { return _table_rpc_throttler.get(); } + + cloud::MSBackpressureHandler* ms_backpressure_handler() { + return _ms_backpressure_handler.get(); + } + + void reset_host_level_rate_limiters(); + void update_backpressure_throttle_params(int top_k, double ratio, double floor_qps); + void update_backpressure_coordinator_params(int upgrade_cooldown_ticks, + int downgrade_after_ticks); + +private: + std::unique_ptr _host_level_ms_rpc_rate_limiters; + std::unique_ptr _table_rpc_qps_registry; + std::unique_ptr _table_rpc_throttler; + std::unique_ptr _ms_backpressure_handler; +}; + +} // namespace doris diff --git a/be/src/cloud/cloud_ms_rpc_rate_limiters.cpp b/be/src/cloud/cloud_ms_rpc_rate_limiters.cpp new file mode 100644 index 00000000000000..74fe85c575e5b7 --- /dev/null +++ b/be/src/cloud/cloud_ms_rpc_rate_limiters.cpp @@ -0,0 +1,189 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "cloud/cloud_ms_rpc_rate_limiters.h" + +#include + +#include + +#include "cloud/config.h" +#include "util/cpu_info.h" + +namespace doris::cloud { + +// Display names for each RPC type (used in bvar metrics) +static constexpr std::string_view META_SERVICE_RPC_DISPLAY_NAMES[] = { +#define DEFINE_DISPLAY_NAME(enum_name, config_suffix, display_name) display_name, + META_SERVICE_RPC_TYPES(DEFINE_DISPLAY_NAME) +#undef DEFINE_DISPLAY_NAME +}; + +std::string_view meta_service_rpc_display_name(MetaServiceRPC rpc) { + size_t idx = static_cast(rpc); + if (idx < static_cast(MetaServiceRPC::COUNT)) { + return META_SERVICE_RPC_DISPLAY_NAMES[idx]; + } + return "unknown"; +} + +// Get QPS config value for each RPC type +// Returns actual QPS (already multiplied by num_cores) +// Returns 0 if rate limiting should be disabled for this RPC +static int get_rpc_qps_from_config(MetaServiceRPC rpc) { + int num_cores = CpuInfo::num_cores(); + int qps_per_core = 0; + + // Get the per-RPC config value, -1 means use default +#define GET_RPC_QPS_CONFIG(enum_name, config_suffix, display_name) \ + case MetaServiceRPC::enum_name: \ + qps_per_core = config::ms_rpc_qps_##config_suffix; \ + break; + + switch (rpc) { + META_SERVICE_RPC_TYPES(GET_RPC_QPS_CONFIG) + default: + return 0; + } +#undef GET_RPC_QPS_CONFIG + + // -1 means use default config + if (qps_per_core < 0) { + qps_per_core = config::ms_rpc_qps_default; + } + + // 0 means disabled + if (qps_per_core <= 0) { + return 0; + } + + return std::max(1, qps_per_core * num_cores); +} + +RpcRateLimiter::RpcRateLimiter(int qps, std::string_view op_name) { + latency_recorder = std::make_unique("host_level_ms_rpc_rate_limit_sleep", + std::string(op_name)); + limiter = std::make_unique( + qps, qps, /*limit=*/0, [this](int64_t sleep_ns) { + if (sleep_ns > 0) { + // Convert ns to us for LatencyRecorder + *latency_recorder << (sleep_ns / 1000); + } + }); +} + +void RpcRateLimiter::reset(int qps) { + limiter->reset(qps, qps, 0); +} + +HostLevelMSRpcRateLimiters::HostLevelMSRpcRateLimiters() { + init_from_config(); +} + +HostLevelMSRpcRateLimiters::HostLevelMSRpcRateLimiters(int uniform_qps) { + init_with_uniform_qps(uniform_qps); +} + +void HostLevelMSRpcRateLimiters::init_from_config() { + LOG(INFO) << "Initializing MS RPC rate limiters from config"; + + // Initialize rate limiters for all RPC types + for (size_t i = 0; i < static_cast(MetaServiceRPC::COUNT); ++i) { + MetaServiceRPC rpc = static_cast(i); + int qps = get_rpc_qps_from_config(rpc); + if (qps > 0) { + _limiters[i].store( + std::make_shared(qps, meta_service_rpc_display_name(rpc))); + LOG(INFO) << " " << meta_service_rpc_display_name(rpc) << ": qps=" << qps; + } else { + _limiters[i].store(nullptr); + LOG(INFO) << " " << meta_service_rpc_display_name(rpc) << ": disabled"; + } + } +} + +void HostLevelMSRpcRateLimiters::init_with_uniform_qps(int qps) { + qps = std::max(qps, 1); + LOG(INFO) << "Initializing MS RPC rate limiters with uniform qps=" << qps; + + // Initialize rate limiters for all RPC types with the same QPS + for (size_t i = 0; i < static_cast(MetaServiceRPC::COUNT); ++i) { + MetaServiceRPC rpc = static_cast(i); + _limiters[i].store( + std::make_shared(qps, meta_service_rpc_display_name(rpc))); + } +} + +int64_t HostLevelMSRpcRateLimiters::limit(MetaServiceRPC rpc) { + if (!config::enable_ms_rpc_host_level_rate_limit) { + return 0; + } + + size_t idx = static_cast(rpc); + if (idx >= static_cast(MetaServiceRPC::COUNT)) { + return 0; + } + + auto limiter = _limiters[idx].load(); + if (limiter && limiter->limiter) { + return limiter->limiter->add(1); + } + return 0; +} + +void HostLevelMSRpcRateLimiters::reset(MetaServiceRPC rpc, int qps) { + size_t idx = static_cast(rpc); + if (idx >= static_cast(MetaServiceRPC::COUNT)) { + return; + } + + qps = std::max(qps, 1); + LOG(INFO) << "Resetting MS RPC rate limiter for " << meta_service_rpc_display_name(rpc) + << " with qps=" << qps; + + auto limiter = _limiters[idx].load(); + if (limiter) { + limiter->reset(qps); + } else { + _limiters[idx].store( + std::make_shared(qps, meta_service_rpc_display_name(rpc))); + } +} + +void HostLevelMSRpcRateLimiters::reset_all() { + LOG(INFO) << "Resetting all MS RPC rate limiters from config"; + + for (size_t i = 0; i < static_cast(MetaServiceRPC::COUNT); ++i) { + MetaServiceRPC rpc = static_cast(i); + int qps = get_rpc_qps_from_config(rpc); + if (qps > 0) { + auto limiter = _limiters[i].load(); + if (limiter) { + limiter->reset(qps); + } else { + _limiters[i].store( + std::make_shared(qps, meta_service_rpc_display_name(rpc))); + } + LOG(INFO) << " " << meta_service_rpc_display_name(rpc) << ": qps=" << qps; + } else { + _limiters[i].store(nullptr); + LOG(INFO) << " " << meta_service_rpc_display_name(rpc) << ": disabled"; + } + } +} + +} // namespace doris::cloud diff --git a/be/src/cloud/cloud_ms_rpc_rate_limiters.h b/be/src/cloud/cloud_ms_rpc_rate_limiters.h new file mode 100644 index 00000000000000..17a4bf465ab6a7 --- /dev/null +++ b/be/src/cloud/cloud_ms_rpc_rate_limiters.h @@ -0,0 +1,118 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include +#include +#include +#include + +#include "common/atomic_shared_ptr.h" +#include "cpp/token_bucket_rate_limiter.h" + +namespace doris::cloud { + +// Macro to define all Meta Service RPC types +// Usage: META_SERVICE_RPC_TYPES(X) where X(enum_name, config_suffix, display_name) +// - enum_name: the enum value name (e.g., GET_TABLET_META) +// - config_suffix: suffix for the config name (e.g., get_tablet_meta -> ms_rpc_qps_get_tablet_meta) +// - display_name: the human-readable name for bvar metrics (e.g., "get tablet meta") +#define META_SERVICE_RPC_TYPES(X) \ + X(GET_TABLET_META, get_tablet_meta, "get tablet meta") \ + X(GET_ROWSET, get_rowset, "get rowset") \ + X(PREPARE_ROWSET, prepare_rowset, "prepare rowset") \ + X(COMMIT_ROWSET, commit_rowset, "commit rowset") \ + X(UPDATE_TMP_ROWSET, update_tmp_rowset, "update tmp rowset") \ + X(COMMIT_TXN, commit_txn, "commit txn") \ + X(ABORT_TXN, abort_txn, "abort txn") \ + X(PRECOMMIT_TXN, precommit_txn, "precommit txn") \ + X(GET_OBJ_STORE_INFO, get_obj_store_info, "get obj store info") \ + X(START_TABLET_JOB, start_tablet_job, "start tablet job") \ + X(FINISH_TABLET_JOB, finish_tablet_job, "finish tablet job") \ + X(GET_DELETE_BITMAP, get_delete_bitmap, "get delete bitmap") \ + X(UPDATE_DELETE_BITMAP, update_delete_bitmap, "update delete bitmap") \ + X(GET_DELETE_BITMAP_UPDATE_LOCK, get_delete_bitmap_update_lock, \ + "get delete bitmap update lock") \ + X(REMOVE_DELETE_BITMAP_UPDATE_LOCK, remove_delete_bitmap_update_lock, \ + "remove delete bitmap update lock") \ + X(GET_INSTANCE, get_instance, "get instance") \ + X(PREPARE_RESTORE_JOB, prepare_restore_job, "prepare restore job") \ + X(COMMIT_RESTORE_JOB, commit_restore_job, "commit restore job") \ + X(FINISH_RESTORE_JOB, finish_restore_job, "finish restore job") \ + X(LIST_SNAPSHOTS, list_snapshots, "list snapshots") \ + X(GET_CLUSTER_STATUS, get_cluster_status, "get cluster status") \ + X(UPDATE_PACKED_FILE_INFO, update_packed_file_info, "update packed file info") + +// Enum class for Meta Service RPC types +enum class MetaServiceRPC : size_t { +#define DEFINE_ENUM(enum_name, config_suffix, display_name) enum_name, + META_SERVICE_RPC_TYPES(DEFINE_ENUM) +#undef DEFINE_ENUM + COUNT // Total number of RPC types +}; + +// Get the display name for a MetaServiceRPC enum value +std::string_view meta_service_rpc_display_name(MetaServiceRPC rpc); + +// Rate limiter with associated metrics for a single RPC method +struct RpcRateLimiter { + std::unique_ptr limiter; + std::unique_ptr latency_recorder; + + RpcRateLimiter(int qps, std::string_view op_name); + + // Reset the rate limiter with new QPS + void reset(int qps); +}; + +// Host-level rate limiters for MS RPCs to prevent burst traffic +// Each RPC method has its own rate limiter and bvar metrics +// Uses enum class MetaServiceRPC as key for O(1) lookup +class HostLevelMSRpcRateLimiters { +public: + // Constructor initializes rate limiters for all RPC types from config + HostLevelMSRpcRateLimiters(); + + // Constructor for testing: initializes all rate limiters with uniform QPS + // This allows unit tests to be independent of config values + explicit HostLevelMSRpcRateLimiters(int uniform_qps); + + ~HostLevelMSRpcRateLimiters() = default; + + // Rate limit the specified RPC method, returns actual sleep time in nanoseconds + // Thread-safe: each limiter handles its own synchronization + int64_t limit(MetaServiceRPC rpc); + + // Reset a specific rate limiter with new QPS + void reset(MetaServiceRPC rpc, int qps); + + // Reset all rate limiters (re-reads QPS from config) + void reset_all(); + +private: + void init_from_config(); + void init_with_uniform_qps(int qps); + + // Use atomic_shared_ptr for thread-safe access during concurrent limit() and reset() calls + std::array, static_cast(MetaServiceRPC::COUNT)> + _limiters; +}; + +} // namespace doris::cloud diff --git a/be/src/cloud/cloud_rowset_builder.cpp b/be/src/cloud/cloud_rowset_builder.cpp index 9534c14a12fdb6..abe36ed5790d71 100644 --- a/be/src/cloud/cloud_rowset_builder.cpp +++ b/be/src/cloud/cloud_rowset_builder.cpp @@ -88,7 +88,8 @@ Status CloudRowsetBuilder::init() { _calc_delete_bitmap_token = _engine.calc_delete_bitmap_executor()->create_token(); if (!_skip_writing_rowset_metadata) { - RETURN_IF_ERROR(_engine.meta_mgr().prepare_rowset(*_rowset_writer->rowset_meta(), "")); + RETURN_IF_ERROR(_engine.meta_mgr().prepare_rowset(*_rowset_writer->rowset_meta(), "", + _tablet->table_id())); } _is_init = true; diff --git a/be/src/cloud/cloud_schema_change_job.cpp b/be/src/cloud/cloud_schema_change_job.cpp index f4915a0536089c..f95ed81263bef9 100644 --- a/be/src/cloud/cloud_schema_change_job.cpp +++ b/be/src/cloud/cloud_schema_change_job.cpp @@ -407,8 +407,8 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam auto rowset_writer = DORIS_TRY(_new_tablet->create_rowset_writer(context, vertical)); RowsetMetaSharedPtr existed_rs_meta; - auto st = _cloud_storage_engine.meta_mgr().prepare_rowset(*rowset_writer->rowset_meta(), - _job_id, &existed_rs_meta); + auto st = _cloud_storage_engine.meta_mgr().prepare_rowset( + *rowset_writer->rowset_meta(), _job_id, _new_tablet->table_id(), &existed_rs_meta); if (!st.ok()) { if (st.is()) { LOG(INFO) << "Rowset " << rs_reader->version() << " has already existed in tablet " @@ -443,8 +443,8 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam st.to_string()); } - st = _cloud_storage_engine.meta_mgr().commit_rowset(*rowset_writer->rowset_meta(), _job_id, - &existed_rs_meta); + st = _cloud_storage_engine.meta_mgr().commit_rowset( + *rowset_writer->rowset_meta(), _job_id, _new_tablet->table_id(), &existed_rs_meta); if (!st.ok()) { if (st.is()) { LOG(INFO) << "Rowset " << rs_reader->version() << " has already existed in tablet " @@ -673,7 +673,8 @@ Status CloudSchemaChangeJob::_process_delete_bitmap(int64_t alter_version, // step4, store delete bitmap RETURN_IF_ERROR(_cloud_storage_engine.meta_mgr().update_delete_bitmap( *_new_tablet, SCHEMA_CHANGE_DELETE_BITMAP_LOCK_ID, initiator, &delete_bitmap, - &delete_bitmap, "", storage_resource, config::delete_bitmap_store_write_version)); + &delete_bitmap, "", storage_resource, config::delete_bitmap_store_write_version, + _new_tablet->table_id())); _new_tablet->tablet_meta()->delete_bitmap() = delete_bitmap; return Status::OK(); diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index 3598108308e768..339a5a757a230b 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -1155,7 +1155,7 @@ Status CloudTablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t tx return Status::InternalError("injected update_tmp_rowset error."); }); const auto& rowset_meta = rowset->rowset_meta(); - RETURN_IF_ERROR(_engine.meta_mgr().update_tmp_rowset(*rowset_meta)); + RETURN_IF_ERROR(_engine.meta_mgr().update_tmp_rowset(*rowset_meta, table_id())); } RETURN_IF_ERROR(save_delete_bitmap_to_ms(cur_version, txn_id, delete_bitmap, lock_id, @@ -1212,7 +1212,7 @@ Status CloudTablet::save_delete_bitmap_to_ms(int64_t cur_version, int64_t txn_id RETURN_IF_ERROR(_engine.meta_mgr().update_delete_bitmap( *this, ms_lock_id, LOAD_INITIATOR_ID, new_delete_bitmap.get(), new_delete_bitmap.get(), rowset->rowset_id().to_string(), storage_resource, - config::delete_bitmap_store_write_version, txn_id, is_explicit_txn, + config::delete_bitmap_store_write_version, table_id(), txn_id, is_explicit_txn, next_visible_version)); return Status::OK(); } @@ -1365,7 +1365,7 @@ Status CloudTablet::calc_delete_bitmap_for_compaction( } auto st = _engine.meta_mgr().update_delete_bitmap( *this, -1, initiator, output_rowset_delete_bitmap.get(), delete_bitmap_v2.get(), - output_rowset->rowset_id().to_string(), storage_resource, store_version); + output_rowset->rowset_id().to_string(), storage_resource, store_version, table_id()); int64_t t6 = MonotonicMicros(); LOG(INFO) << "calc_delete_bitmap_for_compaction, tablet_id=" << tablet_id() << ", get lock cost " << (t2 - t1) << " us, sync rowsets cost " << (t3 - t2) diff --git a/be/src/cloud/cloud_throttle_state_machine.cpp b/be/src/cloud/cloud_throttle_state_machine.cpp new file mode 100644 index 00000000000000..cd174ee7b5d981 --- /dev/null +++ b/be/src/cloud/cloud_throttle_state_machine.cpp @@ -0,0 +1,272 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "cloud/cloud_throttle_state_machine.h" + +#include + +#include + +namespace doris::cloud { + +// Display names for LoadRelatedRpc types +static constexpr std::string_view LOAD_RELATED_RPC_NAMES[] = { + "prepare_rowset", "commit_rowset", "update_tmp_rowset", "update_packed_file_info", + "update_delete_bitmap"}; + +std::string_view load_related_rpc_name(LoadRelatedRpc rpc) { + size_t idx = static_cast(rpc); + if (idx < static_cast(LoadRelatedRpc::COUNT)) { + return LOAD_RELATED_RPC_NAMES[idx]; + } + return "unknown"; +} + +// ============== RpcThrottleStateMachine ============== + +RpcThrottleStateMachine::RpcThrottleStateMachine(RpcThrottleParams params) : _params(params) { + LOG(INFO) << "[ms-throttle] state machine initialized: top_k=" << params.top_k + << ", ratio=" << params.ratio << ", floor_qps=" << params.floor_qps; +} + +void RpcThrottleStateMachine::update_params(RpcThrottleParams params) { + std::lock_guard lock(_mtx); + _params = params; + LOG(INFO) << "[ms-throttle] state machine params updated: top_k=" << params.top_k + << ", ratio=" << params.ratio << ", floor_qps=" << params.floor_qps; +} + +std::vector RpcThrottleStateMachine::on_upgrade( + const std::vector& qps_snapshot) { + std::lock_guard lock(_mtx); + + UpgradeRecord record; + std::vector actions; + + double ratio = _params.ratio; + double floor_qps = _params.floor_qps; + + // Caller is responsible for providing top-k snapshot per RPC type. + // State machine simply applies throttling to every entry in the snapshot. + for (const auto& snapshot : qps_snapshot) { + auto key = std::make_pair(snapshot.rpc_type, snapshot.table_id); + + double old_limit = 0.0; + auto limit_it = _current_limits.find(key); + if (limit_it != _current_limits.end()) { + old_limit = limit_it->second; + } + + double new_limit; + if (old_limit > 0) { + // Already has a limit, reduce it further + new_limit = old_limit * ratio; + } else { + // No limit yet, set based on current QPS + new_limit = snapshot.current_qps * ratio; + } + + // Apply floor + new_limit = std::max(new_limit, floor_qps); + + // Only apply if it's actually limiting + if (new_limit < snapshot.current_qps || old_limit > 0) { + RpcThrottleAction action { + .type = RpcThrottleAction::Type::SET_LIMIT, + .rpc_type = snapshot.rpc_type, + .table_id = snapshot.table_id, + .qps_limit = new_limit, + }; + actions.push_back(action); + record.changes[key] = {old_limit, new_limit}; + _current_limits[key] = new_limit; + + LOG(INFO) << "[ms-throttle] upgrade: rpc=" << load_related_rpc_name(snapshot.rpc_type) + << ", table_id=" << snapshot.table_id + << ", current_qps=" << snapshot.current_qps << ", old_limit=" << old_limit + << ", new_limit=" << new_limit; + } + } + + if (!record.changes.empty()) { + _upgrade_history.push_back(std::move(record)); + } + + LOG(INFO) << "[ms-throttle] on_upgrade done: actions=" << actions.size() + << ", upgrade_level=" << _upgrade_history.size() + << ", snapshot_size=" << qps_snapshot.size(); + + return actions; +} + +std::vector RpcThrottleStateMachine::on_downgrade() { + std::lock_guard lock(_mtx); + + std::vector actions; + + if (_upgrade_history.empty()) { + LOG(INFO) << "[ms-throttle] on_downgrade skipped: no upgrade history"; + return actions; + } + + // Undo the most recent upgrade + const auto& record = _upgrade_history.back(); + + for (const auto& [key, limits] : record.changes) { + const auto& [rpc_type, table_id] = key; + double old_limit = limits.first; + + if (old_limit > 0) { + // Restore the previous limit + RpcThrottleAction action { + .type = RpcThrottleAction::Type::SET_LIMIT, + .rpc_type = rpc_type, + .table_id = table_id, + .qps_limit = old_limit, + }; + + actions.push_back(action); + _current_limits[key] = old_limit; + + LOG(INFO) << "[ms-throttle] downgrade: rpc=" << load_related_rpc_name(rpc_type) + << ", table_id=" << table_id << ", restored_limit=" << old_limit; + } else { + // No previous limit, remove it entirely + RpcThrottleAction action { + .type = RpcThrottleAction::Type::REMOVE_LIMIT, + .rpc_type = rpc_type, + .table_id = table_id, + }; + + actions.push_back(action); + _current_limits.erase(key); + + LOG(INFO) << "[ms-throttle] downgrade: rpc=" << load_related_rpc_name(rpc_type) + << ", table_id=" << table_id << ", removed limit"; + } + } + + _upgrade_history.pop_back(); + + LOG(INFO) << "[ms-throttle] on_downgrade done: actions=" << actions.size() + << ", upgrade_level=" << _upgrade_history.size(); + + return actions; +} + +size_t RpcThrottleStateMachine::upgrade_level() const { + std::lock_guard lock(_mtx); + return _upgrade_history.size(); +} + +double RpcThrottleStateMachine::get_current_limit(LoadRelatedRpc rpc_type, int64_t table_id) const { + std::lock_guard lock(_mtx); + auto it = _current_limits.find({rpc_type, table_id}); + if (it != _current_limits.end()) { + return it->second; + } + return 0.0; +} + +RpcThrottleParams RpcThrottleStateMachine::get_params() const { + std::lock_guard lock(_mtx); + return _params; +} + +// ============== RpcThrottleCoordinator ============== + +RpcThrottleCoordinator::RpcThrottleCoordinator(ThrottleCoordinatorParams params) : _params(params) { + LOG(INFO) << "[ms-throttle] coordinator initialized: upgrade_cooldown_ticks=" + << params.upgrade_cooldown_ticks + << ", downgrade_after_ticks=" << params.downgrade_after_ticks; +} + +void RpcThrottleCoordinator::update_params(ThrottleCoordinatorParams params) { + std::lock_guard lock(_mtx); + _params = params; + LOG(INFO) << "[ms-throttle] coordinator params updated: upgrade_cooldown_ticks=" + << params.upgrade_cooldown_ticks + << ", downgrade_after_ticks=" << params.downgrade_after_ticks; +} + +bool RpcThrottleCoordinator::report_ms_busy() { + std::lock_guard lock(_mtx); + + // Reset tick counter since last MS_BUSY + _ticks_since_last_ms_busy = 0; + + // Check if cooldown has passed + if (_ticks_since_last_upgrade == -1 || + _ticks_since_last_upgrade >= _params.upgrade_cooldown_ticks) { + // Reset upgrade counter + auto actual_ticks = _ticks_since_last_upgrade; + _ticks_since_last_upgrade = 0; + _has_pending_upgrades = true; + + LOG(INFO) << "[ms-throttle] upgrade triggered: ticks_since_last_upgrade=" << actual_ticks + << ", cooldown=" << _params.upgrade_cooldown_ticks; + return true; // Should trigger upgrade + } + return false; // Cooling down +} + +bool RpcThrottleCoordinator::tick(int ticks) { + std::lock_guard lock(_mtx); + + // Increment tick counters + if (_ticks_since_last_ms_busy >= 0) { + _ticks_since_last_ms_busy += ticks; + } + if (_ticks_since_last_upgrade >= 0) { + _ticks_since_last_upgrade += ticks; + } + + // Check if downgrade should be triggered + if (_has_pending_upgrades && _ticks_since_last_ms_busy >= _params.downgrade_after_ticks) { + // Reset for next downgrade cycle + auto actual_ticks = _ticks_since_last_ms_busy; + _ticks_since_last_ms_busy = 0; + + LOG(INFO) << "[ms-throttle] downgrade triggered: ticks_since_last_ms_busy=" << actual_ticks + << ", threshold=" << _params.downgrade_after_ticks; + return true; // Should trigger downgrade + } + + return false; +} + +void RpcThrottleCoordinator::set_has_pending_upgrades(bool has) { + std::lock_guard lock(_mtx); + _has_pending_upgrades = has; +} + +int RpcThrottleCoordinator::ticks_since_last_ms_busy() const { + std::lock_guard lock(_mtx); + return _ticks_since_last_ms_busy; +} + +int RpcThrottleCoordinator::ticks_since_last_upgrade() const { + std::lock_guard lock(_mtx); + return _ticks_since_last_upgrade; +} + +ThrottleCoordinatorParams RpcThrottleCoordinator::get_params() const { + std::lock_guard lock(_mtx); + return _params; +} + +} // namespace doris::cloud diff --git a/be/src/cloud/cloud_throttle_state_machine.h b/be/src/cloud/cloud_throttle_state_machine.h new file mode 100644 index 00000000000000..9a5f53c079be1f --- /dev/null +++ b/be/src/cloud/cloud_throttle_state_machine.h @@ -0,0 +1,176 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace doris::cloud { + +// ============== Common Types ============== + +// Load-related RPC types that need table-level QPS statistics +enum class LoadRelatedRpc : size_t { + PREPARE_ROWSET, + COMMIT_ROWSET, + UPDATE_TMP_ROWSET, + UPDATE_PACKED_FILE_INFO, + UPDATE_DELETE_BITMAP, + COUNT +}; + +// Get the name string for a LoadRelatedRpc type +std::string_view load_related_rpc_name(LoadRelatedRpc rpc); + +// ============== Data Structures ============== + +// QPS snapshot: the current QPS of a table on a specific RPC type +struct RpcQpsSnapshot { + LoadRelatedRpc rpc_type; + int64_t table_id; + double current_qps; +}; + +// Throttle action: describes what action should be taken +struct RpcThrottleAction { + enum class Type { SET_LIMIT, REMOVE_LIMIT }; + + Type type; + LoadRelatedRpc rpc_type; + int64_t table_id; + double qps_limit {0}; // only meaningful for SET_LIMIT +}; + +// ============== ThrottleStateMachine ============== + +// Parameters for throttle state machine +struct RpcThrottleParams { + int top_k = 3; // Number of top tables to throttle on each upgrade + double ratio = 0.5; // Decay ratio for throttle upgrade + double floor_qps = 1.0; // Floor value for table-level QPS limit + + bool operator==(const RpcThrottleParams& other) const { + return top_k == other.top_k && ratio == other.ratio && floor_qps == other.floor_qps; + } +}; + +// Pure state machine for throttle upgrade/downgrade decisions +// - No time awareness: caller drives events via on_upgrade/on_downgrade +// - No config dependency: all parameters passed via constructor/update_params +// - No side effects: only returns action descriptions, doesn't touch throttler +// - Deterministically testable: same event sequence -> same output +class RpcThrottleStateMachine { +public: + explicit RpcThrottleStateMachine(RpcThrottleParams params); + + // Runtime update parameters, takes effect on next on_upgrade + // Note: existing upgrade history is NOT recalculated + void update_params(RpcThrottleParams params); + + // Process a throttle upgrade event + // qps_snapshot: current QPS snapshot for each (rpc, table), provided by caller + // Returns: list of actions to execute + std::vector on_upgrade(const std::vector& qps_snapshot); + + // Process a throttle downgrade event (undo the most recent upgrade) + // Returns: list of actions to execute + std::vector on_downgrade(); + + // Query current state + size_t upgrade_level() const; // Current upgrade level + double get_current_limit(LoadRelatedRpc rpc_type, int64_t table_id) const; // 0 = no limit + RpcThrottleParams get_params() const; + +private: + mutable std::mutex _mtx; + + RpcThrottleParams _params; + + // Upgrade history for downgrade rollback + // changes: (rpc_type, table_id) -> (old_limit, new_limit) + struct UpgradeRecord { + std::map, std::pair> changes; + }; + std::vector _upgrade_history; + + // Current active limits for all (rpc, table) + std::map, double> _current_limits; +}; + +// ============== ThrottleCoordinator ============== + +// Coordinator parameters +struct ThrottleCoordinatorParams { + // Minimum ticks between upgrades + int upgrade_cooldown_ticks = 10; + // Ticks after last MS_BUSY to trigger downgrade + int downgrade_after_ticks = 60; + + bool operator==(const ThrottleCoordinatorParams& other) const { + return upgrade_cooldown_ticks == other.upgrade_cooldown_ticks && + downgrade_after_ticks == other.downgrade_after_ticks; + } +}; + +// Pure timing control for upgrade/downgrade triggers +// - No time awareness: based on tick count, driven by caller +// - No config dependency: all parameters passed via constructor/update_params +// +// Tick semantics: +// - 1 tick = 1 millisecond (fixed unit) +// - upgrade_cooldown_ticks and downgrade_after_ticks are in milliseconds +// - The tick thread advances time by 1000 ticks (1 second) each iteration +class RpcThrottleCoordinator { +public: + explicit RpcThrottleCoordinator(ThrottleCoordinatorParams params); + + // Runtime update parameters, takes effect on subsequent report_ms_busy/tick calls + // Note: existing tick counts are NOT reset + void update_params(ThrottleCoordinatorParams params); + + // Report a MS_BUSY event + // Returns true if upgrade should be triggered + bool report_ms_busy(); + + // Advance by specified number of ticks (caller decides actual time between ticks) + // Returns true if downgrade should be triggered + bool tick(int ticks = 1); + + // Tell coordinator whether there are pending upgrades that can be downgraded + // Called by the state machine consumer after upgrade/downgrade + void set_has_pending_upgrades(bool has); + + // Query state + int ticks_since_last_ms_busy() const; + int ticks_since_last_upgrade() const; + ThrottleCoordinatorParams get_params() const; + +private: + mutable std::mutex _mtx; + + ThrottleCoordinatorParams _params; + int _ticks_since_last_ms_busy = -1; // -1 means never received + int _ticks_since_last_upgrade = -1; // -1 means never upgraded + bool _has_pending_upgrades = false; // Whether there are upgrade records to downgrade +}; + +} // namespace doris::cloud diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index 81559a9b1295e1..d2de00f39d19ff 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -172,4 +172,48 @@ DEFINE_mInt64(cache_read_from_peer_expired_seconds, "-1"); DEFINE_mBool(enable_file_cache_write_base_compaction_index_only, "false"); DEFINE_mBool(enable_file_cache_write_cumu_compaction_index_only, "false"); +// MS RPC rate limiting config +DEFINE_mBool(enable_ms_rpc_host_level_rate_limit, "false"); + +// Per-RPC QPS limit configs (per CPU core) +// QPS limit = config_value * num_cores +// Set to 0 to disable rate limiting for a specific RPC +// Set to -1 to use ms_rpc_qps_default config value +DEFINE_mInt32(ms_rpc_qps_default, "100"); +DEFINE_mInt32(ms_rpc_qps_get_tablet_meta, "-1"); +DEFINE_mInt32(ms_rpc_qps_get_rowset, "-1"); +DEFINE_mInt32(ms_rpc_qps_prepare_rowset, "-1"); +DEFINE_mInt32(ms_rpc_qps_commit_rowset, "-1"); +DEFINE_mInt32(ms_rpc_qps_update_tmp_rowset, "-1"); +DEFINE_mInt32(ms_rpc_qps_commit_txn, "-1"); +DEFINE_mInt32(ms_rpc_qps_abort_txn, "-1"); +DEFINE_mInt32(ms_rpc_qps_precommit_txn, "-1"); +DEFINE_mInt32(ms_rpc_qps_get_obj_store_info, "-1"); +DEFINE_mInt32(ms_rpc_qps_start_tablet_job, "-1"); +DEFINE_mInt32(ms_rpc_qps_finish_tablet_job, "-1"); +DEFINE_mInt32(ms_rpc_qps_get_delete_bitmap, "-1"); +DEFINE_mInt32(ms_rpc_qps_update_delete_bitmap, "-1"); +DEFINE_mInt32(ms_rpc_qps_get_delete_bitmap_update_lock, "-1"); +DEFINE_mInt32(ms_rpc_qps_remove_delete_bitmap_update_lock, "-1"); +DEFINE_mInt32(ms_rpc_qps_get_instance, "-1"); +DEFINE_mInt32(ms_rpc_qps_prepare_restore_job, "-1"); +DEFINE_mInt32(ms_rpc_qps_commit_restore_job, "-1"); +DEFINE_mInt32(ms_rpc_qps_finish_restore_job, "-1"); +DEFINE_mInt32(ms_rpc_qps_list_snapshots, "-1"); +DEFINE_mInt32(ms_rpc_qps_get_cluster_status, "-1"); +DEFINE_mInt32(ms_rpc_qps_update_packed_file_info, "-1"); + +// Table-level backpressure handling config +DEFINE_mBool(enable_ms_backpressure_handling, "false"); +DEFINE_Int32(ms_rpc_table_qps_window_sec, "3"); + +// Throttle upgrade config +DEFINE_mInt32(ms_backpressure_upgrade_interval_ms, "3000"); +DEFINE_mInt32(ms_backpressure_upgrade_top_k, "1"); +DEFINE_mDouble(ms_backpressure_throttle_ratio, "0.75"); +DEFINE_mDouble(ms_rpc_table_qps_limit_floor, "1.0"); + +// Throttle downgrade config +DEFINE_mInt32(ms_backpressure_downgrade_interval_ms, "3000"); + } // namespace doris::config diff --git a/be/src/cloud/config.h b/be/src/cloud/config.h index 7db705e342a7b1..40b41715825965 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -215,4 +215,67 @@ DECLARE_mBool(enable_file_cache_write_base_compaction_index_only); // Cumulative compaction output: only write index files to file cache, not data files DECLARE_mBool(enable_file_cache_write_cumu_compaction_index_only); +// MS RPC rate limiting config +// Enable host-level rate limiting for MS RPCs to prevent burst traffic +DECLARE_mBool(enable_ms_rpc_host_level_rate_limit); + +// Per-RPC QPS limit configs (per CPU core) +// QPS limit = config_value * num_cores +// Set to 0 to disable rate limiting for a specific RPC +// Set to -1 to use ms_rpc_qps_default config value +DECLARE_mInt32(ms_rpc_qps_default); +DECLARE_mInt32(ms_rpc_qps_get_tablet_meta); +DECLARE_mInt32(ms_rpc_qps_get_rowset); +DECLARE_mInt32(ms_rpc_qps_prepare_rowset); +DECLARE_mInt32(ms_rpc_qps_commit_rowset); +DECLARE_mInt32(ms_rpc_qps_update_tmp_rowset); +DECLARE_mInt32(ms_rpc_qps_commit_txn); +DECLARE_mInt32(ms_rpc_qps_abort_txn); +DECLARE_mInt32(ms_rpc_qps_precommit_txn); +DECLARE_mInt32(ms_rpc_qps_get_obj_store_info); +DECLARE_mInt32(ms_rpc_qps_start_tablet_job); +DECLARE_mInt32(ms_rpc_qps_finish_tablet_job); +DECLARE_mInt32(ms_rpc_qps_get_delete_bitmap); +DECLARE_mInt32(ms_rpc_qps_update_delete_bitmap); +DECLARE_mInt32(ms_rpc_qps_get_delete_bitmap_update_lock); +DECLARE_mInt32(ms_rpc_qps_remove_delete_bitmap_update_lock); +DECLARE_mInt32(ms_rpc_qps_get_instance); +DECLARE_mInt32(ms_rpc_qps_prepare_restore_job); +DECLARE_mInt32(ms_rpc_qps_commit_restore_job); +DECLARE_mInt32(ms_rpc_qps_finish_restore_job); +DECLARE_mInt32(ms_rpc_qps_list_snapshots); +DECLARE_mInt32(ms_rpc_qps_get_cluster_status); +DECLARE_mInt32(ms_rpc_qps_update_packed_file_info); + +// ============== Table-level backpressure handling config ============== + +// Enable MS backpressure response handling (table-level adaptive throttling) +DECLARE_mBool(enable_ms_backpressure_handling); + +// Time window (seconds) for computing per-table QPS via bvar::PerSecond. +// Larger window smooths out short-term spikes; smaller window reacts faster. +// Immutable: takes effect only at counter creation time. +DECLARE_Int32(ms_rpc_table_qps_window_sec); + +// ------------ Throttle upgrade config ------------ + +// Minimum interval between throttle upgrades after receiving MS_BUSY (milliseconds) +DECLARE_mInt32(ms_backpressure_upgrade_interval_ms); + +// Number of top-k tables to throttle on each upgrade +DECLARE_mInt32(ms_backpressure_upgrade_top_k); + +// QPS decay ratio when upgrading throttle +// New limit = current QPS * ratio (first time) or current limit * ratio (already limited) +DECLARE_mDouble(ms_backpressure_throttle_ratio); + +// Floor value for table-level QPS limit +// Throttle upgrade will not reduce QPS limit below this value +DECLARE_mDouble(ms_rpc_table_qps_limit_floor); + +// ------------ Throttle downgrade config ------------ + +// Time without MS_BUSY before triggering throttle downgrade (milliseconds) +DECLARE_mInt32(ms_backpressure_downgrade_interval_ms); + } // namespace doris::config diff --git a/be/src/information_schema/schema_backend_ms_rpc_table_throttlers_scanner.cpp b/be/src/information_schema/schema_backend_ms_rpc_table_throttlers_scanner.cpp new file mode 100644 index 00000000000000..a19eea752c54e5 --- /dev/null +++ b/be/src/information_schema/schema_backend_ms_rpc_table_throttlers_scanner.cpp @@ -0,0 +1,113 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "information_schema/schema_backend_ms_rpc_table_throttlers_scanner.h" + +#include + +#include + +#include "cloud/cloud_ms_rpc_rate_limit_services.h" +#include "cloud/cloud_throttle_state_machine.h" +#include "core/block/block.h" +#include "core/data_type/define_primitive_type.h" +#include "core/string_ref.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_state.h" + +namespace doris { + +std::vector SchemaBackendMsRpcTableThrottlersScanner::_s_tbls_columns = { + // name, type, size, is_null + {"BE_ID", TYPE_BIGINT, sizeof(int64_t), false}, + {"TABLE_ID", TYPE_BIGINT, sizeof(int64_t), false}, + {"RPC_TYPE", TYPE_VARCHAR, sizeof(StringRef), false}, + {"QPS_LIMIT", TYPE_DOUBLE, sizeof(double), false}, + {"CURRENT_QPS", TYPE_DOUBLE, sizeof(double), false}, +}; + +SchemaBackendMsRpcTableThrottlersScanner::SchemaBackendMsRpcTableThrottlersScanner() + : SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_BACKEND_MS_RPC_TABLE_THROTTLERS), + _backend_id(ExecEnv::GetInstance()->cluster_info()->backend_id) {} + +SchemaBackendMsRpcTableThrottlersScanner::~SchemaBackendMsRpcTableThrottlersScanner() = default; + +Status SchemaBackendMsRpcTableThrottlersScanner::start(RuntimeState* state) { + auto* services = ExecEnv::GetInstance()->ms_rpc_rate_limit_services(); + if (services != nullptr) { + auto* throttler = services->table_rpc_throttler(); + _entries = throttler->get_all_throttled_entries(); + } + return Status::OK(); +} + +Status SchemaBackendMsRpcTableThrottlersScanner::get_next_block_internal(Block* block, bool* eos) { + if (!_is_init) { + return Status::InternalError("Used before initialized."); + } + + if (nullptr == block || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); + } + + *eos = true; + if (_entries.empty()) { + return Status::OK(); + } + + auto* services = ExecEnv::GetInstance()->ms_rpc_rate_limit_services(); + auto* registry = services != nullptr ? services->table_rpc_qps_registry() : nullptr; + + size_t row_num = _entries.size(); + for (size_t col_idx = 0; col_idx < _s_tbls_columns.size(); ++col_idx) { + std::vector str_refs(row_num); + std::vector double_vals(row_num); + std::vector datas(row_num); + std::vector rpc_names(row_num); + + for (size_t row_idx = 0; row_idx < row_num; ++row_idx) { + const auto& entry = _entries[row_idx]; + switch (col_idx) { + case 0: // BE_ID + datas[row_idx] = &_backend_id; + break; + case 1: // TABLE_ID + datas[row_idx] = const_cast(&entry.table_id); + break; + case 2: { // RPC_TYPE + rpc_names[row_idx] = std::string(cloud::load_related_rpc_name(entry.rpc_type)); + str_refs[row_idx] = StringRef(rpc_names[row_idx].data(), rpc_names[row_idx].size()); + datas[row_idx] = &str_refs[row_idx]; + break; + } + case 3: // QPS_LIMIT + double_vals[row_idx] = entry.qps_limit; + datas[row_idx] = &double_vals[row_idx]; + break; + case 4: // CURRENT_QPS + double_vals[row_idx] = + registry ? registry->get_qps(entry.rpc_type, entry.table_id) : 0; + datas[row_idx] = &double_vals[row_idx]; + break; + } + } + RETURN_IF_ERROR(fill_dest_column_for_range(block, col_idx, datas)); + } + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/information_schema/schema_backend_ms_rpc_table_throttlers_scanner.h b/be/src/information_schema/schema_backend_ms_rpc_table_throttlers_scanner.h new file mode 100644 index 00000000000000..c1acfb6ce4bfce --- /dev/null +++ b/be/src/information_schema/schema_backend_ms_rpc_table_throttlers_scanner.h @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "cloud/cloud_ms_backpressure_handler.h" +#include "common/status.h" +#include "information_schema/schema_scanner.h" + +namespace doris { +class RuntimeState; +class Block; + +class SchemaBackendMsRpcTableThrottlersScanner : public SchemaScanner { + ENABLE_FACTORY_CREATOR(SchemaBackendMsRpcTableThrottlersScanner); + +public: + SchemaBackendMsRpcTableThrottlersScanner(); + ~SchemaBackendMsRpcTableThrottlersScanner() override; + + Status start(RuntimeState* state) override; + Status get_next_block_internal(Block* block, bool* eos) override; + + static std::vector _s_tbls_columns; + +private: + int64_t _backend_id; + std::vector _entries; +}; +} // namespace doris diff --git a/be/src/information_schema/schema_scanner.cpp b/be/src/information_schema/schema_scanner.cpp index 981956330ce7f5..c056617e3972e0 100644 --- a/be/src/information_schema/schema_scanner.cpp +++ b/be/src/information_schema/schema_scanner.cpp @@ -48,6 +48,7 @@ #include "information_schema/schema_backend_active_tasks.h" #include "information_schema/schema_backend_configuration_scanner.h" #include "information_schema/schema_backend_kerberos_ticket_cache.h" +#include "information_schema/schema_backend_ms_rpc_table_throttlers_scanner.h" #include "information_schema/schema_catalog_meta_cache_stats_scanner.h" #include "information_schema/schema_charsets_scanner.h" #include "information_schema/schema_cluster_snapshot_properties_scanner.h" @@ -277,6 +278,8 @@ std::unique_ptr SchemaScanner::create(TSchemaTableType::type type return SchemaTableStreamConsumptionScanner::create_unique(); case TSchemaTableType::SCH_BE_COMPACTION_TASKS: return SchemaCompactionTasksScanner::create_unique(); + case TSchemaTableType::SCH_BACKEND_MS_RPC_TABLE_THROTTLERS: + return SchemaBackendMsRpcTableThrottlersScanner::create_unique(); default: return SchemaDummyScanner::create_unique(); break; diff --git a/be/src/io/cache/cached_remote_file_reader.cpp b/be/src/io/cache/cached_remote_file_reader.cpp index ee5f4ceadbd79d..453b496929ad84 100644 --- a/be/src/io/cache/cached_remote_file_reader.cpp +++ b/be/src/io/cache/cached_remote_file_reader.cpp @@ -38,9 +38,8 @@ #include "cloud/config.h" #include "common/compiler_util.h" // IWYU pragma: keep #include "common/config.h" -#include "common/metrics/doris_metrics.h" -#include "cpp/s3_rate_limiter.h" #include "cpp/sync_point.h" +#include "cpp/token_bucket_rate_limiter.h" #include "io/cache/block_file_cache.h" #include "io/cache/block_file_cache_factory.h" #include "io/cache/block_file_cache_profile.h" diff --git a/be/src/io/fs/packed_file_manager.cpp b/be/src/io/fs/packed_file_manager.cpp index eb46bcc171bf34..29675ac9319207 100644 --- a/be/src/io/fs/packed_file_manager.cpp +++ b/be/src/io/fs/packed_file_manager.cpp @@ -894,7 +894,8 @@ Status PackedFileManager::finalize_packed_file_upload(const std::string& packed_ } Status PackedFileManager::update_meta_service(const std::string& packed_file_path, - const cloud::PackedFileInfoPB& packed_file_info) { + const cloud::PackedFileInfoPB& packed_file_info, + int64_t table_id) { #ifdef BE_TEST TEST_SYNC_POINT_RETURN_WITH_VALUE("PackedFileManager::update_meta_service", Status::OK(), packed_file_path, &packed_file_info); @@ -910,7 +911,7 @@ Status PackedFileManager::update_meta_service(const std::string& packed_file_pat auto& storage_engine = ExecEnv::GetInstance()->storage_engine(); auto& cloud_meta_mgr = storage_engine.to_cloud().meta_mgr(); - return cloud_meta_mgr.update_packed_file_info(packed_file_path, packed_file_info); + return cloud_meta_mgr.update_packed_file_info(packed_file_path, packed_file_info, table_id); } void PackedFileManager::cleanup_expired_data() { diff --git a/be/src/io/fs/packed_file_manager.h b/be/src/io/fs/packed_file_manager.h index 7756a3fd85acd6..13b5a39bfc02c8 100644 --- a/be/src/io/fs/packed_file_manager.h +++ b/be/src/io/fs/packed_file_manager.h @@ -109,8 +109,10 @@ class PackedFileManager { Status finalize_packed_file_upload(const std::string& packed_file_path, FileWriter* writer); // Update meta service with packed file information + // table_id is used for rate limiting; -1 means no specific table (cross-table operation) Status update_meta_service(const std::string& packed_file_path, - const cloud::PackedFileInfoPB& packed_file_info); + const cloud::PackedFileInfoPB& packed_file_info, + int64_t table_id = -1); // Process uploading files void process_uploading_packed_files(); diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index d915fbc5721752..8ce83b2e82b90b 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -56,12 +56,15 @@ class TaskScheduler; struct RuntimeFilterTimerQueue; class WorkloadGroupMgr; struct WriteCooldownMetaExecutors; -class S3RateLimiterHolder; +class TokenBucketRateLimiterHolder; +using S3RateLimiterHolder = TokenBucketRateLimiterHolder; +class MSRpcRateLimitServices; namespace io { class FileCacheFactory; class HdfsMgr; class PackedFileManager; } // namespace io + namespace segment_v2 { class InvertedIndexSearcherCache; class InvertedIndexQueryCache; @@ -308,6 +311,9 @@ class ExecEnv { io::PackedFileManager* packed_file_manager() { return _packed_file_manager; } IndexPolicyMgr* index_policy_mgr() { return _index_policy_mgr; } S3RateLimiterHolder* warmup_download_rate_limiter() { return _warmup_download_rate_limiter; } + MSRpcRateLimitServices* ms_rpc_rate_limit_services() { + return _ms_rpc_rate_limit_services.get(); + } #ifdef BE_TEST void set_tmp_file_dir(std::unique_ptr tmp_file_dirs) { @@ -574,6 +580,7 @@ class ExecEnv { io::HdfsMgr* _hdfs_mgr = nullptr; io::PackedFileManager* _packed_file_manager = nullptr; S3RateLimiterHolder* _warmup_download_rate_limiter = nullptr; + std::unique_ptr _ms_rpc_rate_limit_services; }; template <> diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 7fd6f3344364f6..94658e9e80e9a1 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -32,6 +32,9 @@ #include #include "cloud/cloud_cluster_info.h" +#include "cloud/cloud_meta_mgr.h" +#include "cloud/cloud_ms_rpc_rate_limit_services.h" +#include "cloud/cloud_ms_rpc_rate_limiters.h" #include "cloud/cloud_storage_engine.h" #include "cloud/cloud_stream_load_executor.h" #include "cloud/cloud_tablet_hotspot.h" @@ -44,7 +47,7 @@ #include "common/metrics/doris_metrics.h" #include "common/multi_version.h" #include "common/status.h" -#include "cpp/s3_rate_limiter.h" +#include "cpp/token_bucket_rate_limiter.h" #include "exec/exchange/vdata_stream_mgr.h" #include "exec/pipeline/task_queue.h" #include "exec/pipeline/task_scheduler.h" @@ -430,6 +433,17 @@ Status ExecEnv::_init(const std::vector& store_paths, // Start cluster info background worker for compaction read-write separation static_cast(_cluster_info)->start_bg_worker(); + + // Initialize MS RPC rate limiters and table-level backpressure handling. + _ms_rpc_rate_limit_services = std::make_unique(); + static_cast(_storage_engine.get()) + ->meta_mgr() + .set_host_level_ms_rpc_rate_limiters( + _ms_rpc_rate_limit_services->host_level_ms_rpc_rate_limiters()); + static_cast(_storage_engine.get()) + ->meta_mgr() + .set_ms_backpressure_handler( + _ms_rpc_rate_limit_services->ms_backpressure_handler()); } _index_policy_mgr = new IndexPolicyMgr(); @@ -988,6 +1002,34 @@ void ExecEnv::destroy() { } // namespace doris namespace doris::config { +namespace { + +void refresh_ms_rpc_rate_limiters() { + auto* services = ExecEnv::GetInstance()->ms_rpc_rate_limit_services(); + if (services != nullptr) { + services->reset_host_level_rate_limiters(); + } +} + +void refresh_ms_backpressure_throttle_params() { + auto* services = ExecEnv::GetInstance()->ms_rpc_rate_limit_services(); + if (services != nullptr) { + services->update_backpressure_throttle_params(ms_backpressure_upgrade_top_k, + ms_backpressure_throttle_ratio, + ms_rpc_table_qps_limit_floor); + } +} + +void refresh_ms_backpressure_coordinator_params() { + auto* services = ExecEnv::GetInstance()->ms_rpc_rate_limit_services(); + if (services != nullptr) { + services->update_backpressure_coordinator_params(ms_backpressure_upgrade_interval_ms, + ms_backpressure_downgrade_interval_ms); + } +} + +} // namespace + // Callback to update warmup download rate limiter when config changes is registered DEFINE_ON_UPDATE(file_cache_warmup_download_rate_limit_bytes_per_second, [](int64_t old_val, int64_t new_val) { @@ -1005,4 +1047,45 @@ DEFINE_ON_UPDATE(file_cache_warmup_download_rate_limit_bytes_per_second, } } }); + +DEFINE_ON_UPDATE(ms_rpc_qps_default, [](int32_t old_val, int32_t new_val) { + if (old_val != new_val) { + refresh_ms_rpc_rate_limiters(); + } +}); + +#define DEFINE_MS_RPC_QPS_ON_UPDATE(enum_name, config_suffix, display_name) \ + DEFINE_ON_UPDATE(ms_rpc_qps_##config_suffix, [](int32_t old_val, int32_t new_val) { \ + if (old_val != new_val) { \ + refresh_ms_rpc_rate_limiters(); \ + } \ + }); +META_SERVICE_RPC_TYPES(DEFINE_MS_RPC_QPS_ON_UPDATE) +#undef DEFINE_MS_RPC_QPS_ON_UPDATE + +DEFINE_ON_UPDATE(ms_backpressure_upgrade_interval_ms, [](int32_t old_val, int32_t new_val) { + if (old_val != new_val) { + refresh_ms_backpressure_coordinator_params(); + } +}); +DEFINE_ON_UPDATE(ms_backpressure_downgrade_interval_ms, [](int32_t old_val, int32_t new_val) { + if (old_val != new_val) { + refresh_ms_backpressure_coordinator_params(); + } +}); +DEFINE_ON_UPDATE(ms_backpressure_upgrade_top_k, [](int32_t old_val, int32_t new_val) { + if (old_val != new_val) { + refresh_ms_backpressure_throttle_params(); + } +}); +DEFINE_ON_UPDATE(ms_backpressure_throttle_ratio, [](double old_val, double new_val) { + if (old_val != new_val) { + refresh_ms_backpressure_throttle_params(); + } +}); +DEFINE_ON_UPDATE(ms_rpc_table_qps_limit_floor, [](double old_val, double new_val) { + if (old_val != new_val) { + refresh_ms_backpressure_throttle_params(); + } +}); } // namespace doris::config diff --git a/be/src/storage/compaction/compaction.cpp b/be/src/storage/compaction/compaction.cpp index 886f1206f33914..f47513563aa081 100644 --- a/be/src/storage/compaction/compaction.cpp +++ b/be/src/storage/compaction/compaction.cpp @@ -1643,7 +1643,8 @@ Status CloudCompactionMixin::execute_compact_impl(int64_t permits) { // Currently, updates are only made in the time_series. update_compaction_level(); - RETURN_IF_ERROR(_engine.meta_mgr().commit_rowset(*_output_rowset->rowset_meta().get(), _uuid)); + RETURN_IF_ERROR(_engine.meta_mgr().commit_rowset(*_output_rowset->rowset_meta().get(), _uuid, + _tablet->table_id())); // 4. modify rowsets in memory RETURN_IF_ERROR(modify_rowsets()); @@ -1869,8 +1870,8 @@ Status CloudCompactionMixin::construct_output_rowset_writer(RowsetWriterContext& ctx.job_id = _uuid; _output_rs_writer = DORIS_TRY(_tablet->create_rowset_writer(ctx, _is_vertical)); - RETURN_IF_ERROR( - _engine.meta_mgr().prepare_rowset(*_output_rs_writer->rowset_meta().get(), _uuid)); + RETURN_IF_ERROR(_engine.meta_mgr().prepare_rowset(*_output_rs_writer->rowset_meta().get(), + _uuid, _tablet->table_id())); return Status::OK(); } diff --git a/be/src/util/s3_util.cpp b/be/src/util/s3_util.cpp index eef57234d8459f..ee8293f49bd7ee 100644 --- a/be/src/util/s3_util.cpp +++ b/be/src/util/s3_util.cpp @@ -30,7 +30,7 @@ #include #include #include -#include +#include #include diff --git a/be/src/util/s3_util.h b/be/src/util/s3_util.h index 5cedee997eeff4..23c82c65e134c7 100644 --- a/be/src/util/s3_util.h +++ b/be/src/util/s3_util.h @@ -35,7 +35,7 @@ #include "common/status.h" #include "core/string_ref.h" #include "cpp/aws_common.h" -#include "cpp/s3_rate_limiter.h" +#include "cpp/token_bucket_rate_limiter.h" #include "io/fs/obj_storage_client.h" namespace Aws::S3 { diff --git a/be/test/cloud/cloud_ms_backpressure_handler_test.cpp b/be/test/cloud/cloud_ms_backpressure_handler_test.cpp new file mode 100644 index 00000000000000..4f391c54b5d3db --- /dev/null +++ b/be/test/cloud/cloud_ms_backpressure_handler_test.cpp @@ -0,0 +1,552 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "cloud/cloud_ms_backpressure_handler.h" + +#include + +#include +#include +#include +#include +#include + +#include "cloud/config.h" + +namespace doris::cloud { + +// ============== StrictQpsLimiter Tests ============== + +class StrictQpsLimiterTest : public testing::Test { +protected: + void SetUp() override {} + void TearDown() override {} +}; + +TEST_F(StrictQpsLimiterTest, BasicFunctionality) { + StrictQpsLimiter limiter(10.0); // 10 QPS = 100ms interval + + auto t1 = limiter.reserve(); + auto t2 = limiter.reserve(); + + // Second reserve should return a time point at least ~100ms after the first + auto diff_ms = std::chrono::duration_cast(t2 - t1).count(); + EXPECT_GE(diff_ms, 90); // Allow some tolerance + EXPECT_LE(diff_ms, 110); +} + +TEST_F(StrictQpsLimiterTest, UpdateQps) { + StrictQpsLimiter limiter(10.0); // 10 QPS + + EXPECT_DOUBLE_EQ(limiter.get_qps(), 10.0); + + limiter.update_qps(100.0); + EXPECT_DOUBLE_EQ(limiter.get_qps(), 100.0); +} + +TEST_F(StrictQpsLimiterTest, ZeroQpsDefaultsToOne) { + StrictQpsLimiter limiter(0.0); + EXPECT_DOUBLE_EQ(limiter.get_qps(), 1.0); +} + +TEST_F(StrictQpsLimiterTest, MultiThreadedAccess) { + StrictQpsLimiter limiter(1000.0); // High QPS for fast test + + const int num_threads = 10; + const int calls_per_thread = 100; + std::atomic total_calls {0}; + + std::vector threads; + for (int t = 0; t < num_threads; t++) { + threads.emplace_back([&limiter, &total_calls]() { + for (int i = 0; i < calls_per_thread; i++) { + limiter.reserve(); + total_calls++; + } + }); + } + + for (auto& t : threads) { + t.join(); + } + + EXPECT_EQ(total_calls.load(), num_threads * calls_per_thread); +} + +// ============== TableRpcQpsRegistry Tests ============== + +class TableRpcQpsRegistryTest : public testing::Test { +protected: + void SetUp() override {} + + void TearDown() override {} +}; + +TEST_F(TableRpcQpsRegistryTest, RecordAndGetQps) { + TableRpcQpsRegistry registry; + + // Record some RPC calls + for (int i = 0; i < 100; i++) { + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 12345); + } + + // QPS counter uses bvar's PerSecond which needs time to calculate + // For immediate testing, we just verify the recording doesn't crash + // and get_qps returns some value (may be 0 if no time has passed) + double qps = registry.get_qps(LoadRelatedRpc::PREPARE_ROWSET, 12345); + EXPECT_GE(qps, 0.0); +} + +TEST_F(TableRpcQpsRegistryTest, MultipleTables) { + TableRpcQpsRegistry registry; + + // Record RPCs for different tables + registry.record(LoadRelatedRpc::COMMIT_ROWSET, 100); + registry.record(LoadRelatedRpc::COMMIT_ROWSET, 200); + registry.record(LoadRelatedRpc::COMMIT_ROWSET, 300); + + // Wait for bvar::PerSecond to sample and compute QPS + std::this_thread::sleep_for(std::chrono::milliseconds(1100)); + + // Each table should have independent counters + auto top_tables = registry.get_top_k_tables(LoadRelatedRpc::COMMIT_ROWSET, 3); + ASSERT_EQ(top_tables.size(), 3); + // All have equal QPS, order is undefined; verify table IDs as a set + std::vector ids; + for (const auto& [id, qps] : top_tables) { + ids.push_back(id); + } + std::sort(ids.begin(), ids.end()); + EXPECT_EQ(ids, (std::vector {100, 200, 300})); +} + +TEST_F(TableRpcQpsRegistryTest, GetTopKTables) { + TableRpcQpsRegistry registry; + + // Record different numbers of RPCs for different tables + for (int i = 0; i < 50; i++) { + registry.record(LoadRelatedRpc::UPDATE_TMP_ROWSET, 100); + } + for (int i = 0; i < 30; i++) { + registry.record(LoadRelatedRpc::UPDATE_TMP_ROWSET, 200); + } + for (int i = 0; i < 10; i++) { + registry.record(LoadRelatedRpc::UPDATE_TMP_ROWSET, 300); + } + + // Wait for bvar::PerSecond to sample and compute QPS + std::this_thread::sleep_for(std::chrono::milliseconds(1100)); + + // Get top 2 tables + auto top_tables = registry.get_top_k_tables(LoadRelatedRpc::UPDATE_TMP_ROWSET, 2); + ASSERT_EQ(top_tables.size(), 2); + EXPECT_EQ(top_tables[0].first, 100); // highest QPS + EXPECT_EQ(top_tables[1].first, 200); // second highest QPS +} + +TEST_F(TableRpcQpsRegistryTest, GetTopKTablesWithKZero) { + TableRpcQpsRegistry registry; + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 100); + + auto top_tables = registry.get_top_k_tables(LoadRelatedRpc::PREPARE_ROWSET, 0); + EXPECT_TRUE(top_tables.empty()); +} + +TEST_F(TableRpcQpsRegistryTest, GetTopKTablesWithKNegative) { + TableRpcQpsRegistry registry; + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 100); + + auto top_tables = registry.get_top_k_tables(LoadRelatedRpc::PREPARE_ROWSET, -1); + EXPECT_TRUE(top_tables.empty()); +} + +TEST_F(TableRpcQpsRegistryTest, GetTopKTablesNoRecords) { + TableRpcQpsRegistry registry; + + // No records for this RPC type + auto top_tables = registry.get_top_k_tables(LoadRelatedRpc::UPDATE_DELETE_BITMAP, 5); + EXPECT_TRUE(top_tables.empty()); +} + +TEST_F(TableRpcQpsRegistryTest, GetTopKTablesKLargerThanTableCount) { + TableRpcQpsRegistry registry; + + // Only 2 tables recorded + for (int i = 0; i < 10; i++) { + registry.record(LoadRelatedRpc::COMMIT_ROWSET, 100); + registry.record(LoadRelatedRpc::COMMIT_ROWSET, 200); + } + + // Wait for bvar::PerSecond to sample and compute QPS + std::this_thread::sleep_for(std::chrono::milliseconds(1100)); + + // Request top 100 but only 2 exist + auto top_tables = registry.get_top_k_tables(LoadRelatedRpc::COMMIT_ROWSET, 100); + ASSERT_EQ(top_tables.size(), 2); + // Equal QPS, order is undefined; verify table IDs as a set + std::vector ids; + for (const auto& [id, qps] : top_tables) { + ids.push_back(id); + } + std::sort(ids.begin(), ids.end()); + EXPECT_EQ(ids, (std::vector {100, 200})); +} + +TEST_F(TableRpcQpsRegistryTest, GetTopKTablesResultIsSortedDescending) { + TableRpcQpsRegistry registry; + + // Record many RPCs for multiple tables + for (int i = 0; i < 100; i++) { + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 100); + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 200); + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 300); + } + + // Wait for bvar::PerSecond to sample and compute QPS + std::this_thread::sleep_for(std::chrono::milliseconds(1100)); + + auto top_tables = registry.get_top_k_tables(LoadRelatedRpc::PREPARE_ROWSET, 10); + ASSERT_EQ(top_tables.size(), 3); + // Verify table IDs as a set + std::vector ids; + for (const auto& [id, qps] : top_tables) { + ids.push_back(id); + } + std::sort(ids.begin(), ids.end()); + EXPECT_EQ(ids, (std::vector {100, 200, 300})); + // Verify descending order + for (size_t i = 1; i < top_tables.size(); i++) { + EXPECT_GE(top_tables[i - 1].second, top_tables[i].second) + << "Result should be sorted by QPS descending"; + } +} + +TEST_F(TableRpcQpsRegistryTest, GetTopKTablesCrossRpcTypeIndependence) { + TableRpcQpsRegistry registry; + + // Record only for PREPARE_ROWSET + for (int i = 0; i < 50; i++) { + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 100); + } + + // Wait for bvar::PerSecond to sample and compute QPS + std::this_thread::sleep_for(std::chrono::milliseconds(1100)); + + // COMMIT_ROWSET should have no records + auto top_commit = registry.get_top_k_tables(LoadRelatedRpc::COMMIT_ROWSET, 10); + EXPECT_TRUE(top_commit.empty()); + + // PREPARE_ROWSET should have exactly 1 table + auto top_prepare = registry.get_top_k_tables(LoadRelatedRpc::PREPARE_ROWSET, 10); + ASSERT_EQ(top_prepare.size(), 1); + EXPECT_EQ(top_prepare[0].first, 100); +} + +TEST_F(TableRpcQpsRegistryTest, GetTopKTablesInvalidRpcType) { + TableRpcQpsRegistry registry; + + auto top_tables = registry.get_top_k_tables(LoadRelatedRpc::COUNT, 5); + EXPECT_TRUE(top_tables.empty()); + + top_tables = registry.get_top_k_tables(static_cast(999), 5); + EXPECT_TRUE(top_tables.empty()); +} + +// ============== TableRpcThrottler Tests ============== + +class TableRpcThrottlerTest : public testing::Test { +protected: + void SetUp() override {} + void TearDown() override {} +}; + +TEST_F(TableRpcThrottlerTest, SetAndGetQpsLimit) { + TableRpcThrottler throttler; + + EXPECT_FALSE(throttler.has_limit(LoadRelatedRpc::PREPARE_ROWSET, 12345)); + EXPECT_EQ(throttler.get_qps_limit(LoadRelatedRpc::PREPARE_ROWSET, 12345), 0.0); + + throttler.set_qps_limit(LoadRelatedRpc::PREPARE_ROWSET, 12345, 10.0); + + EXPECT_TRUE(throttler.has_limit(LoadRelatedRpc::PREPARE_ROWSET, 12345)); + EXPECT_DOUBLE_EQ(throttler.get_qps_limit(LoadRelatedRpc::PREPARE_ROWSET, 12345), 10.0); +} + +TEST_F(TableRpcThrottlerTest, RemoveQpsLimit) { + TableRpcThrottler throttler; + + throttler.set_qps_limit(LoadRelatedRpc::COMMIT_ROWSET, 100, 20.0); + EXPECT_TRUE(throttler.has_limit(LoadRelatedRpc::COMMIT_ROWSET, 100)); + + throttler.remove_qps_limit(LoadRelatedRpc::COMMIT_ROWSET, 100); + EXPECT_FALSE(throttler.has_limit(LoadRelatedRpc::COMMIT_ROWSET, 100)); +} + +TEST_F(TableRpcThrottlerTest, ThrottleWithNoLimit) { + TableRpcThrottler throttler; + + auto now = std::chrono::steady_clock::now(); + auto allowed_time = throttler.throttle(LoadRelatedRpc::UPDATE_TMP_ROWSET, 200); + + // Without a limit, should return approximately now + auto diff = std::chrono::duration_cast(allowed_time - now).count(); + EXPECT_LE(diff, 10); // Very small difference +} + +TEST_F(TableRpcThrottlerTest, ThrottleWithLimit) { + TableRpcThrottler throttler; + + // Set a very low limit (1 QPS) + throttler.set_qps_limit(LoadRelatedRpc::UPDATE_DELETE_BITMAP, 300, 1.0); + + // First call should return a time point close to now + auto t1 = throttler.throttle(LoadRelatedRpc::UPDATE_DELETE_BITMAP, 300); + + // Second call should return a time point ~1 second later + auto t2 = throttler.throttle(LoadRelatedRpc::UPDATE_DELETE_BITMAP, 300); + + auto diff_ms = std::chrono::duration_cast(t2 - t1).count(); + EXPECT_GE(diff_ms, 900); // Allow some tolerance + EXPECT_LE(diff_ms, 1100); +} + +TEST_F(TableRpcThrottlerTest, ThrottledTableCount) { + TableRpcThrottler throttler; + + EXPECT_EQ(throttler.get_throttled_table_count(LoadRelatedRpc::PREPARE_ROWSET), 0); + + throttler.set_qps_limit(LoadRelatedRpc::PREPARE_ROWSET, 100, 10.0); + throttler.set_qps_limit(LoadRelatedRpc::PREPARE_ROWSET, 200, 10.0); + + EXPECT_EQ(throttler.get_throttled_table_count(LoadRelatedRpc::PREPARE_ROWSET), 2); + + throttler.remove_qps_limit(LoadRelatedRpc::PREPARE_ROWSET, 100); + + EXPECT_EQ(throttler.get_throttled_table_count(LoadRelatedRpc::PREPARE_ROWSET), 1); +} + +TEST_F(TableRpcThrottlerTest, IndependentRpcTypes) { + TableRpcThrottler throttler; + + throttler.set_qps_limit(LoadRelatedRpc::COMMIT_ROWSET, 100, 10.0); + throttler.set_qps_limit(LoadRelatedRpc::UPDATE_TMP_ROWSET, 100, 20.0); + + EXPECT_DOUBLE_EQ(throttler.get_qps_limit(LoadRelatedRpc::COMMIT_ROWSET, 100), 10.0); + EXPECT_DOUBLE_EQ(throttler.get_qps_limit(LoadRelatedRpc::UPDATE_TMP_ROWSET, 100), 20.0); +} + +// ============== MSBackpressureHandler Tests ============== + +class MSBackpressureHandlerTest : public testing::Test { +protected: + void SetUp() override { + _saved_enable = config::enable_ms_backpressure_handling; + _saved_upgrade_interval = config::ms_backpressure_upgrade_interval_ms; + _saved_downgrade_interval = config::ms_backpressure_downgrade_interval_ms; + _saved_top_k = config::ms_backpressure_upgrade_top_k; + _saved_ratio = config::ms_backpressure_throttle_ratio; + _saved_floor = config::ms_rpc_table_qps_limit_floor; + } + + void TearDown() override { + config::enable_ms_backpressure_handling = _saved_enable; + config::ms_backpressure_upgrade_interval_ms = _saved_upgrade_interval; + config::ms_backpressure_downgrade_interval_ms = _saved_downgrade_interval; + config::ms_backpressure_upgrade_top_k = _saved_top_k; + config::ms_backpressure_throttle_ratio = _saved_ratio; + config::ms_rpc_table_qps_limit_floor = _saved_floor; + } + +private: + bool _saved_enable; + int32_t _saved_upgrade_interval; + int32_t _saved_downgrade_interval; + int32_t _saved_top_k; + double _saved_ratio; + double _saved_floor; +}; + +TEST_F(MSBackpressureHandlerTest, DisabledByDefault) { + config::enable_ms_backpressure_handling = false; + + TableRpcQpsRegistry registry; + TableRpcThrottler throttler; + MSBackpressureHandler handler(®istry, &throttler); + + // on_ms_busy should return false when disabled + EXPECT_FALSE(handler.on_ms_busy()); +} + +TEST_F(MSBackpressureHandlerTest, OnMsBusyTriggersUpgrade) { + config::enable_ms_backpressure_handling = true; + config::ms_backpressure_upgrade_interval_ms = 0; // No cooldown + config::ms_backpressure_upgrade_top_k = 3; + config::ms_backpressure_throttle_ratio = 0.5; + config::ms_rpc_table_qps_limit_floor = 1.0; + + TableRpcQpsRegistry registry; + TableRpcThrottler throttler; + MSBackpressureHandler handler(®istry, &throttler); + + // Record some RPCs so there are tables to throttle + for (int i = 0; i < 100; i++) { + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 12345); + } + + // Trigger MS_BUSY + EXPECT_TRUE(handler.on_ms_busy()); + + // Should have recorded the time + EXPECT_GE(handler.seconds_since_last_ms_busy(), 0); +} + +TEST_F(MSBackpressureHandlerTest, UpgradeIntervalRespected) { + config::enable_ms_backpressure_handling = true; + config::ms_backpressure_upgrade_interval_ms = 100000; // Long cooldown + + TableRpcQpsRegistry registry; + TableRpcThrottler throttler; + MSBackpressureHandler handler(®istry, &throttler); + + // First upgrade should succeed + EXPECT_TRUE(handler.on_ms_busy()); + + // Second upgrade should be blocked by cooldown + EXPECT_FALSE(handler.on_ms_busy()); +} + +TEST_F(MSBackpressureHandlerTest, BeforeAndAfterRpc) { + config::enable_ms_backpressure_handling = true; + + TableRpcQpsRegistry registry; + TableRpcThrottler throttler; + MSBackpressureHandler handler(®istry, &throttler); + + // before_rpc with no limit should return approximately now + auto now = std::chrono::steady_clock::now(); + auto wait_until = handler.before_rpc(LoadRelatedRpc::COMMIT_ROWSET, 12345); + + auto diff = std::chrono::duration_cast(wait_until - now).count(); + EXPECT_LE(diff, 10); + + // after_rpc should record the call (just verify it doesn't crash) + handler.after_rpc(LoadRelatedRpc::COMMIT_ROWSET, 12345); +} + +TEST_F(MSBackpressureHandlerTest, BeforeRpcWithThrottle) { + config::enable_ms_backpressure_handling = true; + + TableRpcQpsRegistry registry; + TableRpcThrottler throttler; + MSBackpressureHandler handler(®istry, &throttler); + + // Set a QPS limit + throttler.set_qps_limit(LoadRelatedRpc::UPDATE_DELETE_BITMAP, 500, 1.0); // 1 QPS + + // First call should return a time close to now + auto t1 = handler.before_rpc(LoadRelatedRpc::UPDATE_DELETE_BITMAP, 500); + + // Second call should return a time ~1 second later + auto t2 = handler.before_rpc(LoadRelatedRpc::UPDATE_DELETE_BITMAP, 500); + + auto diff_ms = std::chrono::duration_cast(t2 - t1).count(); + EXPECT_GE(diff_ms, 900); + EXPECT_LE(diff_ms, 1100); +} + +TEST_F(MSBackpressureHandlerTest, SecondsSinceLastMsBusy) { + config::enable_ms_backpressure_handling = true; + config::ms_backpressure_upgrade_interval_ms = 0; + + TableRpcQpsRegistry registry; + TableRpcThrottler throttler; + MSBackpressureHandler handler(®istry, &throttler); + + // Before any MS_BUSY, should return -1 + EXPECT_EQ(handler.seconds_since_last_ms_busy(), -1); + + handler.on_ms_busy(); + + // After MS_BUSY, should return >= 0 + EXPECT_GE(handler.seconds_since_last_ms_busy(), 0); +} + +TEST_F(MSBackpressureHandlerTest, UpgradeIntervalWithRepeatTrigger) { + config::enable_ms_backpressure_handling = true; + config::ms_backpressure_upgrade_interval_ms = 100; // 100ms cooldown + config::ms_backpressure_downgrade_interval_ms = 600000; // Large, avoid interference + + TableRpcQpsRegistry registry; + TableRpcThrottler throttler; + MSBackpressureHandler handler(®istry, &throttler); + + // First upgrade should succeed + EXPECT_TRUE(handler.on_ms_busy()); + + // Immediately second upgrade should be blocked by cooldown + EXPECT_FALSE(handler.on_ms_busy()); + + // Wait for tick thread to advance past cooldown (tick thread advances 1000 ticks/s) + std::this_thread::sleep_for(std::chrono::milliseconds(1500)); + + // After cooldown expired, third upgrade should succeed + EXPECT_TRUE(handler.on_ms_busy()); +} + +TEST_F(MSBackpressureHandlerTest, DowngradeAfterInterval) { + config::enable_ms_backpressure_handling = true; + config::ms_backpressure_upgrade_interval_ms = 0; // No cooldown + config::ms_backpressure_downgrade_interval_ms = 1000; // 1s downgrade + config::ms_backpressure_upgrade_top_k = 3; + config::ms_backpressure_throttle_ratio = 0.5; + config::ms_rpc_table_qps_limit_floor = 1.0; + + TableRpcQpsRegistry registry; + TableRpcThrottler throttler; + MSBackpressureHandler handler(®istry, &throttler); + + // Record RPCs so there are tables to throttle + for (int i = 0; i < 100; i++) { + registry.record(LoadRelatedRpc::PREPARE_ROWSET, 12345); + } + + std::this_thread::sleep_for(std::chrono::milliseconds(1100)); + + // Trigger upgrade + EXPECT_TRUE(handler.on_ms_busy()); + EXPECT_EQ(handler.upgrade_level(), 1); + + // Wait for tick thread to advance past downgrade interval + std::this_thread::sleep_for(std::chrono::milliseconds(2500)); + + // After downgrade triggered, upgrade level should have decremented + EXPECT_EQ(handler.upgrade_level(), 0); +} + +// ============== LoadRelatedRpc Utility Tests ============== + +TEST(LoadRelatedRpcTest, LoadRelatedRpcName) { + EXPECT_EQ(load_related_rpc_name(LoadRelatedRpc::PREPARE_ROWSET), "prepare_rowset"); + EXPECT_EQ(load_related_rpc_name(LoadRelatedRpc::COMMIT_ROWSET), "commit_rowset"); + EXPECT_EQ(load_related_rpc_name(LoadRelatedRpc::UPDATE_TMP_ROWSET), "update_tmp_rowset"); + EXPECT_EQ(load_related_rpc_name(LoadRelatedRpc::UPDATE_PACKED_FILE_INFO), + "update_packed_file_info"); + EXPECT_EQ(load_related_rpc_name(LoadRelatedRpc::UPDATE_DELETE_BITMAP), "update_delete_bitmap"); + EXPECT_EQ(load_related_rpc_name(LoadRelatedRpc::COUNT), "unknown"); +} + +} // namespace doris::cloud diff --git a/be/test/cloud/cloud_ms_rpc_rate_limiters_test.cpp b/be/test/cloud/cloud_ms_rpc_rate_limiters_test.cpp new file mode 100644 index 00000000000000..27a141e87eab53 --- /dev/null +++ b/be/test/cloud/cloud_ms_rpc_rate_limiters_test.cpp @@ -0,0 +1,310 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "cloud/cloud_ms_rpc_rate_limiters.h" + +#include + +#include +#include +#include + +#include "cloud/config.h" +#include "util/cpu_info.h" + +namespace doris::cloud { + +// Basic tests using uniform QPS constructor (completely independent of config and CPU cores) +class HostLevelMSRpcRateLimitersTest : public testing::Test { +protected: + void SetUp() override { _saved_enable = config::enable_ms_rpc_host_level_rate_limit; } + + void TearDown() override { config::enable_ms_rpc_host_level_rate_limit = _saved_enable; } + +private: + bool _saved_enable; +}; + +// Test that limit returns 0 when rate limiting is disabled +TEST_F(HostLevelMSRpcRateLimitersTest, DisabledRateLimiting) { + config::enable_ms_rpc_host_level_rate_limit = false; + + HostLevelMSRpcRateLimiters limiters(100); + + // Should return 0 immediately when disabled + for (int i = 0; i < 200; i++) { + EXPECT_EQ(limiters.limit(MetaServiceRPC::GET_TABLET_META), 0); + } +} + +// Test that all RPC types exist and can be rate limited +TEST_F(HostLevelMSRpcRateLimitersTest, AllRpcTypesExist) { + config::enable_ms_rpc_host_level_rate_limit = true; + + HostLevelMSRpcRateLimiters limiters(10000); // High QPS to avoid waiting + + // All RPC types should return 0 (no wait) for first call with high QPS + for (size_t i = 0; i < static_cast(MetaServiceRPC::COUNT); ++i) { + MetaServiceRPC rpc = static_cast(i); + EXPECT_EQ(limiters.limit(rpc), 0) << "RPC: " << meta_service_rpc_display_name(rpc); + } +} + +// Test that rate limiting actually throttles requests with low QPS +TEST_F(HostLevelMSRpcRateLimitersTest, RateLimitingThrottles) { + config::enable_ms_rpc_host_level_rate_limit = true; + + // Use low QPS (50) - burst is also 50, so after 50 calls we should start throttling + HostLevelMSRpcRateLimiters limiters(50); + + const int num_calls = 100; + int64_t total_sleep_ns = 0; + for (int i = 0; i < num_calls; i++) { + total_sleep_ns += limiters.limit(MetaServiceRPC::GET_TABLET_META); + } + + // With rate limiting, requests beyond burst should be throttled + EXPECT_GT(total_sleep_ns, 0) << "Rate limiting should have caused some sleep"; +} + +// Test multiple RPC types have independent rate limiters +TEST_F(HostLevelMSRpcRateLimitersTest, IndependentRateLimiters) { + config::enable_ms_rpc_host_level_rate_limit = true; + + HostLevelMSRpcRateLimiters limiters(50); + + const int num_calls = 100; + + // Exhaust one RPC type's tokens + int64_t sleep1 = 0; + for (int i = 0; i < num_calls; i++) { + sleep1 += limiters.limit(MetaServiceRPC::GET_TABLET_META); + } + + // Another RPC type should still have its own tokens + // First call should not sleep (has its own burst) + int64_t sleep2 = limiters.limit(MetaServiceRPC::GET_ROWSET); + + EXPECT_GT(sleep1, 0); + EXPECT_EQ(sleep2, 0); +} + +// Test concurrent access from multiple threads +TEST_F(HostLevelMSRpcRateLimitersTest, MultiThreadedAccess) { + config::enable_ms_rpc_host_level_rate_limit = true; + + HostLevelMSRpcRateLimiters limiters(50); + + const int num_threads = 10; + const int calls_per_thread = 20; + + std::vector threads; + std::atomic total_sleep_ns {0}; + std::atomic total_calls {0}; + + for (int t = 0; t < num_threads; t++) { + threads.emplace_back([&limiters, &total_sleep_ns, &total_calls]() { + for (int i = 0; i < calls_per_thread; i++) { + int64_t sleep = limiters.limit(MetaServiceRPC::GET_TABLET_META); + total_sleep_ns += sleep; + total_calls++; + } + }); + } + + for (auto& t : threads) { + t.join(); + } + + EXPECT_EQ(total_calls.load(), num_threads * calls_per_thread); + // Total calls (200) > burst (50), so there should be throttling + EXPECT_GT(total_sleep_ns.load(), 0); +} + +// Test reset functionality +TEST_F(HostLevelMSRpcRateLimitersTest, ResetSingleRpc) { + config::enable_ms_rpc_host_level_rate_limit = true; + + HostLevelMSRpcRateLimiters limiters(20); + + // Exhaust tokens with low QPS + int64_t sleep1 = 0; + for (int i = 0; i < 50; i++) { + sleep1 += limiters.limit(MetaServiceRPC::GET_TABLET_META); + } + EXPECT_GT(sleep1, 0); + + // Reset to high QPS + limiters.reset(MetaServiceRPC::GET_TABLET_META, 10000); + + // Should not throttle with high QPS + int64_t sleep2 = 0; + for (int i = 0; i < 50; i++) { + sleep2 += limiters.limit(MetaServiceRPC::GET_TABLET_META); + } + EXPECT_EQ(sleep2, 0); +} + +// Test display name function +TEST_F(HostLevelMSRpcRateLimitersTest, DisplayNames) { + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::GET_TABLET_META), "get tablet meta"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::GET_ROWSET), "get rowset"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::PREPARE_ROWSET), "prepare rowset"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::COMMIT_ROWSET), "commit rowset"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::UPDATE_TMP_ROWSET), + "update tmp rowset"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::COMMIT_TXN), "commit txn"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::ABORT_TXN), "abort txn"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::PRECOMMIT_TXN), "precommit txn"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::GET_OBJ_STORE_INFO), + "get obj store info"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::START_TABLET_JOB), "start tablet job"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::FINISH_TABLET_JOB), + "finish tablet job"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::GET_DELETE_BITMAP), + "get delete bitmap"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::UPDATE_DELETE_BITMAP), + "update delete bitmap"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::GET_DELETE_BITMAP_UPDATE_LOCK), + "get delete bitmap update lock"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::REMOVE_DELETE_BITMAP_UPDATE_LOCK), + "remove delete bitmap update lock"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::GET_INSTANCE), "get instance"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::PREPARE_RESTORE_JOB), + "prepare restore job"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::COMMIT_RESTORE_JOB), + "commit restore job"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::FINISH_RESTORE_JOB), + "finish restore job"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::LIST_SNAPSHOTS), "list snapshots"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::GET_CLUSTER_STATUS), + "get cluster status"); + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::UPDATE_PACKED_FILE_INFO), + "update packed file info"); +} + +// Test that invalid MetaServiceRPC returns "unknown" +TEST_F(HostLevelMSRpcRateLimitersTest, InvalidRpcDisplayName) { + EXPECT_EQ(meta_service_rpc_display_name(MetaServiceRPC::COUNT), "unknown"); + EXPECT_EQ(meta_service_rpc_display_name(static_cast(999)), "unknown"); +} + +// Config-dependent tests: verify each config controls the corresponding rate limiter +class HostLevelMSRpcRateLimitersConfigTest : public testing::Test { +protected: + void SetUp() override { + doris::CpuInfo::init(); + + _saved_enable = config::enable_ms_rpc_host_level_rate_limit; + _saved_default = config::ms_rpc_qps_default; + +// Save all per-RPC configs +#define SAVE_CONFIG(enum_name, config_suffix, display_name) \ + _saved_##config_suffix = config::ms_rpc_qps_##config_suffix; + META_SERVICE_RPC_TYPES(SAVE_CONFIG) +#undef SAVE_CONFIG + } + + void TearDown() override { + config::enable_ms_rpc_host_level_rate_limit = _saved_enable; + config::ms_rpc_qps_default = _saved_default; + +// Restore all per-RPC configs +#define RESTORE_CONFIG(enum_name, config_suffix, display_name) \ + config::ms_rpc_qps_##config_suffix = _saved_##config_suffix; + META_SERVICE_RPC_TYPES(RESTORE_CONFIG) +#undef RESTORE_CONFIG + } + +private: + bool _saved_enable; + int32_t _saved_default; + +// Declare saved config variables for each RPC +#define DECLARE_SAVED_CONFIG(enum_name, config_suffix, display_name) int32_t _saved_##config_suffix; + META_SERVICE_RPC_TYPES(DECLARE_SAVED_CONFIG) +#undef DECLARE_SAVED_CONFIG +}; + +// Test that each per-RPC config controls its corresponding rate limiter +// Directly check limiter QPS values instead of relying on throttling behavior +TEST_F(HostLevelMSRpcRateLimitersConfigTest, EachConfigControlsCorrespondingLimiter) { + config::enable_ms_rpc_host_level_rate_limit = true; + config::ms_rpc_qps_default = 10; // Default QPS + + int num_cores = doris::CpuInfo::num_cores(); + +// Test each RPC type with different config values +#define TEST_RPC_CONFIG(enum_name, config_suffix, display_name) \ + { \ + /* Test with config = 0 (disabled) */ \ + config::ms_rpc_qps_##config_suffix = 0; \ + HostLevelMSRpcRateLimiters limiters1; \ + size_t idx = static_cast(MetaServiceRPC::enum_name); \ + EXPECT_EQ(limiters1._limiters[idx].load(), nullptr) \ + << "RPC " << #enum_name << " should be disabled when config=0"; \ + \ + /* Test with config = -1 (use default) */ \ + config::ms_rpc_qps_##config_suffix = -1; \ + HostLevelMSRpcRateLimiters limiters2; \ + ASSERT_NE(limiters2._limiters[idx].load(), nullptr) \ + << "RPC " << #enum_name << " should use default when config=-1"; \ + size_t expected_qps = 10 * num_cores; \ + EXPECT_EQ(limiters2._limiters[idx].load()->limiter->get_max_speed(), expected_qps) \ + << "RPC " << #enum_name << " should use default QPS when config=-1"; \ + \ + /* Test with config = specific value */ \ + config::ms_rpc_qps_##config_suffix = 5; \ + HostLevelMSRpcRateLimiters limiters3; \ + ASSERT_NE(limiters3._limiters[idx].load(), nullptr) \ + << "RPC " << #enum_name << " should be enabled with config=5"; \ + expected_qps = 5 * num_cores; \ + EXPECT_EQ(limiters3._limiters[idx].load()->limiter->get_max_speed(), expected_qps) \ + << "RPC " << #enum_name << " should have QPS=5*num_cores when config=5"; \ + } + + META_SERVICE_RPC_TYPES(TEST_RPC_CONFIG) +#undef TEST_RPC_CONFIG +} + +// Test that reset_all re-reads config +TEST_F(HostLevelMSRpcRateLimitersConfigTest, ResetAllReadsConfig) { + config::enable_ms_rpc_host_level_rate_limit = true; + config::ms_rpc_qps_default = -1; + config::ms_rpc_qps_get_tablet_meta = 0; // Disabled initially + + HostLevelMSRpcRateLimiters limiters; + + // Should be nullptr (disabled) + size_t idx = static_cast(MetaServiceRPC::GET_TABLET_META); + EXPECT_EQ(limiters._limiters[idx].load(), nullptr) << "Should be disabled when config=0"; + + // Change config to enable with specific QPS + config::ms_rpc_qps_get_tablet_meta = 20; + + // Reset all limiters + limiters.reset_all(); + + // Should now be enabled with the new QPS + int num_cores = doris::CpuInfo::num_cores(); + size_t expected_qps = 20 * num_cores; + ASSERT_NE(limiters._limiters[idx].load(), nullptr) << "Should be enabled after reset_all"; + EXPECT_EQ(limiters._limiters[idx].load()->limiter->get_max_speed(), expected_qps) + << "Should have new QPS after reset_all"; +} + +} // namespace doris::cloud diff --git a/be/test/cloud/cloud_throttle_state_machine_test.cpp b/be/test/cloud/cloud_throttle_state_machine_test.cpp new file mode 100644 index 00000000000000..d5fc70c8647ab3 --- /dev/null +++ b/be/test/cloud/cloud_throttle_state_machine_test.cpp @@ -0,0 +1,954 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "cloud/cloud_throttle_state_machine.h" + +#include + +namespace doris::cloud { + +// ============== RpcThrottleStateMachine Tests ============== + +class RpcThrottleStateMachineTest : public testing::Test { +protected: + void SetUp() override {} + void TearDown() override {} +}; + +TEST_F(RpcThrottleStateMachineTest, SingleUpgradeAndDowngrade) { + RpcThrottleParams params {.top_k = 2, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // Construct QPS snapshot: caller already provides top-k (top-2 here) + std::vector snapshot = { + {LoadRelatedRpc::PREPARE_ROWSET, 100, 200.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 200, 100.0}, + }; + + auto actions = sm.on_upgrade(snapshot); + + // Should produce 2 SET_LIMIT actions (top-2) + ASSERT_EQ(actions.size(), 2); + + // table 100: new_limit = 200 * 0.5 = 100 + EXPECT_EQ(actions[0].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 100.0); + + // table 200: new_limit = 100 * 0.5 = 50 + EXPECT_EQ(actions[1].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[1].table_id, 200); + EXPECT_DOUBLE_EQ(actions[1].qps_limit, 50.0); + + EXPECT_EQ(sm.upgrade_level(), 1); + + // Downgrade: undo the above upgrade + auto downgrade_actions = sm.on_downgrade(); + ASSERT_EQ(downgrade_actions.size(), 2); + + // Both tables should be REMOVE_LIMIT (no previous limit) + for (const auto& action : downgrade_actions) { + EXPECT_EQ(action.type, RpcThrottleAction::Type::REMOVE_LIMIT); + } + + EXPECT_EQ(sm.upgrade_level(), 0); +} + +TEST_F(RpcThrottleStateMachineTest, MultipleUpgradesThenDowngrades) { + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // First upgrade + auto a1 = sm.on_upgrade({{LoadRelatedRpc::COMMIT_ROWSET, 100, 80.0}}); + ASSERT_EQ(a1.size(), 1); + EXPECT_DOUBLE_EQ(a1[0].qps_limit, 40.0); // 80 * 0.5 + + // Second upgrade, same table, current limit is 40 + auto a2 = sm.on_upgrade({{LoadRelatedRpc::COMMIT_ROWSET, 100, 40.0}}); + ASSERT_EQ(a2.size(), 1); + EXPECT_DOUBLE_EQ(a2[0].qps_limit, 20.0); // 40 * 0.5 (based on old limit) + + EXPECT_EQ(sm.upgrade_level(), 2); + + // First downgrade: undo second upgrade, restore to 40 + auto d1 = sm.on_downgrade(); + ASSERT_EQ(d1.size(), 1); + EXPECT_EQ(d1[0].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_DOUBLE_EQ(d1[0].qps_limit, 40.0); + + // Second downgrade: undo first upgrade, remove limit + auto d2 = sm.on_downgrade(); + ASSERT_EQ(d2.size(), 1); + EXPECT_EQ(d2[0].type, RpcThrottleAction::Type::REMOVE_LIMIT); + + EXPECT_EQ(sm.upgrade_level(), 0); +} + +TEST_F(RpcThrottleStateMachineTest, FloorQpsEnforced) { + RpcThrottleParams params {.top_k = 1, .ratio = 0.1, .floor_qps = 5.0}; + RpcThrottleStateMachine sm(params); + + // current_qps=10, 10*0.1=1.0 < floor(5.0), should use floor + auto actions = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 10.0}}); + ASSERT_EQ(actions.size(), 1); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 5.0); +} + +TEST_F(RpcThrottleStateMachineTest, DowngradeOnEmptyHistoryIsNoop) { + RpcThrottleParams params {}; + RpcThrottleStateMachine sm(params); + + auto actions = sm.on_downgrade(); + EXPECT_TRUE(actions.empty()); + EXPECT_EQ(sm.upgrade_level(), 0); +} + +TEST_F(RpcThrottleStateMachineTest, UpdateTopKAtRuntime) { + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // First upgrade, top_k=1, caller provides only 1 entry + auto a1 = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 100, 100.0}, + }); + ASSERT_EQ(a1.size(), 1); + EXPECT_EQ(a1[0].table_id, 100); + + // Runtime update top_k=3 + sm.update_params({.top_k = 3, .ratio = 0.5, .floor_qps = 1.0}); + + // Second upgrade, should now throttle both tables + auto a2 = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 100, 50.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 200, 40.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 300, 30.0}, + }); + ASSERT_EQ(a2.size(), 3); + EXPECT_EQ(a2[0].table_id, 100); + EXPECT_EQ(a2[1].table_id, 200); + EXPECT_EQ(a2[2].table_id, 300); +} + +TEST_F(RpcThrottleStateMachineTest, UpdateRatioAtRuntime) { + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // First upgrade, ratio=0.5 + auto a1 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 100.0}}); + EXPECT_DOUBLE_EQ(a1[0].qps_limit, 50.0); + + // Runtime update ratio=0.1 (more aggressive) + sm.update_params({.top_k = 1, .ratio = 0.1, .floor_qps = 1.0}); + + // Second upgrade, new ratio takes effect + auto a2 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 50.0}}); + EXPECT_DOUBLE_EQ(a2[0].qps_limit, 5.0); // 50 * 0.1 +} + +TEST_F(RpcThrottleStateMachineTest, UpdateFloorQpsAtRuntime) { + RpcThrottleParams params {.top_k = 1, .ratio = 0.01, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // First upgrade, qps=10, 10*0.01=0.1 < floor(1.0), use floor + auto a1 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 10.0}}); + EXPECT_DOUBLE_EQ(a1[0].qps_limit, 1.0); + + // Runtime update floor_qps=5.0 + sm.update_params({.top_k = 1, .ratio = 0.01, .floor_qps = 5.0}); + + // Second upgrade, new floor takes effect + auto a2 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 1.0}}); + EXPECT_DOUBLE_EQ(a2[0].qps_limit, 5.0); // 1*0.01=0.01 < floor(5.0) +} + +TEST_F(RpcThrottleStateMachineTest, MultipleRpcTypes) { + RpcThrottleParams params {.top_k = 2, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // Multiple RPC types, caller provides top-2 per RPC type + auto actions = sm.on_upgrade({ + // PREPARE_ROWSET: top-2 + {LoadRelatedRpc::PREPARE_ROWSET, 100, 200.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 200, 100.0}, + // COMMIT_ROWSET: top-2 + {LoadRelatedRpc::COMMIT_ROWSET, 100, 150.0}, + {LoadRelatedRpc::COMMIT_ROWSET, 400, 80.0}, + }); + + // Should have 4 actions: 2 for each RPC type + ASSERT_EQ(actions.size(), 4); + + // Check PREPARE_ROWSET actions + int prepare_count = 0; + int commit_count = 0; + for (const auto& action : actions) { + if (action.rpc_type == LoadRelatedRpc::PREPARE_ROWSET) { + ++prepare_count; + } else if (action.rpc_type == LoadRelatedRpc::COMMIT_ROWSET) { + ++commit_count; + } + } + EXPECT_EQ(prepare_count, 2); + EXPECT_EQ(commit_count, 2); +} + +TEST_F(RpcThrottleStateMachineTest, GetCurrentLimit) { + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 0.0); + + sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 100.0}}); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 50.0); + + sm.on_downgrade(); + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 0.0); +} + +TEST_F(RpcThrottleStateMachineTest, GetParams) { + RpcThrottleParams params {.top_k = 5, .ratio = 0.3, .floor_qps = 2.0}; + RpcThrottleStateMachine sm(params); + + auto got = sm.get_params(); + EXPECT_EQ(got.top_k, 5); + EXPECT_DOUBLE_EQ(got.ratio, 0.3); + EXPECT_DOUBLE_EQ(got.floor_qps, 2.0); + + sm.update_params({.top_k = 10, .ratio = 0.7, .floor_qps = 5.0}); + got = sm.get_params(); + EXPECT_EQ(got.top_k, 10); + EXPECT_DOUBLE_EQ(got.ratio, 0.7); + EXPECT_DOUBLE_EQ(got.floor_qps, 5.0); +} + +TEST_F(RpcThrottleStateMachineTest, NoActionWhenNotLimiting) { + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // If current_qps is already very low, applying ratio would increase it + // In this case, no action should be produced + auto actions = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 0.5}}); + EXPECT_TRUE(actions.empty()) << "Should not produce action when not actually limiting"; +} + +TEST_F(RpcThrottleStateMachineTest, OnlyLimitWhenQpsHighEnough) { + RpcThrottleParams params {.top_k = 2, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // Table 100: high QPS, should be limited + // Table 200: low QPS, should NOT be limited + auto actions = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 100, 100.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 200, 0.5}, + {LoadRelatedRpc::PREPARE_ROWSET, 300, 50.0}, + }); + + // Should only limit table 100 and 300 (top-2 by QPS, but only those where new_limit < current_qps) + ASSERT_EQ(actions.size(), 2); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_EQ(actions[1].table_id, 300); +} + +// ============== Top-K Snapshot Pattern Tests ============== + +TEST_F(RpcThrottleStateMachineTest, EmptySnapshot) { + RpcThrottleParams params {.top_k = 3, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // Empty snapshot should produce no actions and no upgrade record + auto actions = sm.on_upgrade({}); + EXPECT_TRUE(actions.empty()); + EXPECT_EQ(sm.upgrade_level(), 0); +} + +TEST_F(RpcThrottleStateMachineTest, SingleEntrySnapshot) { + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + auto actions = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 200.0}}); + ASSERT_EQ(actions.size(), 1); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 100.0); + EXPECT_EQ(sm.upgrade_level(), 1); +} + +TEST_F(RpcThrottleStateMachineTest, RepeatedUpgradeOnSameTable) { + // Simulate caller providing the same table in consecutive top-k snapshots + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // First upgrade: 100 * 0.5 = 50 + auto a1 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 100.0}}); + ASSERT_EQ(a1.size(), 1); + EXPECT_DOUBLE_EQ(a1[0].qps_limit, 50.0); + + // Second upgrade: existing limit 50 * 0.5 = 25 + auto a2 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 50.0}}); + ASSERT_EQ(a2.size(), 1); + EXPECT_DOUBLE_EQ(a2[0].qps_limit, 25.0); + + // Third upgrade: existing limit 25 * 0.5 = 12.5 + auto a3 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 25.0}}); + ASSERT_EQ(a3.size(), 1); + EXPECT_DOUBLE_EQ(a3[0].qps_limit, 12.5); + + EXPECT_EQ(sm.upgrade_level(), 3); + + // Downgrade 3 times, each restoring previous limit + auto d1 = sm.on_downgrade(); + ASSERT_EQ(d1.size(), 1); + EXPECT_DOUBLE_EQ(d1[0].qps_limit, 25.0); + + auto d2 = sm.on_downgrade(); + ASSERT_EQ(d2.size(), 1); + EXPECT_DOUBLE_EQ(d2[0].qps_limit, 50.0); + + auto d3 = sm.on_downgrade(); + ASSERT_EQ(d3.size(), 1); + EXPECT_EQ(d3[0].type, RpcThrottleAction::Type::REMOVE_LIMIT); + + EXPECT_EQ(sm.upgrade_level(), 0); +} + +TEST_F(RpcThrottleStateMachineTest, TopKTablesChangeAcrossUpgrades) { + // Different tables appear in top-k across successive upgrades + RpcThrottleParams params {.top_k = 2, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // First upgrade: tables 100, 200 + auto a1 = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 100, 200.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 200, 100.0}, + }); + ASSERT_EQ(a1.size(), 2); + EXPECT_EQ(a1[0].table_id, 100); + EXPECT_EQ(a1[1].table_id, 200); + + // Second upgrade: tables 200, 300 (100 dropped out of top-k, 300 is new) + auto a2 = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 200, 50.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 300, 80.0}, + }); + ASSERT_EQ(a2.size(), 2); + + // table 200: already limited at 50, new_limit = 50 * 0.5 = 25 + // table 300: new, 80 * 0.5 = 40 + bool found_200 = false, found_300 = false; + for (const auto& a : a2) { + if (a.table_id == 200) { + EXPECT_DOUBLE_EQ(a.qps_limit, 25.0); + found_200 = true; + } else if (a.table_id == 300) { + EXPECT_DOUBLE_EQ(a.qps_limit, 40.0); + found_300 = true; + } + } + EXPECT_TRUE(found_200); + EXPECT_TRUE(found_300); + + EXPECT_EQ(sm.upgrade_level(), 2); + + // Downgrade: undo second upgrade + // table 200 restored to 50, table 300 removed + auto d1 = sm.on_downgrade(); + ASSERT_EQ(d1.size(), 2); + for (const auto& a : d1) { + if (a.table_id == 200) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_DOUBLE_EQ(a.qps_limit, 50.0); + } else if (a.table_id == 300) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::REMOVE_LIMIT); + } + } + + // table 100 still has limit from first upgrade + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 100.0); + // table 200 restored to first upgrade limit + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 200), 50.0); + // table 300 no longer limited + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 300), 0.0); +} + +TEST_F(RpcThrottleStateMachineTest, MixedLimitedAndNewTables) { + // Snapshot contains both already-limited and never-limited tables + RpcThrottleParams params {.top_k = 3, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // First upgrade: only table 100 + auto a1 = sm.on_upgrade({{LoadRelatedRpc::COMMIT_ROWSET, 100, 100.0}}); + ASSERT_EQ(a1.size(), 1); + EXPECT_DOUBLE_EQ(a1[0].qps_limit, 50.0); + + // Second upgrade: table 100 (already limited) + table 200 (new) + auto a2 = sm.on_upgrade({ + {LoadRelatedRpc::COMMIT_ROWSET, 100, 50.0}, + {LoadRelatedRpc::COMMIT_ROWSET, 200, 80.0}, + }); + ASSERT_EQ(a2.size(), 2); + + for (const auto& a : a2) { + if (a.table_id == 100) { + // Already has limit 50, new_limit = 50 * 0.5 = 25 + EXPECT_DOUBLE_EQ(a.qps_limit, 25.0); + } else if (a.table_id == 200) { + // New, 80 * 0.5 = 40 + EXPECT_DOUBLE_EQ(a.qps_limit, 40.0); + } + } + + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::COMMIT_ROWSET, 100), 25.0); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::COMMIT_ROWSET, 200), 40.0); +} + +TEST_F(RpcThrottleStateMachineTest, FloorQpsWithRepeatedUpgrades) { + // Verify floor_qps is enforced even after many successive upgrades + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 10.0}; + RpcThrottleStateMachine sm(params); + + // 100 * 0.5 = 50 + auto a1 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 100.0}}); + EXPECT_DOUBLE_EQ(a1[0].qps_limit, 50.0); + + // 50 * 0.5 = 25 + auto a2 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 50.0}}); + EXPECT_DOUBLE_EQ(a2[0].qps_limit, 25.0); + + // 25 * 0.5 = 12.5 + auto a3 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 25.0}}); + EXPECT_DOUBLE_EQ(a3[0].qps_limit, 12.5); + + // 12.5 * 0.5 = 6.25 < floor(10), use floor + // NOTE: the ratio is applied to old_limit (12.5), not current_qps + auto a4 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 12.5}}); + EXPECT_DOUBLE_EQ(a4[0].qps_limit, 10.0); + + // Already at floor: 10 * 0.5 = 5 < floor(10), stays at floor + auto a5 = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 10.0}}); + EXPECT_DOUBLE_EQ(a5[0].qps_limit, 10.0); +} + +TEST_F(RpcThrottleStateMachineTest, MultiRpcTypeTopKIndependence) { + // Each RPC type's throttling is independent + RpcThrottleParams params {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(params); + + // Same table_id, different RPC types + auto actions = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 100, 200.0}, + {LoadRelatedRpc::COMMIT_ROWSET, 100, 80.0}, + }); + ASSERT_EQ(actions.size(), 2); + + // Each (rpc_type, table_id) gets its own limit + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 100.0); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::COMMIT_ROWSET, 100), 40.0); + + // Downgrade undoes both + auto downgrade = sm.on_downgrade(); + ASSERT_EQ(downgrade.size(), 2); + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 0.0); + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::COMMIT_ROWSET, 100), 0.0); +} + +// ============== RpcThrottleCoordinator Tests ============== + +class RpcThrottleCoordinatorTest : public testing::Test { +protected: + void SetUp() override {} + void TearDown() override {} +}; + +TEST_F(RpcThrottleCoordinatorTest, UpgradeCooldown) { + ThrottleCoordinatorParams params {.upgrade_cooldown_ticks = 10, .downgrade_after_ticks = 60}; + RpcThrottleCoordinator coord(params); + + // First MS_BUSY should trigger upgrade + EXPECT_TRUE(coord.report_ms_busy()); + + // MS_BUSY during cooldown should not trigger upgrade + for (int i = 0; i < 9; i++) { + coord.tick(); + EXPECT_FALSE(coord.report_ms_busy()); + } + + // After cooldown, can trigger again + coord.tick(); // 10th tick + EXPECT_TRUE(coord.report_ms_busy()); +} + +TEST_F(RpcThrottleCoordinatorTest, DowngradeAfterQuietPeriod) { + ThrottleCoordinatorParams params {.upgrade_cooldown_ticks = 5, .downgrade_after_ticks = 10}; + RpcThrottleCoordinator coord(params); + + // Trigger an upgrade + coord.report_ms_busy(); + coord.set_has_pending_upgrades(true); + + // 9 ticks, should not trigger downgrade + for (int i = 0; i < 9; i++) { + EXPECT_FALSE(coord.tick()); + } + + // 10th tick, trigger downgrade + EXPECT_TRUE(coord.tick()); +} + +TEST_F(RpcThrottleCoordinatorTest, MsBusyResetsDowngradeTimer) { + ThrottleCoordinatorParams params {.upgrade_cooldown_ticks = 10, .downgrade_after_ticks = 10}; + RpcThrottleCoordinator coord(params); + + coord.report_ms_busy(); + coord.set_has_pending_upgrades(true); + + // 8 ticks + for (int i = 0; i < 8; i++) { + coord.tick(); + } + + // Another MS_BUSY, downgrade timer resets + EXPECT_FALSE(coord.report_ms_busy()); // Still in cooldown + + // 9 more ticks, should not trigger downgrade (timer was reset) + for (int i = 0; i < 9; i++) { + EXPECT_FALSE(coord.tick()); + } + + // 10th tick, trigger downgrade + EXPECT_TRUE(coord.tick()); +} + +TEST_F(RpcThrottleCoordinatorTest, NoDowngradeWithoutPendingUpgrades) { + ThrottleCoordinatorParams params {.upgrade_cooldown_ticks = 5, .downgrade_after_ticks = 3}; + RpcThrottleCoordinator coord(params); + + coord.report_ms_busy(); + // Explicitly clear pending upgrades to simulate the case where + // the caller decided not to upgrade (report_ms_busy sets it internally) + coord.set_has_pending_upgrades(false); + + for (int i = 0; i < 100; i++) { + EXPECT_FALSE(coord.tick()); + } +} + +TEST_F(RpcThrottleCoordinatorTest, UpdateUpgradeCooldownAtRuntime) { + ThrottleCoordinatorParams params {.upgrade_cooldown_ticks = 10, .downgrade_after_ticks = 60}; + RpcThrottleCoordinator coord(params); + + EXPECT_TRUE(coord.report_ms_busy()); + + // 4 ticks + for (int i = 0; i < 4; i++) { + coord.tick(); + EXPECT_FALSE(coord.report_ms_busy()); // Cooling down + } + + // Runtime shorten cooldown to 5 ticks + coord.update_params({.upgrade_cooldown_ticks = 5, .downgrade_after_ticks = 60}); + + // 1 more tick (5th total), cooldown ends + coord.tick(); + EXPECT_TRUE(coord.report_ms_busy()); // Can trigger upgrade now +} + +TEST_F(RpcThrottleCoordinatorTest, UpdateDowngradeIntervalAtRuntime) { + ThrottleCoordinatorParams params {.upgrade_cooldown_ticks = 1, .downgrade_after_ticks = 20}; + RpcThrottleCoordinator coord(params); + + coord.report_ms_busy(); + coord.set_has_pending_upgrades(true); + + // 9 ticks, should not trigger downgrade (need 20) + for (int i = 0; i < 9; i++) { + EXPECT_FALSE(coord.tick()); + } + + // Runtime shorten downgrade interval to 10 ticks + coord.update_params({.upgrade_cooldown_ticks = 1, .downgrade_after_ticks = 10}); + + // 1 more tick (10th total), trigger downgrade + EXPECT_TRUE(coord.tick()); +} + +TEST_F(RpcThrottleCoordinatorTest, UpdateBothParamsAtRuntime) { + ThrottleCoordinatorParams params {.upgrade_cooldown_ticks = 100, .downgrade_after_ticks = 100}; + RpcThrottleCoordinator coord(params); + + EXPECT_TRUE(coord.report_ms_busy()); + coord.set_has_pending_upgrades(true); + + // 50 ticks, nothing should happen + for (int i = 0; i < 50; i++) { + coord.tick(); + EXPECT_FALSE(coord.report_ms_busy()); // Cooling down + EXPECT_FALSE(coord.tick()); // Downgrade interval not reached + } + + // Runtime drastically shorten both intervals + coord.update_params({.upgrade_cooldown_ticks = 1, .downgrade_after_ticks = 1}); + + // Next tick should trigger downgrade + EXPECT_TRUE(coord.tick()); + + // Next report_ms_busy should also allow upgrade + EXPECT_TRUE(coord.report_ms_busy()); +} + +TEST_F(RpcThrottleCoordinatorTest, TicksSinceLastMsBusy) { + ThrottleCoordinatorParams params {}; + RpcThrottleCoordinator coord(params); + + EXPECT_EQ(coord.ticks_since_last_ms_busy(), -1); + + coord.report_ms_busy(); + EXPECT_EQ(coord.ticks_since_last_ms_busy(), 0); + + coord.tick(); + EXPECT_EQ(coord.ticks_since_last_ms_busy(), 1); + + for (int i = 0; i < 5; i++) { + coord.tick(); + } + EXPECT_EQ(coord.ticks_since_last_ms_busy(), 6); +} + +TEST_F(RpcThrottleCoordinatorTest, TicksSinceLastUpgrade) { + ThrottleCoordinatorParams params {}; + RpcThrottleCoordinator coord(params); + + EXPECT_EQ(coord.ticks_since_last_upgrade(), -1); + + coord.report_ms_busy(); // Triggers upgrade + EXPECT_EQ(coord.ticks_since_last_upgrade(), 0); + + coord.tick(); + EXPECT_EQ(coord.ticks_since_last_upgrade(), 1); + + // Second report_ms_busy doesn't trigger upgrade (cooldown) + coord.tick(); + EXPECT_FALSE(coord.report_ms_busy()); + EXPECT_EQ(coord.ticks_since_last_upgrade(), 2); + + // After cooldown + for (int i = 0; i < 100; i++) { + coord.tick(); + } + coord.report_ms_busy(); // Triggers upgrade again + EXPECT_EQ(coord.ticks_since_last_upgrade(), 0); +} + +TEST_F(RpcThrottleCoordinatorTest, GetParams) { + ThrottleCoordinatorParams params {.upgrade_cooldown_ticks = 15, .downgrade_after_ticks = 30}; + RpcThrottleCoordinator coord(params); + + auto got = coord.get_params(); + EXPECT_EQ(got.upgrade_cooldown_ticks, 15); + EXPECT_EQ(got.downgrade_after_ticks, 30); + + coord.update_params({.upgrade_cooldown_ticks = 5, .downgrade_after_ticks = 10}); + got = coord.get_params(); + EXPECT_EQ(got.upgrade_cooldown_ticks, 5); + EXPECT_EQ(got.downgrade_after_ticks, 10); +} + +// ============== Integration Tests ============== + +TEST(RpcThrottleIntegrationTest, FullUpgradeDowngradeCycle) { + // Full in-memory upgrade/downgrade cycle, no time/config/bvar dependency + RpcThrottleParams tp {.top_k = 2, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(tp); + + ThrottleCoordinatorParams cp {.upgrade_cooldown_ticks = 3, .downgrade_after_ticks = 5}; + RpcThrottleCoordinator coord(cp); + + // Simulate 2 tables with QPS data (caller provides top-k, here top-2) + auto make_snapshot = [](double qps1, double qps2) { + return std::vector { + {LoadRelatedRpc::PREPARE_ROWSET, 100, qps1}, + {LoadRelatedRpc::PREPARE_ROWSET, 200, qps2}, + }; + }; + + // T=0: MS_BUSY, trigger first upgrade + ASSERT_TRUE(coord.report_ms_busy()); + auto actions = sm.on_upgrade(make_snapshot(100, 50)); + // top-2: table 100 (limit=50), table 200 (limit=25) + ASSERT_EQ(actions.size(), 2); + EXPECT_EQ(actions[0].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 50.0); + EXPECT_EQ(actions[1].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[1].table_id, 200); + EXPECT_DOUBLE_EQ(actions[1].qps_limit, 25.0); + coord.set_has_pending_upgrades(sm.upgrade_level() > 0); + + // T=1,2: tick, nothing happens + EXPECT_FALSE(coord.tick()); + EXPECT_FALSE(coord.tick()); + + // T=3: Another MS_BUSY (cooldown just passed), trigger second upgrade + coord.tick(); + ASSERT_TRUE(coord.report_ms_busy()); + // table 100 already has limit=50, new_limit = 50*0.5 = 25 + // table 200 already has limit=25, new_limit = 25*0.5 = 12.5 + actions = sm.on_upgrade(make_snapshot(50, 25)); + ASSERT_EQ(actions.size(), 2); + EXPECT_EQ(actions[0].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 25.0); + EXPECT_EQ(actions[1].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[1].table_id, 200); + EXPECT_DOUBLE_EQ(actions[1].qps_limit, 12.5); + EXPECT_EQ(sm.upgrade_level(), 2); + coord.set_has_pending_upgrades(true); + + // T=4..8: 5 ticks without MS_BUSY, trigger downgrade + for (int i = 0; i < 4; i++) { + EXPECT_FALSE(coord.tick()); + } + ASSERT_TRUE(coord.tick()); // T=8: 5 ticks since T=3 MS_BUSY + + // Undo second upgrade: restore table 100 to 50, table 200 to 25 + actions = sm.on_downgrade(); + ASSERT_EQ(actions.size(), 2); + for (const auto& a : actions) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::SET_LIMIT); + if (a.table_id == 100) { + EXPECT_DOUBLE_EQ(a.qps_limit, 50.0); + } else if (a.table_id == 200) { + EXPECT_DOUBLE_EQ(a.qps_limit, 25.0); + } else { + FAIL() << "Unexpected table_id: " << a.table_id; + } + } + EXPECT_EQ(sm.upgrade_level(), 1); + coord.set_has_pending_upgrades(true); + + // 5 more ticks, trigger second downgrade + for (int i = 0; i < 4; i++) { + EXPECT_FALSE(coord.tick()); + } + ASSERT_TRUE(coord.tick()); + + // Undo first upgrade: remove limits for both tables + actions = sm.on_downgrade(); + ASSERT_EQ(actions.size(), 2); + for (const auto& a : actions) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::REMOVE_LIMIT); + EXPECT_TRUE(a.table_id == 100 || a.table_id == 200) + << "Unexpected table_id: " << a.table_id; + } + EXPECT_EQ(sm.upgrade_level(), 0); + coord.set_has_pending_upgrades(false); + + // No more downgrades even after many ticks + for (int i = 0; i < 20; i++) { + EXPECT_FALSE(coord.tick()); + } +} + +TEST(RpcThrottleIntegrationTest, DynamicParamsDuringCycle) { + RpcThrottleParams tp {.top_k = 1, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(tp); + + ThrottleCoordinatorParams cp {.upgrade_cooldown_ticks = 5, .downgrade_after_ticks = 10}; + RpcThrottleCoordinator coord(cp); + + // T=0: First upgrade + coord.report_ms_busy(); + auto actions = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 100.0}}); + ASSERT_EQ(actions.size(), 1); + EXPECT_EQ(actions[0].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[0].rpc_type, LoadRelatedRpc::PREPARE_ROWSET); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 50.0); + coord.set_has_pending_upgrades(true); + + // T=1..3: ticks + for (int i = 0; i < 3; i++) { + coord.tick(); + } + + // Runtime update: change ratio to 0.1 + sm.update_params({.top_k = 1, .ratio = 0.1, .floor_qps = 1.0}); + + // T=4,5: ticks + coord.tick(); + coord.tick(); + + // T=5: cooldown passed, second upgrade with new ratio + EXPECT_TRUE(coord.report_ms_busy()); + actions = sm.on_upgrade({{LoadRelatedRpc::PREPARE_ROWSET, 100, 50.0}}); + ASSERT_EQ(actions.size(), 1); + EXPECT_EQ(actions[0].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[0].rpc_type, LoadRelatedRpc::PREPARE_ROWSET); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 5.0); // 50 * 0.1, not 50 * 0.5 + coord.set_has_pending_upgrades(true); + + // Runtime update: shorten downgrade interval to 3 ticks + coord.update_params({.upgrade_cooldown_ticks = 5, .downgrade_after_ticks = 3}); + + // 3 ticks without MS_BUSY, trigger downgrade + for (int i = 0; i < 2; i++) { + EXPECT_FALSE(coord.tick()); + } + EXPECT_TRUE(coord.tick()); + + // Undo second upgrade: restore table 100 to 50.0 + actions = sm.on_downgrade(); + ASSERT_EQ(actions.size(), 1); + EXPECT_EQ(actions[0].type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_EQ(actions[0].rpc_type, LoadRelatedRpc::PREPARE_ROWSET); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 50.0); +} + +TEST(RpcThrottleIntegrationTest, TopKTablesChangeDuringUpgradeDowngrade) { + RpcThrottleParams tp {.top_k = 2, .ratio = 0.5, .floor_qps = 1.0}; + RpcThrottleStateMachine sm(tp); + + ThrottleCoordinatorParams cp {.upgrade_cooldown_ticks = 1, .downgrade_after_ticks = 5}; + RpcThrottleCoordinator coord(cp); + + // Upgrade 1: top-2 = {100, 200} + ASSERT_TRUE(coord.report_ms_busy()); + auto actions = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 100, 200.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 200, 100.0}, + }); + ASSERT_EQ(actions.size(), 2); + EXPECT_EQ(actions[0].table_id, 100); + EXPECT_DOUBLE_EQ(actions[0].qps_limit, 100.0); // 200 * 0.5 + EXPECT_EQ(actions[1].table_id, 200); + EXPECT_DOUBLE_EQ(actions[1].qps_limit, 50.0); // 100 * 0.5 + EXPECT_EQ(sm.upgrade_level(), 1); + coord.set_has_pending_upgrades(true); + + // Upgrade 2: top-2 = {200, 300}, table 100 dropped out, table 300 is new + coord.tick(); + ASSERT_TRUE(coord.report_ms_busy()); + actions = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 200, 50.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 300, 80.0}, + }); + ASSERT_EQ(actions.size(), 2); + for (const auto& a : actions) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::SET_LIMIT); + if (a.table_id == 200) { + EXPECT_DOUBLE_EQ(a.qps_limit, 25.0); // old_limit=50, 50*0.5=25 + } else if (a.table_id == 300) { + EXPECT_DOUBLE_EQ(a.qps_limit, 40.0); // new, 80*0.5=40 + } else { + FAIL() << "Unexpected table_id: " << a.table_id; + } + } + EXPECT_EQ(sm.upgrade_level(), 2); + coord.set_has_pending_upgrades(true); + + // Upgrade 3: top-2 = {300, 400}, table 200 dropped out, table 400 is new + coord.tick(); + ASSERT_TRUE(coord.report_ms_busy()); + actions = sm.on_upgrade({ + {LoadRelatedRpc::PREPARE_ROWSET, 300, 40.0}, + {LoadRelatedRpc::PREPARE_ROWSET, 400, 60.0}, + }); + ASSERT_EQ(actions.size(), 2); + for (const auto& a : actions) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::SET_LIMIT); + if (a.table_id == 300) { + EXPECT_DOUBLE_EQ(a.qps_limit, 20.0); // old_limit=40, 40*0.5=20 + } else if (a.table_id == 400) { + EXPECT_DOUBLE_EQ(a.qps_limit, 30.0); // new, 60*0.5=30 + } else { + FAIL() << "Unexpected table_id: " << a.table_id; + } + } + EXPECT_EQ(sm.upgrade_level(), 3); + // Verify all active limits + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 100.0); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 200), 25.0); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 300), 20.0); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 400), 30.0); + coord.set_has_pending_upgrades(true); + + // Downgrade 1: undo upgrade 3, restore {300→40, 400→removed} + for (int i = 0; i < 4; i++) { + EXPECT_FALSE(coord.tick()); + } + ASSERT_TRUE(coord.tick()); + actions = sm.on_downgrade(); + ASSERT_EQ(actions.size(), 2); + for (const auto& a : actions) { + if (a.table_id == 300) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_DOUBLE_EQ(a.qps_limit, 40.0); + } else if (a.table_id == 400) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::REMOVE_LIMIT); + } else { + FAIL() << "Unexpected table_id: " << a.table_id; + } + } + EXPECT_EQ(sm.upgrade_level(), 2); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 100.0); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 200), 25.0); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 300), 40.0); + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 400), 0.0); + coord.set_has_pending_upgrades(true); + + // Downgrade 2: undo upgrade 2, restore {200→50, 300→removed} + for (int i = 0; i < 4; i++) { + EXPECT_FALSE(coord.tick()); + } + ASSERT_TRUE(coord.tick()); + actions = sm.on_downgrade(); + ASSERT_EQ(actions.size(), 2); + for (const auto& a : actions) { + if (a.table_id == 200) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::SET_LIMIT); + EXPECT_DOUBLE_EQ(a.qps_limit, 50.0); + } else if (a.table_id == 300) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::REMOVE_LIMIT); + } else { + FAIL() << "Unexpected table_id: " << a.table_id; + } + } + EXPECT_EQ(sm.upgrade_level(), 1); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 100.0); + EXPECT_DOUBLE_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 200), 50.0); + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 300), 0.0); + coord.set_has_pending_upgrades(true); + + // Downgrade 3: undo upgrade 1, restore {100→removed, 200→removed} + for (int i = 0; i < 4; i++) { + EXPECT_FALSE(coord.tick()); + } + ASSERT_TRUE(coord.tick()); + actions = sm.on_downgrade(); + ASSERT_EQ(actions.size(), 2); + for (const auto& a : actions) { + EXPECT_EQ(a.type, RpcThrottleAction::Type::REMOVE_LIMIT); + EXPECT_TRUE(a.table_id == 100 || a.table_id == 200) + << "Unexpected table_id: " << a.table_id; + } + EXPECT_EQ(sm.upgrade_level(), 0); + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 100), 0.0); + EXPECT_EQ(sm.get_current_limit(LoadRelatedRpc::PREPARE_ROWSET, 200), 0.0); + coord.set_has_pending_upgrades(false); +} + +} // namespace doris::cloud diff --git a/cloud/src/common/http_helper.cpp b/cloud/src/common/http_helper.cpp index 1d21cccde15848..150ee8498999d8 100644 --- a/cloud/src/common/http_helper.cpp +++ b/cloud/src/common/http_helper.cpp @@ -38,7 +38,7 @@ #include #include "common/metric.h" -#include "cpp/s3_rate_limiter.h" +#include "cpp/token_bucket_rate_limiter.h" #include "meta-service/meta_service.h" #include "meta-service/meta_service_helper.h" #include "meta-service/meta_service_http.h" diff --git a/cloud/src/meta-service/injection_point_http.cpp b/cloud/src/meta-service/injection_point_http.cpp index 20e35b07f8a29c..3c2ae4fee33a65 100644 --- a/cloud/src/meta-service/injection_point_http.cpp +++ b/cloud/src/meta-service/injection_point_http.cpp @@ -111,6 +111,35 @@ static void register_suites() { }); LOG_INFO("enable Transaction::commit.enable_inject"); }); + suite_map.emplace("MetaServiceProxy::inject_ms_too_busy", []() { + auto* sp = SyncPoint::get_instance(); + sp->set_call_back("MetaServiceProxy::call_impl::inject_ms_too_busy", [](auto&& args) { + // args: [&retry_times, resp->mutable_status(), &req_name, &pred] + auto* req_name = try_any_cast(args[2]); + if (*req_name != "CreateRowsetRequest") { + return; + } + auto* retry_times = try_any_cast(args[0]); + if (*retry_times == 0) { + std::mt19937 gen {std::random_device {}()}; + double p {-1.0}; + TEST_INJECTION_POINT_CALLBACK( + "MetaServiceProxy::call_impl::inject_ms_too_busy.set_p", &p); + if (p < 0 || p > 1.0) { + p = 0.01; // default injection possibility is 1% + } + std::bernoulli_distribution inject_fault {p}; + if (inject_fault(gen)) { + auto* status = try_any_cast(args[1]); + status->set_code(MetaServiceCode::MS_TOO_BUSY); + status->set_msg("injected ms too busy"); + LOG_WARNING("inject ms too busy on {} with probability {}", *req_name, p); + *try_any_cast(args.back()) = true; + } + } + }); + LOG_INFO("enable MetaServiceProxy::inject_ms_too_busy"); + }); } bool url_decode(const std::string& in, std::string* out) { diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index 95dbfc959ef1cc..a0594a945d6f6f 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -1062,10 +1062,13 @@ class MetaServiceProxy final : public MetaService { TEST_SYNC_POINT("MetaServiceProxy::call_impl:1"); + std::string req_name = req->GetDescriptor()->name(); int32_t retry_times = 0; uint64_t duration_ms = 0, retry_drift_ms = 0; while (true) { resp->Clear(); // reset the response message in case it is reused for retry + TEST_SYNC_POINT_RETURN_WITH_VOID("MetaServiceProxy::call_impl::inject_ms_too_busy", + &retry_times, resp->mutable_status(), &req_name); (impl_.get()->*method)(ctrl, req, resp, brpc::DoNothing()); MetaServiceCode code = resp->status().code(); if (code != MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE && diff --git a/cloud/src/recycler/azure_obj_client.cpp b/cloud/src/recycler/azure_obj_client.cpp index 65c0a45240353e..94901cde48b6c1 100644 --- a/cloud/src/recycler/azure_obj_client.cpp +++ b/cloud/src/recycler/azure_obj_client.cpp @@ -35,8 +35,8 @@ #include "common/config.h" #include "common/logging.h" #include "common/stopwatch.h" -#include "cpp/s3_rate_limiter.h" #include "cpp/sync_point.h" +#include "cpp/token_bucket_rate_limiter.h" #include "recycler/s3_accessor.h" #include "recycler/util.h" diff --git a/cloud/src/recycler/recycler_service.cpp b/cloud/src/recycler/recycler_service.cpp index 37912741312903..79ee2edf907711 100644 --- a/cloud/src/recycler/recycler_service.cpp +++ b/cloud/src/recycler/recycler_service.cpp @@ -40,7 +40,7 @@ #include "common/http_helper.h" #include "common/logging.h" #include "common/util.h" -#include "cpp/s3_rate_limiter.h" +#include "cpp/token_bucket_rate_limiter.h" #include "meta-service/meta_service_http.h" #include "meta-store/keys.h" #include "meta-store/txn_kv_error.h" diff --git a/cloud/src/recycler/recycler_service.h b/cloud/src/recycler/recycler_service.h index 576dee85a2eba2..3441fa7303c25e 100644 --- a/cloud/src/recycler/recycler_service.h +++ b/cloud/src/recycler/recycler_service.h @@ -17,7 +17,7 @@ #pragma once -#include +#include #include #include "meta-service/txn_lazy_committer.h" diff --git a/cloud/src/recycler/s3_accessor.cpp b/cloud/src/recycler/s3_accessor.cpp index 0f2a7776fcc7fe..9c98d3dc8af23e 100644 --- a/cloud/src/recycler/s3_accessor.cpp +++ b/cloud/src/recycler/s3_accessor.cpp @@ -47,8 +47,8 @@ #include "cpp/aws_logger.h" #include "cpp/custom_aws_credentials_provider_chain.h" #include "cpp/obj_retry_strategy.h" -#include "cpp/s3_rate_limiter.h" #include "cpp/sync_point.h" +#include "cpp/token_bucket_rate_limiter.h" #include "cpp/util.h" #ifdef USE_AZURE #include "recycler/azure_obj_client.h" diff --git a/cloud/src/recycler/s3_accessor.h b/cloud/src/recycler/s3_accessor.h index 04b29481494021..652f0c0681a2b2 100644 --- a/cloud/src/recycler/s3_accessor.h +++ b/cloud/src/recycler/s3_accessor.h @@ -33,7 +33,8 @@ class S3Client; } // namespace Aws::S3 namespace doris { -class S3RateLimiterHolder; +class TokenBucketRateLimiterHolder; +using S3RateLimiterHolder = TokenBucketRateLimiterHolder; enum class S3RateLimitType; namespace cloud { diff --git a/cloud/src/recycler/s3_obj_client.cpp b/cloud/src/recycler/s3_obj_client.cpp index 81d791cabac465..c45a6aae168c0c 100644 --- a/cloud/src/recycler/s3_obj_client.cpp +++ b/cloud/src/recycler/s3_obj_client.cpp @@ -32,8 +32,8 @@ #include "common/config.h" #include "common/logging.h" #include "common/stopwatch.h" -#include "cpp/s3_rate_limiter.h" #include "cpp/sync_point.h" +#include "cpp/token_bucket_rate_limiter.h" #include "recycler/s3_accessor.h" #include "recycler/util.h" diff --git a/cloud/test/s3_accessor_client_test.cpp b/cloud/test/s3_accessor_client_test.cpp index f6a8802cf22f46..0ea980e873443b 100644 --- a/cloud/test/s3_accessor_client_test.cpp +++ b/cloud/test/s3_accessor_client_test.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include diff --git a/cloud/test/s3_rate_limiter_test.cpp b/cloud/test/s3_rate_limiter_test.cpp index c0824f36f43397..b1fd5a8115fa6b 100644 --- a/cloud/test/s3_rate_limiter_test.cpp +++ b/cloud/test/s3_rate_limiter_test.cpp @@ -16,7 +16,7 @@ // under the License. #include -#include +#include #include #include diff --git a/common/cpp/s3_rate_limiter.cpp b/common/cpp/token_bucket_rate_limiter.cpp similarity index 75% rename from common/cpp/s3_rate_limiter.cpp rename to common/cpp/token_bucket_rate_limiter.cpp index 78e7fa582d13ec..0e30f870de368e 100644 --- a/common/cpp/s3_rate_limiter.cpp +++ b/common/cpp/token_bucket_rate_limiter.cpp @@ -15,8 +15,9 @@ // specific language governing permissions and limitations // under the License. -#include "s3_rate_limiter.h" +#include "token_bucket_rate_limiter.h" +#include #include // IWYU pragma: export #include @@ -31,7 +32,7 @@ namespace doris { // Just 10^6. static constexpr auto NS = 1000000000UL; -class S3RateLimiter::SimpleSpinLock { +class TokenBucketRateLimiter::SimpleSpinLock { public: SimpleSpinLock() = default; ~SimpleSpinLock() = default; @@ -43,7 +44,8 @@ class S3RateLimiter::SimpleSpinLock { spin_count++; if (spin_count >= MAX_SPIN_COUNT) { LOG(WARNING) << "Warning: Excessive spinning detected while acquiring lock. Spin " - "count: " << spin_count; + "count: " + << spin_count; spin_count = 0; } // Spin until we acquire the lock @@ -56,18 +58,18 @@ class S3RateLimiter::SimpleSpinLock { std::atomic_flag _flag = ATOMIC_FLAG_INIT; }; -S3RateLimiter::S3RateLimiter(size_t max_speed, size_t max_burst, size_t limit) +TokenBucketRateLimiter::TokenBucketRateLimiter(size_t max_speed, size_t max_burst, size_t limit) : _max_speed(max_speed), _max_burst(max_burst), _limit(limit), - _mutex(std::make_unique()), + _mutex(std::make_unique()), _remain_tokens(max_burst) {} -S3RateLimiter::~S3RateLimiter() = default; +TokenBucketRateLimiter::~TokenBucketRateLimiter() = default; -S3RateLimiterHolder::~S3RateLimiterHolder() = default; +TokenBucketRateLimiterHolder::~TokenBucketRateLimiterHolder() = default; -std::pair S3RateLimiter::_update_remain_token(long now, size_t amount) { +std::pair TokenBucketRateLimiter::_update_remain_token(long now, size_t amount) { // Values obtained under lock to be checked after release size_t count_value; double tokens_value; @@ -88,7 +90,7 @@ std::pair S3RateLimiter::_update_remain_token(long now, size_t a return {count_value, tokens_value}; } -int64_t S3RateLimiter::add(size_t amount) { +int64_t TokenBucketRateLimiter::add(size_t amount) { // Values obtained under lock to be checked after release auto duration = std::chrono::steady_clock::now().time_since_epoch(); auto time_nano_count = std::chrono::duration_cast(duration).count(); @@ -103,18 +105,19 @@ int64_t S3RateLimiter::add(size_t amount) { int64_t sleep_time_ns = 0; if (_max_speed && tokens_value < 0) { sleep_time_ns = static_cast(-tokens_value / _max_speed * NS); - std::this_thread::sleep_for(std::chrono::nanoseconds(sleep_time_ns)); + bthread_usleep(sleep_time_ns / 1000); } return sleep_time_ns; } -S3RateLimiterHolder::S3RateLimiterHolder(size_t max_speed, size_t max_burst, size_t limit, - std::function metric_func) - : rate_limiter(std::make_unique(max_speed, max_burst, limit)), +TokenBucketRateLimiterHolder::TokenBucketRateLimiterHolder(size_t max_speed, size_t max_burst, + size_t limit, + std::function metric_func) + : rate_limiter(std::make_unique(max_speed, max_burst, limit)), metric_func(std::move(metric_func)) {} -int64_t S3RateLimiterHolder::add(size_t amount) { +int64_t TokenBucketRateLimiterHolder::add(size_t amount) { int64_t sleep; { std::shared_lock read {rate_limiter_rw_lock}; @@ -124,10 +127,10 @@ int64_t S3RateLimiterHolder::add(size_t amount) { return sleep; } -int S3RateLimiterHolder::reset(size_t max_speed, size_t max_burst, size_t limit) { +int TokenBucketRateLimiterHolder::reset(size_t max_speed, size_t max_burst, size_t limit) { { std::unique_lock write {rate_limiter_rw_lock}; - rate_limiter = std::make_unique(max_speed, max_burst, limit); + rate_limiter = std::make_unique(max_speed, max_burst, limit); } return 0; } diff --git a/common/cpp/s3_rate_limiter.h b/common/cpp/token_bucket_rate_limiter.h similarity index 84% rename from common/cpp/s3_rate_limiter.h rename to common/cpp/token_bucket_rate_limiter.h index 357b8f9fcc17ba..dbeca3b897db03 100644 --- a/common/cpp/s3_rate_limiter.h +++ b/common/cpp/token_bucket_rate_limiter.h @@ -41,12 +41,12 @@ inline auto metric_func_factory(bvar::Adder& ns_bvar, bvar::Adder metric_func); - ~S3RateLimiterHolder(); + TokenBucketRateLimiterHolder(size_t max_speed, size_t max_burst, size_t limit, + std::function metric_func); + ~TokenBucketRateLimiterHolder(); int64_t add(size_t amount); @@ -89,9 +89,11 @@ class S3RateLimiterHolder { private: std::shared_mutex rate_limiter_rw_lock; - std::unique_ptr rate_limiter; + std::unique_ptr rate_limiter; // Record the correspoding sleeping time(unit is ms) std::function metric_func; }; +using S3RateLimiter = TokenBucketRateLimiter; +using S3RateLimiterHolder = TokenBucketRateLimiterHolder; } // namespace doris \ No newline at end of file diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java index 8807381c2ccc03..605f53b1cb9efb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java @@ -122,7 +122,9 @@ public enum SchemaTableType { TSchemaTableType.SCH_TABLE_STREAM_CONSUMPTION), SCH_BE_COMPACTION_TASKS("BE_COMPACTION_TASKS", "BE_COMPACTION_TASKS", TSchemaTableType.SCH_BE_COMPACTION_TASKS), - SCH_ROLE_MAPPINGS("ROLE_MAPPINGS", "ROLE_MAPPINGS", TSchemaTableType.SCH_ROLE_MAPPINGS); + SCH_ROLE_MAPPINGS("ROLE_MAPPINGS", "ROLE_MAPPINGS", TSchemaTableType.SCH_ROLE_MAPPINGS), + SCH_BACKEND_MS_RPC_TABLE_THROTTLERS("BACKEND_MS_RPC_TABLE_THROTTLERS", "BACKEND_MS_RPC_TABLE_THROTTLERS", + TSchemaTableType.SCH_BACKEND_MS_RPC_TABLE_THROTTLERS); private static final String dbName = "INFORMATION_SCHEMA"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java index da32fa376e8335..6b8b7b4d9d4579 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java @@ -927,6 +927,14 @@ public class SchemaTable extends Table { .column("VERTICAL_COMPLETED_GROUPS", ScalarType.createType(PrimitiveType.BIGINT)) .column("STATUS_MSG", ScalarType.createVarchar(1024)) .build())) + .put("backend_ms_rpc_table_throttlers", + new SchemaTable(SystemIdGenerator.getNextId(), "backend_ms_rpc_table_throttlers", TableType.SCHEMA, + builder().column("BE_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("TABLE_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("RPC_TYPE", ScalarType.createVarchar(64)) + .column("QPS_LIMIT", ScalarType.createType(PrimitiveType.DOUBLE)) + .column("CURRENT_QPS", ScalarType.createType(PrimitiveType.DOUBLE)) + .build())) .build(); private boolean fetchAllFe = false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java index 7dc4dc93470ebd..44b38d3055d39c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java @@ -79,6 +79,7 @@ public class BackendPartitionedSchemaScanNode extends SchemaScanNode { BACKEND_TABLE.add("column_data_sizes"); BACKEND_TABLE.add("be_compaction_tasks"); + BACKEND_TABLE.add("backend_ms_rpc_table_throttlers"); } public static boolean isBackendPartitionedSchemaTable(String tableName) { diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index 432da0d49c3f9a..f53ea523e0633c 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -219,6 +219,7 @@ enum TSchemaTableType { SCH_TABLE_STREAM_CONSUMPTION = 69; SCH_BE_COMPACTION_TASKS = 70; SCH_ROLE_MAPPINGS = 71; + SCH_BACKEND_MS_RPC_TABLE_THROTTLERS = 72; } enum THdfsCompression { diff --git a/regression-test/suites/fault_injection_p0/cloud/test_cloud_ms_rpc_table_throttle.groovy b/regression-test/suites/fault_injection_p0/cloud/test_cloud_ms_rpc_table_throttle.groovy new file mode 100644 index 00000000000000..2293cf9709db74 --- /dev/null +++ b/regression-test/suites/fault_injection_p0/cloud/test_cloud_ms_rpc_table_throttle.groovy @@ -0,0 +1,215 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.apache.doris.regression.suite.ClusterOptions + +suite('test_cloud_ms_rpc_table_throttle', 'docker') { + if (!isCloudMode()) { + return + } + + def options = new ClusterOptions() + options.cloudMode = true + options.setFeNum(1) + options.setBeNum(1) + options.beConfigs += [ + 'enable_ms_backpressure_handling=true', + // Short intervals for faster test feedback + 'ms_backpressure_upgrade_interval_ms=2000', + 'ms_backpressure_downgrade_interval_ms=5000', + 'ms_backpressure_upgrade_top_k=2', + 'ms_backpressure_throttle_ratio=0.5', + 'ms_rpc_table_qps_limit_floor=1.0', + ] + + def token = "greedisgood9999" + + def ms_inject_suite = { msHttpPort, suiteName -> + httpTest { + op "get" + endpoint msHttpPort + uri "/MetaService/http/v1/injection_point?token=${token}&op=apply_suite&name=${suiteName}" + check { respCode, body -> + log.info("apply suite ${suiteName} resp: ${body} ${respCode}".toString()) + } + } + } + + def ms_inject_enable = { msHttpPort -> + httpTest { + op "get" + endpoint msHttpPort + uri "/MetaService/http/v1/injection_point?token=${token}&op=enable" + check { respCode, body -> + log.info("enable inject resp: ${body} ${respCode}".toString()) + } + } + } + + def ms_inject_change_args = { msHttpPort, key, value -> + httpTest { + op "get" + endpoint msHttpPort + uri "/MetaService/http/v1/injection_point?token=${token}&op=set&name=${key}&behavior=change_args&value=${value}" + check { respCode, body -> + log.info("set inject args ${key}=${value} resp: ${body} ${respCode}".toString()) + } + } + } + + def ms_inject_clear = { msHttpPort -> + httpTest { + op "get" + endpoint msHttpPort + uri "/MetaService/http/v1/injection_point?token=${token}&op=clear" + check { respCode, body -> + log.info("clear inject resp: ${body} ${respCode}".toString()) + } + } + } + + docker(options) { + def ms = cluster.getAllMetaservices().get(0) + def msHttpPort = ms.host + ":" + ms.httpPort + logger.info("ms addr={}, port={}, endpoint={}", ms.host, ms.httpPort, msHttpPort) + + def tableName1 = "throttle_test_tbl1" + def tableName2 = "throttle_test_tbl2" + + sql """ DROP TABLE IF EXISTS ${tableName1} FORCE """ + sql """ DROP TABLE IF EXISTS ${tableName2} FORCE """ + sql """ + CREATE TABLE ${tableName1} ( + `k` INT NOT NULL, + `v` INT NOT NULL + ) DUPLICATE KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS 5 + PROPERTIES ("replication_num" = "1") + """ + sql """ + CREATE TABLE ${tableName2} ( + `k` INT NOT NULL, + `v` INT NOT NULL + ) DUPLICATE KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS 5 + PROPERTIES ("replication_num" = "1") + """ + + // Phase 0: Initially no throttlers + def result = sql_return_maparray """ + SELECT * FROM information_schema.backend_ms_rpc_table_throttlers + """ + logger.info("Phase 0 - initial throttlers: ${result}") + assertEquals(0, result.size()) + + // Generate RPC traffic so bvar QPS counters have data + for (int i = 0; i < 5; i++) { + sql """ INSERT INTO ${tableName1} VALUES (${i}, ${i * 10}) """ + } + for (int i = 0; i < 5; i++) { + sql """ INSERT INTO ${tableName2} VALUES (${i}, ${i * 20}) """ + } + sleep(1000) + + // Phase 1: Enable MS injection to return MS_TOO_BUSY + logger.info("Phase 1 - enabling MS injection to trigger backpressure upgrade") + ms_inject_suite.call(msHttpPort, "MetaServiceProxy::inject_ms_too_busy") + ms_inject_enable.call(msHttpPort) + // Set injection probability to 100% for deterministic testing + ms_inject_change_args.call(msHttpPort, + "MetaServiceProxy::call_impl::inject_ms_too_busy.set_p", + URLEncoder.encode('[0.5]', "UTF-8")) + + // Generate traffic to trigger MS_BUSY -> upgrade + def insertThreads = [] + for (int i = 100; i < 106; i++) { + def val = i + insertThreads << Thread.start { + try { + sql """ INSERT INTO ${tableName1} VALUES (${val}, ${val}) """ + } catch (Exception e) { + logger.info("Insert failed (expected during throttle): ${e.getMessage()}") + } + } + } + for (int i = 100; i < 106; i++) { + def val = i + insertThreads << Thread.start { + try { + sql """ INSERT INTO ${tableName2} VALUES (${val}, ${val}) """ + } catch (Exception e) { + logger.info("Insert failed (expected during throttle): ${e.getMessage()}") + } + } + } + sleep(3000) + + // Disable MS injection so subsequent operations succeed + ms_inject_clear.call(msHttpPort) + + // Phase 2: Verify throttlers exist after upgrade + def upgraded = false + for (int attempt = 0; attempt < 5; attempt++) { + result = sql_return_maparray """ + SELECT * FROM information_schema.backend_ms_rpc_table_throttlers + """ + logger.info("Phase 2 - attempt ${attempt}, throttlers: ${result}") + if (result.size() > 0) { + upgraded = true + break + } + sleep(2000) + } + assertTrue(upgraded, "Expected throttle entries after backpressure upgrade") + + for (def row : result) { + logger.info(" Throttled: BE_ID=${row.BE_ID}, TABLE_ID=${row.TABLE_ID}, " + + "RPC_TYPE=${row.RPC_TYPE}, QPS_LIMIT=${row.QPS_LIMIT}, " + + "CURRENT_QPS=${row.CURRENT_QPS}") + assertTrue(Double.parseDouble(row.QPS_LIMIT.toString()) > 0, + "QPS_LIMIT should be positive") + } + + insertThreads.each { it.join() } + + // Phase 3: Wait for downgrade (no more MS_BUSY) + logger.info("Phase 3 - waiting for throttle downgrade...") + def downgraded = false + for (int attempt = 0; attempt < 10; attempt++) { + sleep(3000) + result = sql_return_maparray """ + SELECT * FROM information_schema.backend_ms_rpc_table_throttlers + """ + logger.info("Phase 3 - attempt ${attempt}, throttlers: ${result}") + if (result.size() == 0) { + downgraded = true + break + } + } + assertTrue(downgraded, "Expected throttle entries to be removed after downgrade") + logger.info("Phase 3 - downgrade confirmed, all throttle entries removed") + + // Phase 4: Verify normal operation resumes + sql """ INSERT INTO ${tableName1} VALUES (999, 999) """ + sql """ INSERT INTO ${tableName2} VALUES (999, 999) """ + def cnt1 = sql """ SELECT COUNT(*) FROM ${tableName1} """ + def cnt2 = sql """ SELECT COUNT(*) FROM ${tableName2} """ + logger.info("Phase 4 - final row counts: tbl1=${cnt1[0][0]}, tbl2=${cnt2[0][0]}") + assertTrue(Integer.parseInt(cnt1[0][0].toString()) > 0) + assertTrue(Integer.parseInt(cnt2[0][0].toString()) > 0) + } +}