diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index d324a2c9ea5..ae8f8b30df6 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -115,6 +115,8 @@ namespace DB M(exception_when_fetch_disagg_pages) \ M(cop_send_failure) \ M(file_cache_fg_download_fail) \ + M(file_cache_bg_download_fail) \ + M(file_cache_bg_download_schedule_fail) \ M(force_set_parallel_prehandle_threshold) \ M(force_raise_prehandle_exception) \ M(force_agg_on_partial_block) \ @@ -135,6 +137,7 @@ namespace DB M(force_join_v2_probe_disable_lm) \ M(force_s3_random_access_file_init_fail) \ M(force_s3_random_access_file_read_fail) \ + M(force_s3_random_access_file_seek_chunked) \ M(force_release_snap_meet_null_storage) #define APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) \ diff --git a/dbms/src/Common/TiFlashMetrics.cpp b/dbms/src/Common/TiFlashMetrics.cpp index 4b8a30e3ebf..dc5332610d4 100644 --- a/dbms/src/Common/TiFlashMetrics.cpp +++ b/dbms/src/Common/TiFlashMetrics.cpp @@ -22,6 +22,27 @@ namespace DB { +namespace +{ +constexpr std::array remote_cache_file_type_labels = {"merged", "coldata", "other"}; +constexpr std::array remote_cache_wait_result_labels = {"hit", "timeout", "failed"}; +constexpr std::array remote_cache_reject_reason_labels = {"too_many_download"}; +constexpr std::array remote_cache_download_stage_labels = {"queue_wait", "download"}; +constexpr auto remote_cache_wait_on_downloading_buckets = ExpBuckets{0.0001, 2, 20}; +constexpr auto remote_cache_bg_download_stage_buckets = ExpBuckets{0.0001, 2, 20}; + +static_assert( + remote_cache_file_type_labels.size() == static_cast(TiFlashMetrics::RemoteCacheFileTypeMetric::Count)); +static_assert( + remote_cache_wait_result_labels.size() == static_cast(TiFlashMetrics::RemoteCacheWaitResultMetric::Count)); +static_assert( + remote_cache_reject_reason_labels.size() + == static_cast(TiFlashMetrics::RemoteCacheRejectReasonMetric::Count)); +static_assert( + remote_cache_download_stage_labels.size() + == static_cast(TiFlashMetrics::RemoteCacheDownloadStageMetric::Count)); +} // namespace + TiFlashMetrics & TiFlashMetrics::instance() { static TiFlashMetrics inst; // Instantiated on first use. @@ -78,6 +99,82 @@ TiFlashMetrics::TiFlashMetrics() .Name("tiflash_storage_s3_store_summary_bytes") .Help("S3 storage summary bytes by store and file type") .Register(*registry); + + registered_remote_cache_wait_on_downloading_result_family + = &prometheus::BuildCounter() + .Name("tiflash_storage_remote_cache_wait_on_downloading_result") + .Help("Bounded wait result of remote cache downloading") + .Register(*registry); + registered_remote_cache_wait_on_downloading_bytes_family + = &prometheus::BuildCounter() + .Name("tiflash_storage_remote_cache_wait_on_downloading_bytes") + .Help("Bytes covered by remote cache bounded wait") + .Register(*registry); + // Timeline for one cache miss with possible follower requests: + // + // req A: miss -> create Empty -> enqueue bg task ---- queue_wait ---- download ---- Complete/Failed + // req B: sees Empty -> -------- wait_on_downloading_seconds --------> hit/timeout/failed + // req C: sees Empty -> --- wait_on_downloading_seconds ---> hit/timeout/failed + // + // `tiflash_storage_remote_cache_bg_download_stage_seconds` + // - downloader-task view + // - measures how long the background download itself spent in `queue_wait` and `download` + registered_remote_cache_bg_download_stage_seconds_family + = &prometheus::BuildHistogram() + .Name("tiflash_storage_remote_cache_bg_download_stage_seconds") + .Help("Remote cache background download stage duration") + .Register(*registry); + // `tiflash_storage_remote_cache_wait_on_downloading_seconds` + // - follower-request view + // - measures how long a request waited on an existing `Empty` segment before ending as hit/timeout/failed + registered_remote_cache_wait_on_downloading_seconds_family + = &prometheus::BuildHistogram() + .Name("tiflash_storage_remote_cache_wait_on_downloading_seconds") + .Help("Bounded wait duration of remote cache downloading") + .Register(*registry); + registered_remote_cache_reject_family = &prometheus::BuildCounter() + .Name("tiflash_storage_remote_cache_reject") + .Help("Remote cache admission rejection by reason and file type") + .Register(*registry); + + for (size_t file_type_idx = 0; file_type_idx < remote_cache_file_type_labels.size(); ++file_type_idx) + { + for (size_t result_idx = 0; result_idx < remote_cache_wait_result_labels.size(); ++result_idx) + { + auto labels = prometheus::Labels{ + {"result", std::string(remote_cache_wait_result_labels[result_idx])}, + {"file_type", std::string(remote_cache_file_type_labels[file_type_idx])}, + }; + remote_cache_wait_on_downloading_result_metrics[file_type_idx][result_idx] + = ®istered_remote_cache_wait_on_downloading_result_family->Add(labels); + remote_cache_wait_on_downloading_bytes_metrics[file_type_idx][result_idx] + = ®istered_remote_cache_wait_on_downloading_bytes_family->Add(labels); + prometheus::Histogram::BucketBoundaries wait_buckets = ExpBuckets{ + remote_cache_wait_on_downloading_buckets.start, + remote_cache_wait_on_downloading_buckets.base, + remote_cache_wait_on_downloading_buckets.size}; + remote_cache_wait_on_downloading_seconds_metrics[file_type_idx][result_idx] + = ®istered_remote_cache_wait_on_downloading_seconds_family->Add(labels, wait_buckets); + } + for (size_t reason_idx = 0; reason_idx < remote_cache_reject_reason_labels.size(); ++reason_idx) + { + remote_cache_reject_metrics[file_type_idx][reason_idx] = ®istered_remote_cache_reject_family->Add( + {{"reason", std::string(remote_cache_reject_reason_labels[reason_idx])}, + {"file_type", std::string(remote_cache_file_type_labels[file_type_idx])}}); + } + for (size_t stage_idx = 0; stage_idx < remote_cache_download_stage_labels.size(); ++stage_idx) + { + prometheus::Histogram::BucketBoundaries buckets = ExpBuckets{ + remote_cache_bg_download_stage_buckets.start, + remote_cache_bg_download_stage_buckets.base, + remote_cache_bg_download_stage_buckets.size}; + remote_cache_bg_download_stage_seconds_metrics[file_type_idx][stage_idx] + = ®istered_remote_cache_bg_download_stage_seconds_family->Add( + {{"stage", std::string(remote_cache_download_stage_labels[stage_idx])}, + {"file_type", std::string(remote_cache_file_type_labels[file_type_idx])}}, + buckets); + } + } } void TiFlashMetrics::addReplicaSyncRU(UInt32 keyspace_id, UInt64 ru) @@ -287,4 +384,41 @@ void TiFlashMetrics::setS3StoreSummaryBytes(UInt64 store_id, UInt64 data_file_by it->second.data_file_bytes->Set(data_file_bytes); it->second.dt_file_bytes->Set(dt_file_bytes); } + +prometheus::Counter & TiFlashMetrics::getRemoteCacheWaitOnDownloadingResultCounter( + TiFlashMetrics::RemoteCacheFileTypeMetric file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric result) +{ + return *remote_cache_wait_on_downloading_result_metrics[static_cast(file_type)] + [static_cast(result)]; +} + +prometheus::Counter & TiFlashMetrics::getRemoteCacheWaitOnDownloadingBytesCounter( + TiFlashMetrics::RemoteCacheFileTypeMetric file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric result) +{ + return *remote_cache_wait_on_downloading_bytes_metrics[static_cast(file_type)][static_cast(result)]; +} + +prometheus::Histogram & TiFlashMetrics::getRemoteCacheWaitOnDownloadingSecondsHistogram( + TiFlashMetrics::RemoteCacheFileTypeMetric file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric result) +{ + return *remote_cache_wait_on_downloading_seconds_metrics[static_cast(file_type)] + [static_cast(result)]; +} + +prometheus::Histogram & TiFlashMetrics::getRemoteCacheBgDownloadStageSecondsHistogram( + TiFlashMetrics::RemoteCacheFileTypeMetric file_type, + TiFlashMetrics::RemoteCacheDownloadStageMetric stage) +{ + return *remote_cache_bg_download_stage_seconds_metrics[static_cast(file_type)][static_cast(stage)]; +} + +prometheus::Counter & TiFlashMetrics::getRemoteCacheRejectCounter( + TiFlashMetrics::RemoteCacheFileTypeMetric file_type, + TiFlashMetrics::RemoteCacheRejectReasonMetric reason) +{ + return *remote_cache_reject_metrics[static_cast(file_type)][static_cast(reason)]; +} } // namespace DB diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 38f0d38fc24..25504f9330d 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -27,6 +27,7 @@ #include #include +#include #include #include #include @@ -374,7 +375,7 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva Histogram, \ F(type_read, {{"type", "read"}}, EqualWidthBuckets{1 * 1024 * 1024, 60, 50 * 1024 * 1024})) \ M(tiflash_storage_io_limiter, \ - "Storage I/O limiter metrics", \ + "Storage I/O limiter byte flow", \ Counter, \ F(type_fg_read_req_bytes, {"type", "fg_read_req_bytes"}), \ F(type_fg_read_alloc_bytes, {"type", "fg_read_alloc_bytes"}), \ @@ -383,14 +384,33 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_fg_write_req_bytes, {"type", "fg_write_req_bytes"}), \ F(type_fg_write_alloc_bytes, {"type", "fg_write_alloc_bytes"}), \ F(type_bg_write_req_bytes, {"type", "bg_write_req_bytes"}), \ - F(type_bg_write_alloc_bytes, {"type", "bg_write_alloc_bytes"})) \ + F(type_bg_write_alloc_bytes, {"type", "bg_write_alloc_bytes"}), \ + F(type_s3_direct_read_bytes, {"type", "s3_direct_read_bytes"}), \ + F(type_s3_filecache_download_bytes, {"type", "s3_filecache_download_bytes"})) \ M(tiflash_storage_io_limiter_curr, \ "Current limit bytes per second of Storage I/O limiter", \ Gauge, \ F(type_fg_read_bytes, {"type", "fg_read_bytes"}), \ F(type_bg_read_bytes, {"type", "bg_read_bytes"}), \ F(type_fg_write_bytes, {"type", "fg_write_bytes"}), \ - F(type_bg_write_bytes, {"type", "bg_write_bytes"})) \ + F(type_bg_write_bytes, {"type", "bg_write_bytes"}), \ + F(type_s3_read_bytes, {"type", "s3_read_bytes"})) \ + M(tiflash_storage_io_limiter_pending_seconds, \ + "I/O limiter pending duration in seconds", \ + Histogram, \ + F(type_fg_read, {{"type", "fg_read"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_bg_read, {{"type", "bg_read"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_fg_write, {{"type", "fg_write"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_bg_write, {{"type", "bg_write"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_s3_read_byte, {{"type", "s3_read_byte"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_storage_io_limiter_pending_count, \ + "I/O limiter pending count", \ + Counter, \ + F(type_fg_read, {"type", "fg_read"}), \ + F(type_bg_read, {"type", "bg_read"}), \ + F(type_fg_write, {"type", "fg_write"}), \ + F(type_bg_write, {"type", "bg_write"}), \ + F(type_s3_read_byte, {"type", "s3_read_byte"})) \ M(tiflash_storage_rough_set_filter_rate, \ "Bucketed histogram of rough set filter rate", \ Histogram, \ @@ -882,6 +902,10 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_dtfile_full, {"type", "dtfile_full"}), \ F(type_dtfile_download, {"type", "dtfile_download"}), \ F(type_dtfile_download_failed, {"type", "dtfile_download_failed"}), \ + F(type_wait_on_downloading, {"type", "wait_on_downloading"}), \ + F(type_wait_on_downloading_hit, {"type", "wait_on_downloading_hit"}), \ + F(type_wait_on_downloading_timeout, {"type", "wait_on_downloading_timeout"}), \ + F(type_wait_on_downloading_failed, {"type", "wait_on_downloading_failed"}), \ F(type_page_hit, {"type", "page_hit"}), \ F(type_page_miss, {"type", "page_miss"}), \ F(type_page_evict, {"type", "page_evict"}), \ @@ -896,13 +920,11 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_page_evict_bytes, {"type", "page_evict_bytes"}), \ F(type_page_download_bytes, {"type", "page_download_bytes"}), \ F(type_page_read_bytes, {"type", "page_read_bytes"})) \ - M(tiflash_storage_io_limiter_pending_seconds, \ - "I/O limiter pending duration in seconds", \ - Histogram, \ - F(type_fg_read, {{"type", "fg_read"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_bg_read, {{"type", "bg_read"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_fg_write, {{"type", "fg_write"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_bg_write, {{"type", "bg_write"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_storage_remote_cache_status, \ + "Remote cache status", \ + Gauge, \ + F(type_bg_downloading_count, {{"type", "bg_downloading_count"}}), \ + F(type_bg_download_queue_count, {{"type", "bg_download_queue_count"}})) \ M(tiflash_system_seconds, \ "system calls duration in seconds", \ Histogram, \ @@ -994,13 +1016,6 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_load_dmfile_local, {{"type", "load_dmfile_local"}}, ExpBuckets{0.001, 2, 20}), \ F(type_load_dmfile_s3, {{"type", "load_dmfile_s3"}}, ExpBuckets{0.001, 2, 20}), \ F(type_search, {{"type", "search"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_storage_io_limiter_pending_count, \ - "I/O limiter pending count", \ - Counter, \ - F(type_fg_read, {"type", "fg_read"}), \ - F(type_bg_read, {"type", "bg_read"}), \ - F(type_fg_write, {"type", "fg_write"}), \ - F(type_bg_write, {"type", "bg_write"})) \ M(tiflash_read_thread_internal_us, \ "Durations of read thread internal components", \ Histogram, \ @@ -1314,6 +1329,35 @@ class TiFlashMetrics public: static TiFlashMetrics & instance(); + enum class RemoteCacheFileTypeMetric : size_t + { + Merged = 0, + ColData, + Other, + Count, + }; + + enum class RemoteCacheWaitResultMetric : size_t + { + Hit = 0, + Timeout, + Failed, + Count, + }; + + enum class RemoteCacheRejectReasonMetric : size_t + { + TooManyDownload = 0, + Count, + }; + + enum class RemoteCacheDownloadStageMetric : size_t + { + QueueWait = 0, + Download, + Count, + }; + void addReplicaSyncRU(UInt32 keyspace_id, UInt64 ru); UInt64 debugQueryReplicaSyncRU(UInt32 keyspace_id); enum class MemoryAllocType @@ -1335,6 +1379,21 @@ class TiFlashMetrics const DM::ReadRUType type); void setS3StoreSummaryBytes(UInt64 store_id, UInt64 data_file_bytes, UInt64 dt_file_bytes); + prometheus::Counter & getRemoteCacheWaitOnDownloadingResultCounter( + RemoteCacheFileTypeMetric file_type, + RemoteCacheWaitResultMetric result); + prometheus::Counter & getRemoteCacheWaitOnDownloadingBytesCounter( + RemoteCacheFileTypeMetric file_type, + RemoteCacheWaitResultMetric result); + prometheus::Histogram & getRemoteCacheWaitOnDownloadingSecondsHistogram( + RemoteCacheFileTypeMetric file_type, + RemoteCacheWaitResultMetric result); + prometheus::Histogram & getRemoteCacheBgDownloadStageSecondsHistogram( + RemoteCacheFileTypeMetric file_type, + RemoteCacheDownloadStageMetric stage); + prometheus::Counter & getRemoteCacheRejectCounter( + RemoteCacheFileTypeMetric file_type, + RemoteCacheRejectReasonMetric reason); private: TiFlashMetrics(); @@ -1386,6 +1445,32 @@ class TiFlashMetrics std::shared_mutex s3_store_summary_bytes_mtx; std::unordered_map registered_s3_store_summary_bytes_metrics; + prometheus::Family * registered_remote_cache_wait_on_downloading_result_family; + std::array< + std::array(RemoteCacheWaitResultMetric::Count)>, + static_cast(RemoteCacheFileTypeMetric::Count)> + remote_cache_wait_on_downloading_result_metrics{}; + prometheus::Family * registered_remote_cache_wait_on_downloading_bytes_family; + std::array< + std::array(RemoteCacheWaitResultMetric::Count)>, + static_cast(RemoteCacheFileTypeMetric::Count)> + remote_cache_wait_on_downloading_bytes_metrics{}; + prometheus::Family * registered_remote_cache_wait_on_downloading_seconds_family; + std::array< + std::array(RemoteCacheWaitResultMetric::Count)>, + static_cast(RemoteCacheFileTypeMetric::Count)> + remote_cache_wait_on_downloading_seconds_metrics{}; + prometheus::Family * registered_remote_cache_bg_download_stage_seconds_family; + std::array< + std::array(RemoteCacheDownloadStageMetric::Count)>, + static_cast(RemoteCacheFileTypeMetric::Count)> + remote_cache_bg_download_stage_seconds_metrics{}; + prometheus::Family * registered_remote_cache_reject_family; + std::array< + std::array(RemoteCacheRejectReasonMetric::Count)>, + static_cast(RemoteCacheFileTypeMetric::Count)> + remote_cache_reject_metrics{}; + public: #define MAKE_METRIC_MEMBER_M(family_name, help, type, ...) \ MetricFamily family_name \ diff --git a/dbms/src/IO/BaseFile/IORateLimitConfig.cpp b/dbms/src/IO/BaseFile/IORateLimitConfig.cpp index 90bc9abd5e4..8a44d4f91d1 100644 --- a/dbms/src/IO/BaseFile/IORateLimitConfig.cpp +++ b/dbms/src/IO/BaseFile/IORateLimitConfig.cpp @@ -52,6 +52,7 @@ void IORateLimitConfig::parse(const String & storage_io_rate_limit, const Logger readConfig(config, "max_bytes_per_sec", max_bytes_per_sec); readConfig(config, "max_read_bytes_per_sec", max_read_bytes_per_sec); readConfig(config, "max_write_bytes_per_sec", max_write_bytes_per_sec); + readConfig(config, "s3_max_read_bytes_per_sec", s3_max_read_bytes_per_sec); readConfig(config, "foreground_write_weight", fg_write_weight); readConfig(config, "background_write_weight", bg_write_weight); readConfig(config, "foreground_read_weight", fg_read_weight); @@ -72,6 +73,7 @@ std::string IORateLimitConfig::toString() const { return fmt::format( "IORateLimitConfig{{max_bytes_per_sec={} max_read_bytes_per_sec={} max_write_bytes_per_sec={} " + "s3_max_read_bytes_per_sec={} " "use_max_bytes_per_sec={} " "fg_write_weight={} bg_write_weight={} fg_read_weight={} bg_read_weight={} " "fg_write_max_bytes_per_sec={} bg_write_max_bytes_per_sec={} " @@ -80,6 +82,7 @@ std::string IORateLimitConfig::toString() const max_bytes_per_sec, max_read_bytes_per_sec, max_write_bytes_per_sec, + s3_max_read_bytes_per_sec, use_max_bytes_per_sec, fg_write_weight, bg_write_weight, @@ -165,11 +168,13 @@ UInt64 IORateLimitConfig::getReadMaxBytesPerSec() const bool IORateLimitConfig::operator==(const IORateLimitConfig & config) const { return config.max_bytes_per_sec == max_bytes_per_sec && config.max_read_bytes_per_sec == max_read_bytes_per_sec - && config.max_write_bytes_per_sec == max_write_bytes_per_sec && config.bg_write_weight == bg_write_weight - && config.fg_write_weight == fg_write_weight && config.bg_read_weight == bg_read_weight - && config.fg_read_weight == fg_read_weight && config.emergency_pct == emergency_pct - && config.high_pct == high_pct && config.medium_pct == medium_pct && config.tune_base == tune_base - && config.min_bytes_per_sec == min_bytes_per_sec && config.auto_tune_sec == auto_tune_sec; + && config.max_write_bytes_per_sec == max_write_bytes_per_sec + && config.s3_max_read_bytes_per_sec == s3_max_read_bytes_per_sec // + && config.bg_write_weight == bg_write_weight && config.fg_write_weight == fg_write_weight + && config.bg_read_weight == bg_read_weight && config.fg_read_weight == fg_read_weight + && config.emergency_pct == emergency_pct && config.high_pct == high_pct && config.medium_pct == medium_pct + && config.tune_base == tune_base && config.min_bytes_per_sec == min_bytes_per_sec + && config.auto_tune_sec == auto_tune_sec; } } // namespace DB diff --git a/dbms/src/IO/BaseFile/IORateLimitConfig.h b/dbms/src/IO/BaseFile/IORateLimitConfig.h index f8901e776ce..68652622017 100644 --- a/dbms/src/IO/BaseFile/IORateLimitConfig.h +++ b/dbms/src/IO/BaseFile/IORateLimitConfig.h @@ -28,6 +28,8 @@ struct IORateLimitConfig // For disk that read bandwidth and write bandwith are calculated separately, such as GCP's persistent disks. UInt64 max_read_bytes_per_sec; UInt64 max_write_bytes_per_sec; + // Node-level byte budget shared by all S3 direct reads and FileCache downloads. `0` disables byte throttling. + UInt64 s3_max_read_bytes_per_sec; // only true when both max_read_bytes_per_sec and max_write_bytes_per_sec are 0 bool use_max_bytes_per_sec; @@ -54,6 +56,7 @@ struct IORateLimitConfig : max_bytes_per_sec(0) , max_read_bytes_per_sec(0) , max_write_bytes_per_sec(0) + , s3_max_read_bytes_per_sec(0) , use_max_bytes_per_sec(true) // only limit background write by default , fg_write_weight(0) diff --git a/dbms/src/IO/BaseFile/RateLimiter.cpp b/dbms/src/IO/BaseFile/RateLimiter.cpp index b36df903586..dab45dbc62e 100644 --- a/dbms/src/IO/BaseFile/RateLimiter.cpp +++ b/dbms/src/IO/BaseFile/RateLimiter.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -482,6 +483,12 @@ ReadLimiterPtr IORateLimiter::getReadLimiter() return is_background_thread ? bg_read_limiter : fg_read_limiter; } +std::shared_ptr IORateLimiter::getS3ReadLimiter() +{ + std::lock_guard lock(limiter_mtx); + return s3_read_limiter; +} + void IORateLimiter::updateConfig(Poco::Util::AbstractConfiguration & config_) { if (!reloadConfig(config_)) @@ -518,6 +525,28 @@ void IORateLimiter::updateLimiterByConfig(const IORateLimitConfig & cfg) std::lock_guard lock(limiter_mtx); updateReadLimiter(cfg.getBgReadMaxBytesPerSec(), cfg.getFgReadMaxBytesPerSec()); updateWriteLimiter(cfg.getBgWriteMaxBytesPerSec(), cfg.getFgWriteMaxBytesPerSec()); + + // updateS3ReadLimiter + // Keep an existing S3 limiter object alive across reloads so readers that already snapped the + // shared_ptr can observe `nonzero -> 0` disable updates via `updateConfig(0)` instead of being + // stuck with a stale throttling state. Today we intentionally accept a narrower semantic on the + // first `0 -> nonzero` transition: if startup published no limiter object, only newly created + // readers will see the limiter after it is first created here. + // TODO: Consider publishing a no-op S3ReadLimiter even when the configured rate is 0, so a later + // `0 -> nonzero` reload can also reach readers that previously snapped a nullptr. + if (s3_read_limiter == nullptr) + { + if (cfg.s3_max_read_bytes_per_sec != 0) + { + s3_read_limiter = std::make_shared(cfg.s3_max_read_bytes_per_sec); + if (stop.load(std::memory_order_relaxed)) + s3_read_limiter->setStop(); + } + } + else + { + s3_read_limiter->updateConfig(cfg.s3_max_read_bytes_per_sec); + } } void IORateLimiter::updateReadLimiter(Int64 bg_bytes, Int64 fg_bytes) @@ -685,6 +714,8 @@ void IORateLimiter::setStop() auto sz = fg_read_limiter->setStop(); LOG_DEBUG(log, "fg_read_limiter setStop request size {}", sz); } + if (s3_read_limiter != nullptr) + s3_read_limiter->setStop(); } void IORateLimiter::runAutoTune() diff --git a/dbms/src/IO/BaseFile/RateLimiter.h b/dbms/src/IO/BaseFile/RateLimiter.h index 3fce18e6e6e..96037169108 100644 --- a/dbms/src/IO/BaseFile/RateLimiter.h +++ b/dbms/src/IO/BaseFile/RateLimiter.h @@ -33,6 +33,10 @@ namespace DB { class LimiterStat; class IOLimitTuner; +namespace S3 +{ +class S3ReadLimiter; +} enum class LimiterType { @@ -216,6 +220,7 @@ class IORateLimiter WriteLimiterPtr getBgWriteLimiter(); ReadLimiterPtr getReadLimiter(); + std::shared_ptr getS3ReadLimiter(); void init(Poco::Util::AbstractConfiguration & config_); void updateConfig(Poco::Util::AbstractConfiguration & config_); @@ -250,6 +255,7 @@ class IORateLimiter // Background read and foreground read ReadLimiterPtr bg_read_limiter; ReadLimiterPtr fg_read_limiter; + std::shared_ptr s3_read_limiter; std::mutex bg_thread_ids_mtx; std::vector bg_thread_ids; diff --git a/dbms/src/IO/BaseFile/tests/gtest_rate_limiter.cpp b/dbms/src/IO/BaseFile/tests/gtest_rate_limiter.cpp index a50dcf880d6..9418fa4703d 100644 --- a/dbms/src/IO/BaseFile/tests/gtest_rate_limiter.cpp +++ b/dbms/src/IO/BaseFile/tests/gtest_rate_limiter.cpp @@ -14,12 +14,14 @@ #include #include +#include #include #include #include #include #include +#include #include #include @@ -374,6 +376,67 @@ TEST(ReadLimiterTest, ReadMany) ASSERT_EQ(read_limiter.alloc_bytes, 100); } +TEST(S3ReadLimiterTest, ByteRequestsWaitForRefill) +{ + S3::S3ReadLimiter limiter(1000, 100); + // Consume the initial 100-byte burst, then verify the next request waits for at least one refill period. + limiter.requestBytes(100, S3::S3ReadSource::DirectRead); + AtomicStopwatch watch; + limiter.requestBytes(100, S3::S3ReadSource::DirectRead); + ASSERT_GE(watch.elapsedMilliseconds(), 80); +} + +TEST(S3ReadLimiterTest, UpdateConfigDisablesWaitingBytes) +{ + S3::S3ReadLimiter limiter(1000, 100); + // Exhaust the initial burst, then make sure disabling the byte limit wakes a waiting requester promptly. + limiter.requestBytes(100, S3::S3ReadSource::DirectRead); + + std::promise waiter_started; + auto waiter_started_future = waiter_started.get_future(); + auto future = std::async(std::launch::async, [&]() { + AtomicStopwatch watch; + waiter_started.set_value(); + limiter.requestBytes(100, S3::S3ReadSource::DirectRead); + return watch.elapsedMilliseconds(); + }); + + ASSERT_EQ(waiter_started_future.wait_for(1s), std::future_status::ready); + ASSERT_EQ(future.wait_for(50ms), std::future_status::timeout); + + limiter.updateConfig(/*max_read_bytes_per_sec*/ 0); + ASSERT_LT(future.get(), 100); +} + +TEST(S3ReadLimiterTest, SuggestedChunkSizeTracksBurstLimit) +{ + // The suggested chunk size should never exceed one refill-period burst when byte limiting is enabled. + S3::S3ReadLimiter limiter(/*max_read_bytes_per_sec*/ 1000, /*refill_period_ms*/ 100); + ASSERT_EQ(limiter.getSuggestedChunkSize(128 * 1024), 100); + + limiter.updateConfig(/*max_read_bytes_per_sec*/ 5000); + ASSERT_EQ(limiter.getSuggestedChunkSize(128 * 1024), 500); + + limiter.updateConfig(/*max_read_bytes_per_sec*/ 0); + ASSERT_EQ(limiter.getSuggestedChunkSize(4096), 4096); +} + +TEST(S3ReadLimiterTest, LargeRequestDoesNotWaitForever) +{ + S3::S3ReadLimiter limiter(/*max_read_bytes_per_sec*/ 1000, /*refill_period_ms*/ 100); + + // The initial burst is only 100 bytes, but callers that request a larger chunk should still make + // forward progress instead of waiting forever for a budget that can never accumulate. + auto future = std::async(std::launch::async, [&] { + AtomicStopwatch watch; + limiter.requestBytes(128 * 1024, S3::S3ReadSource::DirectRead); + return watch.elapsedMilliseconds(); + }); + + ASSERT_EQ(future.wait_for(1s), std::future_status::ready); + ASSERT_LT(future.get(), 200); +} + #ifdef __linux__ TEST(IORateLimiterTest, IOStat) { diff --git a/dbms/src/IO/Checksum/ChecksumBuffer.cpp b/dbms/src/IO/Checksum/ChecksumBuffer.cpp index a8aa4e4b4da..99f0fad9fd2 100644 --- a/dbms/src/IO/Checksum/ChecksumBuffer.cpp +++ b/dbms/src/IO/Checksum/ChecksumBuffer.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include namespace DB { @@ -53,10 +54,26 @@ off_t FramedChecksumReadBuffer::doSeek(off_t offset, int whence) auto result = in->seek(static_cast(header_offset), SEEK_SET); if (result < 0) { + LOG_WARNING( + Logger::get("FramedChecksumReadBuffer"), + "failed to seek underlying reader while loading checksum frame, file={} whence={} target_frame={} " + "target_offset={} header_offset={} underlying_seek_ret={}", + in->getFileName(), + whence, + target_frame, + target_offset, + header_offset, + result); throw TiFlashException( Errors::Checksum::IOFailure, - "checksum framed file {} is not seekable", - in->getFileName()); + "failed to seek checksum framed file {} to frame boundary: underlying reader returned {} for " + "header_offset={} target_frame={} target_offset={} whence={}", + in->getFileName(), + result, + header_offset, + target_frame, + target_offset, + whence); } auto length = expectRead( working_buffer.begin() - sizeof(ChecksumFrame), diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 4a27061e924..8497cd26260 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1217,6 +1217,15 @@ void Context::dropMarkCache() const shared->mark_cache->reset(); } +bool Context::dropMarkCacheAndReport() const +{ + auto lock = getLock(); + if (shared->mark_cache == nullptr) + return false; + shared->mark_cache->reset(); + return true; +} + void Context::setMinMaxIndexCache(size_t cache_size_in_bytes) { @@ -1241,6 +1250,15 @@ void Context::dropMinMaxIndexCache() const shared->minmax_index_cache->reset(); } +bool Context::dropMinMaxIndexCacheAndReport() const +{ + auto lock = getLock(); + if (shared->minmax_index_cache == nullptr) + return false; + shared->minmax_index_cache->reset(); + return true; +} + void Context::setLocalIndexCache(size_t light_local_index_cache, size_t heavy_cache_entities) { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index fc9ed1735f9..8790f46e5d8 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -392,10 +392,14 @@ class Context void setMarkCache(size_t cache_size_in_bytes); std::shared_ptr getMarkCache() const; void dropMarkCache() const; + /// Reset MarkCache and report whether it was enabled before the reset. + bool dropMarkCacheAndReport() const; void setMinMaxIndexCache(size_t cache_size_in_bytes); std::shared_ptr getMinMaxIndexCache() const; void dropMinMaxIndexCache() const; + /// Reset MinMaxIndexCache and report whether it was enabled before the reset. + bool dropMinMaxIndexCacheAndReport() const; void setLocalIndexCache(size_t light_local_index_cache, size_t heavy_cache_entities); std::shared_ptr getLightLocalIndexCache() const; diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 4bdf83ae6df..a0b362dd8a3 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -238,16 +238,17 @@ struct Settings M(SettingBool, remote_checkpoint_only_upload_manifest, true, "Only upload manifest data when uploading checkpoint") \ M(SettingInt64, remote_gc_method, 1, "The method of running GC task on the remote store. 1 - lifecycle, 2 - scan.") \ M(SettingInt64, remote_gc_interval_seconds, 3600, "The interval of running GC task on the remote store. Unit is second.") \ - M(SettingInt64, remote_summary_interval_seconds, 0, "The interval of collecting remote S3 storage summary. Unit is second. <=0 disables periodic summary task.") \ + M(SettingInt64, remote_summary_interval_seconds, 0, "The interval of collecting remote S3 storage summary. Unit is second. <=0 disables periodic summary task.") \ M(SettingInt64, remote_gc_verify_consistency, 0, "[testing] Verify the consistenct of valid locks when doing GC") \ M(SettingInt64, remote_gc_min_age_seconds, 3600, "The file will NOT be compacted when the time difference between the last modification is less than this threshold") \ M(SettingDouble, remote_gc_ratio, 0.5, "The files with valid rate less than this threshold will be compacted") \ M(SettingInt64, remote_gc_small_size, 128 * 1024, "The files with total size less than this threshold will be compacted") \ /* Disagg arch reading settings */ \ M(SettingUInt64, dt_write_page_cache_limit_size, 2 * 1024 * 1024, "Limit size per write batch when compute node writing to PageStorage cache") \ - M(SettingDouble, dt_filecache_downloading_count_scale, 2.0, "Max concurrency of download task count of FileCache = number of logical cpu cores * dt_filecache_downloading_count_scale.") \ + M(SettingDouble, dt_filecache_downloading_count_scale, 2.0, "Max concurrency of download task count of FileCache = number of logical cpu cores * dt_filecache_downloading_count_scale.") \ M(SettingDouble, dt_filecache_max_downloading_count_scale, 10.0, "Max queue size of download task count of FileCache = number of logical cpu cores * dt_filecache_max_downloading_count_scale.") \ M(SettingUInt64, dt_filecache_min_age_seconds, 1800, "Files of the same priority can only be evicted from files that were not accessed within `dt_filecache_min_age_seconds` seconds.") \ + M(SettingUInt64, dt_filecache_wait_on_downloading_ms, 0, "When a remote cache lookup sees the same key is already being downloaded, wait up to this many milliseconds for that download to finish. 0 disables the bounded wait.") \ M(SettingBool, dt_enable_fetch_memtableset, true, "Whether fetching delta cache in FetchDisaggPages") \ M(SettingUInt64, dt_fetch_pages_packet_limit_size, 512 * 1024, "Response packet bytes limit of FetchDisaggPages, 0 means one page per packet") \ M(SettingDouble, dt_fetch_page_concurrency_scale, 4.0, "Concurrency of fetching pages of one query equals to num_streams * dt_fetch_page_concurrency_scale.") \ @@ -337,7 +338,7 @@ struct Settings M(SettingUInt64, cop_timeout_for_remote_read, 60, "cop timeout seconds for remote read") \ M(SettingUInt64, auto_spill_check_min_interval_ms, 10, "The minimum interval in millisecond between two successive auto spill check, default value is 100, 0 means no limit") \ M(SettingUInt64, join_probe_cache_columns_threshold, 1000, "The threshold that a join key will cache its output columns during probe stage, 0 means never cache") \ - M(SettingBool, enable_hash_join_v2, false, "Enable hash join v2") \ + M(SettingBool, enable_hash_join_v2, false, "Enable hash join v2") \ M(SettingUInt64, join_v2_max_block_size, 8192, "hash join v2 max block size") \ M(SettingUInt64, join_v2_probe_enable_prefetch_threshold, 1024 * 1024, "hash join v2 minimum row number of join build table to use prefetch during join probe phase") \ M(SettingUInt64, join_v2_probe_prefetch_step, 16, "hash join v2 probe prefetch length") \ diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 969e7a3b9ea..082b487973b 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -698,6 +698,10 @@ try config, server_info.cpu_info.logical_cores, global_context->getIORateLimiter()); + // FileCache::initialize() only constructs the global instance. Push the current settings once + // here so startup-time values like dt_filecache_wait_on_downloading_ms take effect immediately + // instead of waiting for a later config reload. + FileCache::instance()->updateConfig(global_context->getSettingsRef()); } /// Determining PageStorage run mode based on current files on disk and storage config. @@ -864,6 +868,9 @@ try /// Initialize RateLimiter. global_context->initializeRateLimiter(config(), bg_pool, blockable_bg_pool); + // ClientFactory keeps the process-wide shared S3 client. Publish the latest limiter explicitly so + // every existing and future `TiFlashS3Client` observes the same node-level S3 read budget. + S3::ClientFactory::instance().setS3ReadLimiter(global_context->getIORateLimiter().getS3ReadLimiter()); global_context->setServerInfo(server_info); if (server_info.memory_info.capacity == 0) @@ -893,6 +900,8 @@ try buildLoggers(*config); global_context->getTMTContext().reloadConfig(*config); global_context->getIORateLimiter().updateConfig(*config); + // Config reload may replace the limiter instance or disable it. Re-publish it to the shared S3 client. + S3::ClientFactory::instance().setS3ReadLimiter(global_context->getIORateLimiter().getS3ReadLimiter()); global_context->reloadDeltaTreeConfig(*config); DM::SegmentReadTaskScheduler::instance().updateConfig(global_context->getSettingsRef()); if (FileCache::instance() != nullptr) diff --git a/dbms/src/Server/tests/gtest_storage_config.cpp b/dbms/src/Server/tests/gtest_storage_config.cpp index 4d4561069cc..6459fac75e6 100644 --- a/dbms/src/Server/tests/gtest_storage_config.cpp +++ b/dbms/src/Server/tests/gtest_storage_config.cpp @@ -595,6 +595,7 @@ try max_bytes_per_sec=0 max_read_bytes_per_sec=0 max_write_bytes_per_sec=0 +s3_max_read_bytes_per_sec=0 foreground_write_weight=1 background_write_weight=2 foreground_read_weight=5 @@ -606,6 +607,7 @@ background_read_weight=2 max_bytes_per_sec=1024000 max_read_bytes_per_sec=0 max_write_bytes_per_sec=0 +s3_max_read_bytes_per_sec=2048000 foreground_write_weight=1 background_write_weight=2 foreground_read_weight=5 @@ -617,6 +619,7 @@ background_read_weight=2 max_bytes_per_sec=0 max_read_bytes_per_sec=1024000 max_write_bytes_per_sec=1024000 +s3_max_read_bytes_per_sec=1024 foreground_write_weight=1 background_write_weight=2 foreground_read_weight=5 @@ -628,6 +631,7 @@ background_read_weight=2 max_bytes_per_sec=1024000 max_read_bytes_per_sec=1024000 max_write_bytes_per_sec=1024000 +s3_max_read_bytes_per_sec=4096 foreground_write_weight=1 background_write_weight=2 foreground_read_weight=5 @@ -638,6 +642,7 @@ background_read_weight=2 [storage] [storage.io_rate_limit] max_bytes_per_sec=1024000 + s3_max_read_bytes_per_sec=8192 foreground_write_weight=80 background_write_weight=20 foreground_read_weight=0 @@ -651,6 +656,7 @@ background_read_weight=2 ASSERT_EQ(io_config.max_bytes_per_sec, 0); ASSERT_EQ(io_config.max_read_bytes_per_sec, 0); ASSERT_EQ(io_config.max_write_bytes_per_sec, 0); + ASSERT_EQ(io_config.s3_max_read_bytes_per_sec, 0); ASSERT_TRUE(io_config.use_max_bytes_per_sec); ASSERT_EQ(io_config.fg_write_weight, 0); ASSERT_EQ(io_config.bg_write_weight, 100); @@ -669,6 +675,7 @@ background_read_weight=2 ASSERT_EQ(io_config.max_bytes_per_sec, 0); ASSERT_EQ(io_config.max_read_bytes_per_sec, 0); ASSERT_EQ(io_config.max_write_bytes_per_sec, 0); + ASSERT_EQ(io_config.s3_max_read_bytes_per_sec, 0); ASSERT_TRUE(io_config.use_max_bytes_per_sec); ASSERT_EQ(io_config.fg_write_weight, 1); ASSERT_EQ(io_config.bg_write_weight, 2); @@ -687,6 +694,7 @@ background_read_weight=2 ASSERT_EQ(io_config.max_bytes_per_sec, 1024000); ASSERT_EQ(io_config.max_read_bytes_per_sec, 0); ASSERT_EQ(io_config.max_write_bytes_per_sec, 0); + ASSERT_EQ(io_config.s3_max_read_bytes_per_sec, 2048000); ASSERT_TRUE(io_config.use_max_bytes_per_sec); ASSERT_EQ(io_config.fg_write_weight, 1); ASSERT_EQ(io_config.bg_write_weight, 2); @@ -705,6 +713,7 @@ background_read_weight=2 ASSERT_EQ(io_config.max_bytes_per_sec, 0); // ignored ASSERT_EQ(io_config.max_read_bytes_per_sec, 1024000); ASSERT_EQ(io_config.max_write_bytes_per_sec, 1024000); + ASSERT_EQ(io_config.s3_max_read_bytes_per_sec, 1024); ASSERT_FALSE(io_config.use_max_bytes_per_sec); // use max_read_bytes_per_sec and max_write_bytes_per_sec ASSERT_EQ(io_config.fg_write_weight, 1); ASSERT_EQ(io_config.bg_write_weight, 2); @@ -725,6 +734,7 @@ background_read_weight=2 ASSERT_EQ(io_config.max_bytes_per_sec, 1024000); // ignored ASSERT_EQ(io_config.max_read_bytes_per_sec, 1024000); ASSERT_EQ(io_config.max_write_bytes_per_sec, 1024000); + ASSERT_EQ(io_config.s3_max_read_bytes_per_sec, 4096); ASSERT_FALSE(io_config.use_max_bytes_per_sec); // use max_read_bytes_per_sec and max_write_bytes_per_sec ASSERT_EQ(io_config.fg_write_weight, 1); ASSERT_EQ(io_config.bg_write_weight, 2); @@ -745,6 +755,7 @@ background_read_weight=2 ASSERT_EQ(io_config.max_bytes_per_sec, 1024000); ASSERT_EQ(io_config.max_read_bytes_per_sec, 0); ASSERT_EQ(io_config.max_write_bytes_per_sec, 0); + ASSERT_EQ(io_config.s3_max_read_bytes_per_sec, 8192); ASSERT_TRUE(io_config.use_max_bytes_per_sec); ASSERT_EQ(io_config.fg_write_weight, 80); ASSERT_EQ(io_config.bg_write_weight, 20); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp index 91f18c8836a..62ae9a3d34a 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp @@ -544,9 +544,31 @@ ColumnPtr DMFileReader::readFromDisk( [&](const IDataType::SubstreamPath & substream_path) { const auto substream_name = DMFile::getFileNameBase(cd.id, substream_path); auto & sub_stream = column_streams.at(substream_name); - sub_stream->buf->seek( - sub_stream->getOffsetInFile(start_pack_id), - sub_stream->getOffsetInDecompressedBlock(start_pack_id)); + const auto offset_in_file = sub_stream->getOffsetInFile(start_pack_id); + const auto offset_in_decompressed_block = sub_stream->getOffsetInDecompressedBlock(start_pack_id); + try + { + sub_stream->buf->seek(offset_in_file, offset_in_decompressed_block); + } + catch (...) + { + tryLogCurrentWarningException( + log, + fmt::format( + "DMFile substream seek failed, dmfile={} column_id={} type_on_disk={} stream_name={} " + "substream_name={} start_pack_id={} read_rows={} offset_in_file={} " + "offset_in_decompressed_block={}", + path(), + cd.id, + type_on_disk->getName(), + stream_name, + substream_name, + start_pack_id, + read_rows, + offset_in_file, + offset_in_decompressed_block)); + throw; + } return sub_stream->buf.get(); }, read_rows, diff --git a/dbms/src/Storages/KVStore/FFI/ProxyFFIStatusService.cpp b/dbms/src/Storages/KVStore/FFI/ProxyFFIStatusService.cpp index 5817d2820dd..e736f7da3bc 100644 --- a/dbms/src/Storages/KVStore/FFI/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/KVStore/FFI/ProxyFFIStatusService.cpp @@ -16,6 +16,8 @@ #include #include #include +#include +#include #include #include #include @@ -36,6 +38,7 @@ #include #include #include +#include #include namespace DB @@ -579,6 +582,88 @@ RemoteCacheEvictRequest parseEvictRequest( return req; } +std::optional parseCacheEvictType(std::string_view path, std::string_view api_name, String & err_msg) +{ + auto trim_path = path.substr(api_name.size()); + if (trim_path == "/mark") + return CacheEvictType::Mark; + if (trim_path == "/minmax") + return CacheEvictType::MinMax; + + err_msg = fmt::format("invalid cache evict request: {}", path); + return std::nullopt; +} + +namespace +{ +String buildCacheEvictOkBody(std::string_view cache_name, std::optional message = std::nullopt) +{ + if (message.has_value()) + return fmt::format(R"json({{"status":"ok","cache":"{}","message":"{}"}})json", cache_name, *message); + return fmt::format(R"json({{"status":"ok","cache":"{}"}})json", cache_name); +} + +String buildJsonErrorBody(const String & err_msg) +{ + WriteBufferFromOwnString msg_buf; + writeJSONString(err_msg, msg_buf); + return fmt::format(R"json({{"status":"error","message":{}}})json", msg_buf.releaseStr()); +} + +bool evictLocalCacheAndReport(Context & global_ctx, CacheEvictType cache_type) +{ + switch (cache_type) + { + case CacheEvictType::Mark: + return global_ctx.dropMarkCacheAndReport(); + case CacheEvictType::MinMax: + return global_ctx.dropMinMaxIndexCacheAndReport(); + } + __builtin_unreachable(); +} + +std::string_view cacheTypeName(CacheEvictType cache_type) +{ + switch (cache_type) + { + case CacheEvictType::Mark: + return "mark"; + case CacheEvictType::MinMax: + return "minmax"; + } + __builtin_unreachable(); +} +} // namespace + +HttpRequestRes HandleHttpRequestLocalCacheEvict( + EngineStoreServerWrap * server, + std::string_view path, + const std::string & api_name, + std::string_view, + std::string_view) +{ + auto & global_ctx = server->tmt->getContext(); + auto log = Logger::get("HandleHttpRequestLocalCacheEvict"); + + String err_msg; + auto cache_type = parseCacheEvictType(path, api_name, err_msg); + if (!cache_type.has_value()) + { + auto body = buildJsonErrorBody(err_msg); + LOG_WARNING(log, "invalid local cache evict request, path={} api_name={}", path, api_name); + return buildRespWithCode(HttpRequestStatus::BadRequest, api_name, std::move(body)); + } + + const auto cache_name = cacheTypeName(*cache_type); + // `drop*Cache()` eventually calls `LRUCache::reset()`, which clears the registry under the + // cache mutex while leaving already-held `shared_ptr` values valid for in-flight readers. + const bool cache_enabled = evictLocalCacheAndReport(global_ctx, *cache_type); + LOG_INFO(log, "manual cache eviction, action=evict cache={} result={}", cache_name, cache_enabled ? "ok" : "noop"); + return buildOkResp( + api_name, + cache_enabled ? buildCacheEvictOkBody(cache_name) : buildCacheEvictOkBody(cache_name, "cache not enabled")); +} + HttpRequestRes HandleHttpRequestRemoteCacheEvict( EngineStoreServerWrap * server, std::string_view path, @@ -817,6 +902,7 @@ using HANDLE_HTTP_URI_METHOD = HttpRequestRes (*)( std::string_view); // A registry of available HTTP URI prefix (API name) and their handler methods. +// Keep `docs/tiflash_http_api.md` in sync whenever adding or changing a public HTTP API here. static const std::map AVAILABLE_HTTP_URI = { {"/tiflash/sync-status/", HandleHttpRequestSyncStatus}, {"/tiflash/sync-region/", HandleHttpRequestSyncRegion}, @@ -836,6 +922,7 @@ static const std::map AVAILABLE_HTTP_URI = {"/tiflash/remote/gc", HandleHttpRequestRemoteGC}, {"/tiflash/remote/upload", HandleHttpRequestRemoteReUpload}, {"/tiflash/remote/info", HandleHttpRequestRemoteInfo}, + {"/tiflash/cache/evict", HandleHttpRequestLocalCacheEvict}, {"/tiflash/remote/cache/evict", HandleHttpRequestRemoteCacheEvict}, {"/tiflash/remote/cache/info", HandleHttpRequestRemoteCacheInfo}, }; diff --git a/dbms/src/Storages/KVStore/FFI/ProxyFFIStatusService.h b/dbms/src/Storages/KVStore/FFI/ProxyFFIStatusService.h index 72c00a846dc..73891c5fb72 100644 --- a/dbms/src/Storages/KVStore/FFI/ProxyFFIStatusService.h +++ b/dbms/src/Storages/KVStore/FFI/ProxyFFIStatusService.h @@ -16,6 +16,8 @@ #include +#include + namespace DB { @@ -28,6 +30,12 @@ enum class EvictMethod ByEvictSize, }; +enum class CacheEvictType +{ + Mark = 0, + MinMax, +}; + struct RemoteCacheEvictRequest { EvictMethod evict_method; @@ -40,6 +48,10 @@ struct RemoteCacheEvictRequest RemoteCacheEvictRequest parseEvictRequest(std::string_view path, std::string_view api_name, std::string_view query); +/// Parse `/tiflash/cache/evict/` and resolve the target node-local cache type. +/// Returns `std::nullopt` and sets `err_msg` when the path suffix is invalid. +std::optional parseCacheEvictType(std::string_view path, std::string_view api_name, String & err_msg); + std::tuple, String> parseStoreIds(std::string_view path); } // namespace DB diff --git a/dbms/src/Storages/KVStore/FFI/tests/gtest_status_server.cpp b/dbms/src/Storages/KVStore/FFI/tests/gtest_status_server.cpp index aaff2584290..588a31a0a1b 100644 --- a/dbms/src/Storages/KVStore/FFI/tests/gtest_status_server.cpp +++ b/dbms/src/Storages/KVStore/FFI/tests/gtest_status_server.cpp @@ -13,9 +13,19 @@ // limitations under the License. #include +#include +#include +#include #include #include #include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -78,6 +88,13 @@ class StatusServerTest : public ::testing::Test p = path + "/data/"; TiFlashTestEnv::tryRemovePath(p, /*recreate=*/true); } + + static void releaseResp(const EngineStoreServerHelper & helper, HttpRequestRes && res) + { + // release all raw cpp ptr in HttpRequestRes + helper.fn_gc_raw_cpp_ptr(res.res.inner.ptr, res.res.inner.type); + helper.fn_gc_raw_cpp_ptr(res.api_name.inner.ptr, res.api_name.inner.type); + } }; TEST_F(StatusServerTest, TestSyncSchema) @@ -383,4 +400,79 @@ TEST_F(StatusServerTest, TestParseRemoteCacheEvictRequest) } } +TEST_F(StatusServerTest, TestParseLocalCacheEvictType) +{ + String err_msg; + + auto cache_type = parseCacheEvictType("/tiflash/cache/evict/mark", "/tiflash/cache/evict", err_msg); + ASSERT_TRUE(cache_type.has_value()); + ASSERT_EQ(*cache_type, CacheEvictType::Mark); + ASSERT_TRUE(err_msg.empty()); + + err_msg.clear(); + cache_type = parseCacheEvictType("/tiflash/cache/evict/minmax", "/tiflash/cache/evict", err_msg); + ASSERT_TRUE(cache_type.has_value()); + ASSERT_EQ(*cache_type, CacheEvictType::MinMax); + ASSERT_TRUE(err_msg.empty()); + + err_msg.clear(); + cache_type = parseCacheEvictType("/tiflash/cache/evict/unknown", "/tiflash/cache/evict", err_msg); + ASSERT_FALSE(cache_type.has_value()); + ASSERT_FALSE(err_msg.empty()); +} + +TEST_F(StatusServerTest, TestLocalCacheEvict) +{ + auto ctx = TiFlashTestEnv::getContext(); + EngineStoreServerWrap store_server_wrap{}; + store_server_wrap.tmt = &ctx->getTMTContext(); + auto helper = GetEngineStoreServerHelper(&store_server_wrap); + + if (!ctx->getMarkCache()) + ctx->setMarkCache(1024 * 1024); + if (!ctx->getMinMaxIndexCache()) + ctx->setMinMaxIndexCache(1024 * 1024); + + { + auto mark_cache = ctx->getMarkCache(); + ASSERT_NE(mark_cache, nullptr); + mark_cache->reset(); + auto marks = std::make_shared(); + marks->push_back(MarkInCompressedFile{1, 2}); + mark_cache->set("mark-key", marks); + ASSERT_EQ(mark_cache->count(), 1); + + String path = "/tiflash/cache/evict/mark"; + auto res = helper.fn_handle_http_request( + &store_server_wrap, + BaseBuffView{path.data(), path.length()}, + BaseBuffView{"", 0}, + BaseBuffView{"", 0}); + EXPECT_EQ(res.status, HttpRequestStatus::Ok); + EXPECT_EQ(std::string_view(res.res.view.data, res.res.view.len), R"json({"status":"ok","cache":"mark"})json"); + EXPECT_EQ(mark_cache->count(), 0); + releaseResp(helper, std::move(res)); + } + + { + auto minmax_cache = ctx->getMinMaxIndexCache(); + ASSERT_NE(minmax_cache, nullptr); + minmax_cache->reset(); + auto index = std::make_shared(DataTypeInt64()); + minmax_cache->set("minmax-key", index); + ASSERT_EQ(minmax_cache->count(), 1); + + String path = "/tiflash/cache/evict/minmax"; + auto res = helper.fn_handle_http_request( + &store_server_wrap, + BaseBuffView{path.data(), path.length()}, + BaseBuffView{"", 0}, + BaseBuffView{"", 0}); + EXPECT_EQ(res.status, HttpRequestStatus::Ok); + EXPECT_EQ(std::string_view(res.res.view.data, res.res.view.len), R"json({"status":"ok","cache":"minmax"})json"); + EXPECT_EQ(minmax_cache->count(), 0); + releaseResp(helper, std::move(res)); + } +} + } // namespace DB::tests diff --git a/dbms/src/Storages/S3/FileCache.cpp b/dbms/src/Storages/S3/FileCache.cpp index 3f953639d8f..a29740b9711 100644 --- a/dbms/src/Storages/S3/FileCache.cpp +++ b/dbms/src/Storages/S3/FileCache.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -75,42 +76,264 @@ extern const int FILE_DOESNT_EXIST; namespace DB::FailPoints { extern const char file_cache_fg_download_fail[]; +extern const char file_cache_bg_download_fail[]; +extern const char file_cache_bg_download_schedule_fail[]; } // namespace DB::FailPoints namespace DB { using FileType = FileSegment::FileType; +namespace +{ +// A tiny FileCache-only ReadBuffer variant that charges the shared S3 limiter before each refill. +// This lets downloadToLocal keep using the existing copyData/write-buffer path instead of maintaining +// a separate hand-written read/write loop for limiter-enabled downloads. +class ReadBufferFromIStreamWithLimiter : public BufferWithOwnMemory +{ +public: + ReadBufferFromIStreamWithLimiter( + std::istream & istr_, + size_t size, + const std::shared_ptr & limiter_, + S3::S3ReadSource source_) + : BufferWithOwnMemory(size) + , istr(istr_) + , limiter(limiter_) + , source(source_) + {} + +private: + bool nextImpl() override + { + if (limiter != nullptr) + { + // Charge the requested refill size before the actual `istream.read()`. This is intentionally + // conservative: short reads still spend the full reserved budget for this refill. If we need tighter + // accounting later, we can extend this path to compensate with the actual bytes read back from S3. + limiter->requestBytes(internal_buffer.size(), source); + } + + istr.read(internal_buffer.begin(), internal_buffer.size()); + auto gcount = istr.gcount(); + if (!gcount) + { + if (istr.eof()) + return false; + throw Exception(ErrorCodes::CANNOT_READ_FROM_ISTREAM, "Cannot read from istream"); + } + + working_buffer.resize(gcount); + return true; + } + + std::istream & istr; + std::shared_ptr limiter; + S3::S3ReadSource source; +}; + +enum class WaitResult +{ + Hit, + Timeout, + Failed, +}; + +enum class BgDownloadStage +{ + QueueWait, + Download, +}; + +TiFlashMetrics::RemoteCacheFileTypeMetric toMetricFileType(FileType file_type) +{ + switch (file_type) + { + case FileType::Merged: + return TiFlashMetrics::RemoteCacheFileTypeMetric::Merged; + case FileType::DeleteMarkColData: + case FileType::VersionColData: + case FileType::HandleColData: + case FileType::ColData: + return TiFlashMetrics::RemoteCacheFileTypeMetric::ColData; + default: + return TiFlashMetrics::RemoteCacheFileTypeMetric::Other; + } +} + +void observeWaitOnDownloadingMetrics(FileType file_type, WaitResult result, UInt64 bytes, double wait_seconds) +{ + GET_METRIC(tiflash_storage_remote_cache, type_wait_on_downloading).Increment(); + auto & metrics = TiFlashMetrics::instance(); + auto metric_file_type = toMetricFileType(file_type); + switch (result) + { + case WaitResult::Hit: + metrics + .getRemoteCacheWaitOnDownloadingResultCounter( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Hit) + .Increment(); + metrics + .getRemoteCacheWaitOnDownloadingSecondsHistogram( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Hit) + .Observe(wait_seconds); + metrics + .getRemoteCacheWaitOnDownloadingBytesCounter( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Hit) + .Increment(bytes); + break; + case WaitResult::Timeout: + metrics + .getRemoteCacheWaitOnDownloadingResultCounter( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Timeout) + .Increment(); + metrics + .getRemoteCacheWaitOnDownloadingSecondsHistogram( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Timeout) + .Observe(wait_seconds); + metrics + .getRemoteCacheWaitOnDownloadingBytesCounter( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Timeout) + .Increment(bytes); + break; + case WaitResult::Failed: + metrics + .getRemoteCacheWaitOnDownloadingResultCounter( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Failed) + .Increment(); + metrics + .getRemoteCacheWaitOnDownloadingSecondsHistogram( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Failed) + .Observe(wait_seconds); + metrics + .getRemoteCacheWaitOnDownloadingBytesCounter( + metric_file_type, + TiFlashMetrics::RemoteCacheWaitResultMetric::Failed) + .Increment(bytes); + break; + } + + switch (result) + { + case WaitResult::Hit: + GET_METRIC(tiflash_storage_remote_cache, type_wait_on_downloading_hit).Increment(); + break; + case WaitResult::Timeout: + GET_METRIC(tiflash_storage_remote_cache, type_wait_on_downloading_timeout).Increment(); + break; + case WaitResult::Failed: + GET_METRIC(tiflash_storage_remote_cache, type_wait_on_downloading_failed).Increment(); + break; + } +} + +void observeRemoteCacheRejectMetrics(FileType file_type) +{ + TiFlashMetrics::instance() + .getRemoteCacheRejectCounter( + toMetricFileType(file_type), + TiFlashMetrics::RemoteCacheRejectReasonMetric::TooManyDownload) + .Increment(); +} + +void updateBgDownloadStatusMetrics(Int64 bg_downloading_count, Int64 running_limit) +{ + GET_METRIC(tiflash_storage_remote_cache_status, type_bg_downloading_count).Set(bg_downloading_count); + GET_METRIC(tiflash_storage_remote_cache_status, type_bg_download_queue_count) + .Set(std::max(0, bg_downloading_count - running_limit)); +} + +void observeBgDownloadStageMetrics(FileType file_type, BgDownloadStage stage, double seconds) +{ + auto & metrics = TiFlashMetrics::instance(); + auto metric_file_type = toMetricFileType(file_type); + switch (stage) + { + case BgDownloadStage::QueueWait: + metrics + .getRemoteCacheBgDownloadStageSecondsHistogram( + metric_file_type, + TiFlashMetrics::RemoteCacheDownloadStageMetric::QueueWait) + .Observe(seconds); + break; + case BgDownloadStage::Download: + metrics + .getRemoteCacheBgDownloadStageSecondsHistogram( + metric_file_type, + TiFlashMetrics::RemoteCacheDownloadStageMetric::Download) + .Observe(seconds); + break; + } +} + +} // namespace + std::unique_ptr FileCache::global_file_cache_instance; FileSegment::Status FileSegment::waitForNotEmpty() { + // Foreground callers expect the file to become readable eventually. This path keeps logging + // slow waits and fails hard after the built-in timeout instead of silently returning `Empty`. + return waitForNotEmptyImpl(std::nullopt, /*log_progress*/ true, /*throw_on_timeout*/ true); +} + +FileSegment::Status FileSegment::waitForNotEmptyFor(std::chrono::milliseconds timeout) +{ + // Bounded-wait callers treat timeout as a normal outcome and will fall back to another path, + // so this variant waits only once for the specified budget and returns the current status. + return waitForNotEmptyImpl(timeout, /*log_progress*/ false, /*throw_on_timeout*/ false); +} + +FileSegment::Status FileSegment::waitForNotEmptyImpl( + std::optional timeout, + bool log_progress, + bool throw_on_timeout) +{ + constexpr UInt64 default_wait_log_interval_seconds = 30; + constexpr UInt64 wait_ready_timeout_seconds = 300; + std::unique_lock lock(mtx); if (status != Status::Empty) return status; - PerfContext::file_cache.fg_wait_download_from_s3++; + if (log_progress) + PerfContext::file_cache.fg_wait_download_from_s3++; Stopwatch watch; while (true) { + auto wait_interval = timeout.value_or(std::chrono::seconds(default_wait_log_interval_seconds)); SYNC_FOR("before_FileSegment::waitForNotEmpty_wait"); // just before actual waiting... - auto is_done = cv_ready.wait_for(lock, std::chrono::seconds(30), [&] { return status != Status::Empty; }); + auto is_done = cv_ready.wait_for(lock, wait_interval, [&] { return status != Status::Empty; }); if (is_done) break; + if (timeout.has_value()) + break; + double elapsed_secs = watch.elapsedSeconds(); - LOG_WARNING( - Logger::get(), - "FileCache is still waiting FileSegment ready, file={} elapsed={}s", - local_fname, - elapsed_secs); + if (log_progress) + { + LOG_WARNING( + Logger::get(), + "FileCache is still waiting FileSegment ready, file={} elapsed={}s", + local_fname, + elapsed_secs); + } // Snapshot time is 300s - if (elapsed_secs > 300) + if (throw_on_timeout && elapsed_secs > wait_ready_timeout_seconds) { throw Exception( ErrorCodes::S3_ERROR, @@ -245,6 +468,7 @@ FileCache::FileCache( , log(Logger::get("FileCache")) { CurrentMetrics::set(CurrentMetrics::DTFileCacheCapacity, cache_capacity); + updateBgDownloadStatusMetrics(0, /*running_limit*/ 0); prepareDir(cache_dir); restore(); } @@ -346,6 +570,7 @@ FileSegmentPtr FileCache::get(const S3::S3FilenameView & s3_fname, const std::op auto & table = tables[static_cast(file_type)]; FileSegmentPtr file_seg; + UInt64 wait_ms = 0; { std::unique_lock lock(mtx); if (auto f = table.get(s3_key); f != nullptr) @@ -353,55 +578,93 @@ FileSegmentPtr FileCache::get(const S3::S3FilenameView & s3_fname, const std::op f->setLastAccessTime(std::chrono::system_clock::now()); if (f->isReadyToRead()) { + // Hot-cache fast path: the file is already materialized locally, so return the existing segment + // immediately without touching any download scheduling or bounded-wait logic. GET_METRIC(tiflash_storage_remote_cache, type_dtfile_hit).Increment(); return f; } - else + + // Another thread is already downloading the same object. Optionally wait for a bounded time and + // reuse that result instead of opening one more `GetObject` stream for the same key. + wait_ms = wait_on_downloading_ms.load(std::memory_order_relaxed); + if (wait_ms == 0) { GET_METRIC(tiflash_storage_remote_cache, type_dtfile_miss).Increment(); return nullptr; } + file_seg = f; } - - GET_METRIC(tiflash_storage_remote_cache, type_dtfile_miss).Increment(); - switch (canCache(file_type)) + else { - case ShouldCacheRes::RejectTypeNotMatch: - GET_METRIC(tiflash_storage_remote_cache, type_dtfile_not_cache_type).Increment(); - return nullptr; - case ShouldCacheRes::RejectTooManyDownloading: - GET_METRIC(tiflash_storage_remote_cache, type_dtfile_too_many_download).Increment(); - return nullptr; - case ShouldCacheRes::Cache: - break; - } + GET_METRIC(tiflash_storage_remote_cache, type_dtfile_miss).Increment(); + // Admission control before any reservation work: skip file types that should never enter FileCache, + // and stop creating new `Empty` placeholders once background downloading is already saturated. + switch (canCache(file_type)) + { + case ShouldCacheRes::RejectTypeNotMatch: + GET_METRIC(tiflash_storage_remote_cache, type_dtfile_not_cache_type).Increment(); + return nullptr; + case ShouldCacheRes::RejectTooManyDownloading: + GET_METRIC(tiflash_storage_remote_cache, type_dtfile_too_many_download).Increment(); + observeRemoteCacheRejectMetrics(file_type); + return nullptr; + case ShouldCacheRes::Cache: + break; + } - // File not exists, try to download and cache it in background. + // File not exists, try to download and cache it in background. - // We don't know the exact size of a object/file, but we need reserve space to save the object/file. - // A certain amount of space is reserved for each file type. - auto estimated_size = filesize ? *filesize : getEstimatedSizeOfFileType(file_type); - if (!reserveSpaceImpl(file_type, estimated_size, EvictMode::TryEvict, lock)) + // We don't know the exact size of a object/file, but we need reserve space to save the object/file. + // A certain amount of space is reserved for each file type. + auto estimated_size = filesize ? *filesize : getEstimatedSizeOfFileType(file_type); + if (!reserveSpaceImpl(file_type, estimated_size, EvictMode::TryEvict, lock)) + { + // Space still not enough after eviction. + GET_METRIC(tiflash_storage_remote_cache, type_dtfile_full).Increment(); + LOG_DEBUG( + log, + "s3_key={} space not enough(capacity={} used={} estimated_size={}), skip cache", + s3_key, + cache_capacity, + cache_used, + estimated_size); + return nullptr; + } + + file_seg = std::make_shared( + toLocalFilename(s3_key), + FileSegment::Status::Empty, + estimated_size, + file_type); + table.set(s3_key, file_seg); + } + } // Release the lock before submitting bg download task. Because bgDownload may be blocked when the queue is full. + + if (wait_ms != 0) + { + // Follower path: another thread already inserted the `Empty` segment and is downloading this key. + // Wait only for the configured bounded budget, then either reuse the completed file or return miss + // so the caller can fall back without opening a duplicate download stream for the same object. + Stopwatch wait_watch; + auto status = file_seg->waitForNotEmptyFor(std::chrono::milliseconds(wait_ms)); + const auto waited_bytes = filesize.value_or(file_seg->getSize()); + if (status == FileSegment::Status::Complete) { - // Space still not enough after eviction. - GET_METRIC(tiflash_storage_remote_cache, type_dtfile_full).Increment(); - LOG_DEBUG( - log, - "s3_key={} space not enough(capacity={} used={} estimated_size={}), skip cache", - s3_key, - cache_capacity, - cache_used, - estimated_size); - return nullptr; + observeWaitOnDownloadingMetrics(file_type, WaitResult::Hit, waited_bytes, wait_watch.elapsedSeconds()); + GET_METRIC(tiflash_storage_remote_cache, type_dtfile_hit).Increment(); + return file_seg; } - file_seg = std::make_shared( - toLocalFilename(s3_key), - FileSegment::Status::Empty, - estimated_size, - file_type); - table.set(s3_key, file_seg); - } // Release the lock before submiting bg download task. Because bgDownload may be blocked when the queue is full. + observeWaitOnDownloadingMetrics( + file_type, + status == FileSegment::Status::Failed ? WaitResult::Failed : WaitResult::Timeout, + waited_bytes, + wait_watch.elapsedSeconds()); + // Timeout is intentionally surfaced as a cache miss here. The caller can fall back to another read path, + // while the original downloader keeps making progress in background instead of being duplicated by followers. + GET_METRIC(tiflash_storage_remote_cache, type_dtfile_miss).Increment(); + return nullptr; + } bgDownload(s3_key, file_seg); @@ -516,7 +779,8 @@ std::pair::iterator> FileCache::removeImpl( LRUFileTable & table, const String & s3_key, FileSegmentPtr & f, - bool force) + bool force, + bool count_as_evict) { // Except current thread and the FileTable, // there are other threads hold this FileSegment object. @@ -531,8 +795,11 @@ std::pair::iterator> FileCache::removeImpl( removeDiskFile(temp_fname, /*update_fsize_metrics*/ false); auto release_size = f->getSize(); - GET_METRIC(tiflash_storage_remote_cache, type_dtfile_evict).Increment(); - GET_METRIC(tiflash_storage_remote_cache_bytes, type_dtfile_evict_bytes).Increment(release_size); + if (count_as_evict) + { + GET_METRIC(tiflash_storage_remote_cache, type_dtfile_evict).Increment(); + GET_METRIC(tiflash_storage_remote_cache_bytes, type_dtfile_evict_bytes).Increment(release_size); + } releaseSpaceImpl(release_size); return {release_size, table.remove(s3_key)}; } @@ -990,7 +1257,9 @@ void downloadToLocal( Aws::IOStream & istr, const String & fname, Int64 content_length, - const WriteLimiterPtr & write_limiter) + const WriteLimiterPtr & write_limiter, + const std::shared_ptr & s3_read_limiter, + const std::shared_ptr & s3_read_metrics_recorder) { // create an empty file with write_limiter // each time `ofile.write` is called, the write speed will be controlled by the write_limiter. @@ -1000,9 +1269,35 @@ void downloadToLocal( return; GET_METRIC(tiflash_storage_remote_cache_bytes, type_dtfile_download_bytes).Increment(content_length); - static const Int64 MAX_BUFFER_SIZE = 128 * 1024; // 128k - ReadBufferFromIStream rbuf(istr, std::min(content_length, MAX_BUFFER_SIZE)); - WriteBufferFromWritableFile wbuf(ofile, std::min(content_length, MAX_BUFFER_SIZE)); + constexpr Int64 max_buffer_size = 128 * 1024; // 128 KiB + auto buffer_size = std::min(content_length, max_buffer_size); + if (s3_read_limiter == nullptr || s3_read_limiter->maxReadBytesPerSec() == 0) + { + ReadBufferFromIStream rbuf(istr, buffer_size); + WriteBufferFromWritableFile wbuf(ofile, buffer_size); + SCOPE_EXIT({ + if (s3_read_metrics_recorder != nullptr) + s3_read_metrics_recorder->recordBytes(rbuf.count(), S3::S3ReadSource::FileCacheDownload); + }); + copyData(rbuf, wbuf, content_length); + wbuf.sync(); + return; + } + + // Keep each refill within the limiter-suggested chunk size. Otherwise a low byte limit would + // turn every 128 KiB refill into an oversized borrowing request and let downloads run ahead + // of the configured node-level budget. + buffer_size = std::min( + buffer_size, + static_cast(s3_read_limiter->getSuggestedChunkSize(static_cast(buffer_size)))); + // The limiter-aware buffer preserves the old copyData/write-buffer path while charging the shared + // S3 budget before each refill from the remote body stream. + ReadBufferFromIStreamWithLimiter rbuf(istr, buffer_size, s3_read_limiter, S3::S3ReadSource::FileCacheDownload); + WriteBufferFromWritableFile wbuf(ofile, buffer_size); + SCOPE_EXIT({ + if (s3_read_metrics_recorder != nullptr) + s3_read_metrics_recorder->recordBytes(rbuf.count(), S3::S3ReadSource::FileCacheDownload); + }); copyData(rbuf, wbuf, content_length); wbuf.sync(); } @@ -1011,6 +1306,8 @@ void FileCache::downloadImpl(const String & s3_key, FileSegmentPtr & file_seg, c { Stopwatch sw; auto client = S3::ClientFactory::instance().sharedTiFlashClient(); + auto s3_read_limiter = client->getS3ReadLimiter(); + auto s3_read_metrics_recorder = client->getS3ReadMetricsRecorder(); Aws::S3::Model::GetObjectRequest req; client->setBucketAndKeyWithRoot(req, s3_key); ProfileEvents::increment(ProfileEvents::S3GetObject); @@ -1036,12 +1333,23 @@ void FileCache::downloadImpl(const String & s3_key, FileSegmentPtr & file_seg, c file_seg->setStatus(FileSegment::Status::Failed); return; } + // finalizeReservedSize() has already adjusted cache_used to the actual object size. Keep the + // segment size in sync before any later throw point so failed downloads release the correct + // reservation instead of the old estimated size. + file_seg->setSize(content_length); const auto & local_fname = file_seg->getLocalFileName(); // download as a temp file then rename to a formal file prepareParentDir(local_fname); auto temp_fname = toTemporaryFilename(local_fname); - downloadToLocal(result.GetBody(), temp_fname, content_length, write_limiter); + SYNC_FOR("before_FileCache::downloadImpl_download_to_local"); + downloadToLocal( + result.GetBody(), + temp_fname, + content_length, + write_limiter, + s3_read_limiter, + s3_read_metrics_recorder); std::filesystem::rename(temp_fname, local_fname); #ifndef NDEBUG @@ -1070,10 +1378,20 @@ void FileCache::downloadImpl(const String & s3_key, FileSegmentPtr & file_seg, c void FileCache::bgDownloadExecutor( const String & s3_key, FileSegmentPtr & file_seg, - const WriteLimiterPtr & write_limiter) + const WriteLimiterPtr & write_limiter, + std::chrono::steady_clock::time_point enqueue_time, + Int64 running_limit) { + observeBgDownloadStageMetrics( + file_seg->getFileType(), + BgDownloadStage::QueueWait, + std::chrono::duration_cast>(std::chrono::steady_clock::now() - enqueue_time) + .count()); + Stopwatch download_watch; try { + SYNC_FOR("before_FileCache::bgDownloadExecutor_fail_point"); + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::file_cache_bg_download_fail); GET_METRIC(tiflash_storage_remote_cache, type_dtfile_download).Increment(); downloadImpl(s3_key, file_seg, write_limiter); } @@ -1082,40 +1400,76 @@ void FileCache::bgDownloadExecutor( // ignore the exception here, and log as warning. tryLogCurrentWarningException(log, fmt::format("Download s3_key={} failed", s3_key)); } - + observeBgDownloadStageMetrics(file_seg->getFileType(), BgDownloadStage::Download, download_watch.elapsedSeconds()); if (!file_seg->isReadyToRead()) { - file_seg->setStatus(FileSegment::Status::Failed); GET_METRIC(tiflash_storage_remote_cache, type_dtfile_download_failed).Increment(); bg_download_fail_count.fetch_add(1, std::memory_order_relaxed); - file_seg.reset(); - remove(s3_key); + cleanupFailedDownload(s3_key, file_seg); } else { bg_download_succ_count.fetch_add(1, std::memory_order_relaxed); } - bg_downloading_count.fetch_sub(1, std::memory_order_relaxed); - LOG_DEBUG( - log, - "downloading count {} => s3_key {} finished", - bg_downloading_count.load(std::memory_order_relaxed), - s3_key); + finishBgDownload(s3_key, running_limit); } void FileCache::bgDownload(const String & s3_key, FileSegmentPtr & file_seg) { bg_downloading_count.fetch_add(1, std::memory_order_relaxed); + // Capture the pool concurrency limit before scheduling. Background workers still update + // queue gauges while finishing, but tests may shut the global S3FileCachePool down at the + // same time. Re-reading the singleton from the worker tail would race with shutdown. + const auto running_limit = static_cast(S3FileCachePool::get().getMaxThreads()); + updateBgDownloadStatusMetrics(bg_downloading_count.load(std::memory_order_relaxed), running_limit); LOG_DEBUG( log, "downloading count {} => s3_key {} start", bg_downloading_count.load(std::memory_order_relaxed), s3_key); auto write_limiter = rate_limiter.getBgWriteLimiter(); - S3FileCachePool::get().scheduleOrThrowOnError( - [this, s3_key = s3_key, file_seg = file_seg, limiter = std::move(write_limiter)]() mutable { - bgDownloadExecutor(s3_key, file_seg, limiter); - }); + auto enqueue_time = std::chrono::steady_clock::now(); + try + { + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::file_cache_bg_download_schedule_fail); + S3FileCachePool::get().scheduleOrThrowOnError( + [this, + s3_key = s3_key, + file_seg = file_seg, + limiter = std::move(write_limiter), + enqueue_time, + running_limit]() mutable { bgDownloadExecutor(s3_key, file_seg, limiter, enqueue_time, running_limit); }); + } + catch (...) + { + tryLogCurrentWarningException(log, fmt::format("Schedule background download s3_key={} failed", s3_key)); + GET_METRIC(tiflash_storage_remote_cache, type_dtfile_download_failed).Increment(); + bg_download_fail_count.fetch_add(1, std::memory_order_relaxed); + cleanupFailedDownload(s3_key, file_seg); + finishBgDownload(s3_key, running_limit); + } +} + +void FileCache::finishBgDownload(const String & s3_key, Int64 running_limit) +{ + const auto count_after_finish = bg_downloading_count.fetch_sub(1, std::memory_order_relaxed) - 1; + updateBgDownloadStatusMetrics(count_after_finish, running_limit); + LOG_DEBUG(log, "downloading count {} => s3_key {} finished", count_after_finish, s3_key); +} + +void FileCache::cleanupFailedDownload(const String & s3_key, FileSegmentPtr & file_seg) +{ + file_seg->setStatus(FileSegment::Status::Failed); + file_seg.reset(); + // Followers may still hold the failed segment while waking up from bounded wait. Force removal so + // the failed placeholder does not stay published in the cache table and block later retries. + // This is failed-download cleanup rather than cache eviction, so do not count eviction metrics. + auto file_type = getFileType(s3_key); + auto & table = tables[static_cast(file_type)]; + std::unique_lock lock(mtx); + auto f = table.get(s3_key, /*update_lru*/ false); + if (f != nullptr) + std::ignore = removeImpl(table, s3_key, f, /*force*/ true, /*count_as_evict*/ false); } void FileCache::fgDownload(const String & s3_key, FileSegmentPtr & file_seg) @@ -1139,7 +1493,12 @@ void FileCache::fgDownload(const String & s3_key, FileSegmentPtr & file_seg) file_seg->setStatus(FileSegment::Status::Failed); GET_METRIC(tiflash_storage_remote_cache, type_dtfile_download_failed).Increment(); file_seg.reset(); - remove(s3_key); + auto file_type = getFileType(s3_key); + auto & table = tables[static_cast(file_type)]; + std::unique_lock lock(mtx); + auto f = table.get(s3_key, /*update_lru*/ false); + if (f != nullptr) + std::ignore = removeImpl(table, s3_key, f, /*force*/ true, /*count_as_evict*/ false); } LOG_DEBUG(log, "foreground downloading => s3_key {} finished", s3_key); @@ -1374,6 +1733,17 @@ void FileCache::updateConfig(const Settings & settings) cache_min_age); cache_min_age_seconds.store(cache_min_age, std::memory_order_relaxed); } + + UInt64 new_wait_ms = settings.dt_filecache_wait_on_downloading_ms; + if (new_wait_ms != wait_on_downloading_ms.load(std::memory_order_relaxed)) + { + LOG_INFO( + log, + "Update S3FileCache bounded wait config: wait_on_downloading_ms {} => {}", + wait_on_downloading_ms.load(std::memory_order_relaxed), + new_wait_ms); + wait_on_downloading_ms.store(new_wait_ms, std::memory_order_relaxed); + } } // Evict the cached files until no file of >= `file_type` is in cache. diff --git a/dbms/src/Storages/S3/FileCache.h b/dbms/src/Storages/S3/FileCache.h index 6e1ef0fbb74..e64ba8f1ff9 100644 --- a/dbms/src/Storages/S3/FileCache.h +++ b/dbms/src/Storages/S3/FileCache.h @@ -36,6 +36,7 @@ #include #include #include +#include #include namespace DB @@ -88,6 +89,7 @@ class FileSegment } Status waitForNotEmpty(); + Status waitForNotEmptyFor(std::chrono::milliseconds timeout); void setComplete(UInt64 size_) { @@ -105,6 +107,14 @@ class FileSegment cv_ready.notify_all(); } + /// Update the reserved size without changing readiness state. This is used after reservation has + /// been rebased to the real object size but before the download finishes. + void setSize(UInt64 size_) + { + std::lock_guard lock(mtx); + size = size_; + } + Status getStatus() const { std::lock_guard lock(mtx); @@ -148,6 +158,11 @@ class FileSegment } private: + Status waitForNotEmptyImpl( + std::optional timeout, + bool log_progress, + bool throw_on_timeout); + mutable std::mutex mtx; const String local_fname; Status status; @@ -351,7 +366,14 @@ class FileCache void bgDownload(const String & s3_key, FileSegmentPtr & file_seg); void fgDownload(const String & s3_key, FileSegmentPtr & file_seg); - void bgDownloadExecutor(const String & s3_key, FileSegmentPtr & file_seg, const WriteLimiterPtr & write_limiter); + void bgDownloadExecutor( + const String & s3_key, + FileSegmentPtr & file_seg, + const WriteLimiterPtr & write_limiter, + std::chrono::steady_clock::time_point enqueue_time, + Int64 running_limit); + void finishBgDownload(const String & s3_key, Int64 running_limit); + void cleanupFailedDownload(const String & s3_key, FileSegmentPtr & file_seg); void downloadImpl(const String & s3_key, FileSegmentPtr & file_seg, const WriteLimiterPtr & write_limiter); static String toTemporaryFilename(const String & fname); @@ -372,7 +394,8 @@ class FileCache LRUFileTable & table, const String & s3_key, FileSegmentPtr & f, - bool force = false); + bool force = false, + bool count_as_evict = true); void removeDiskFile(const String & local_fname, bool update_fsize_metrics) const; // Estimated size is an empirical value. @@ -465,6 +488,7 @@ class FileCache const UInt16 logical_cores; IORateLimiter & rate_limiter; std::atomic cache_min_age_seconds = 1800; + std::atomic wait_on_downloading_ms = 0; std::atomic download_count_scale = 2.0; std::atomic max_downloading_count_scale = 10.0; // the on-going background download count diff --git a/dbms/src/Storages/S3/MockS3Client.h b/dbms/src/Storages/S3/MockS3Client.h index fcd4f17e67b..4a93a5b69fe 100644 --- a/dbms/src/Storages/S3/MockS3Client.h +++ b/dbms/src/Storages/S3/MockS3Client.h @@ -29,8 +29,18 @@ class MockS3Client final : public S3::TiFlashS3Client const String & bucket, const String & root, const Aws::Auth::AWSCredentials & cred, - const Aws::Client::ClientConfiguration & cfg) - : TiFlashS3Client(bucket, root, cred, cfg, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, false) + const Aws::Client::ClientConfiguration & cfg, + std::shared_ptr s3_read_limiter = nullptr, + std::shared_ptr s3_read_metrics_recorder = nullptr) + : TiFlashS3Client( + bucket, + root, + cred, + cfg, + Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, + false, + std::move(s3_read_limiter), + std::move(s3_read_metrics_recorder)) {} ~MockS3Client() override = default; diff --git a/dbms/src/Storages/S3/S3Common.cpp b/dbms/src/Storages/S3/S3Common.cpp index 419f9979c61..f6e84b42de9 100644 --- a/dbms/src/Storages/S3/S3Common.cpp +++ b/dbms/src/Storages/S3/S3Common.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -182,20 +183,28 @@ TiFlashS3Client::TiFlashS3Client( const Aws::Auth::AWSCredentials & credentials, const Aws::Client::ClientConfiguration & clientConfiguration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy signPayloads, - bool useVirtualAddressing) + bool useVirtualAddressing, + std::shared_ptr s3_read_limiter_, + std::shared_ptr s3_read_metrics_recorder_) : Aws::S3::S3Client(credentials, clientConfiguration, signPayloads, useVirtualAddressing) , bucket_name(bucket_name_) , key_root(normalizedRoot(root_)) + , s3_read_limiter(std::move(s3_read_limiter_)) + , s3_read_metrics_recorder(std::move(s3_read_metrics_recorder_)) , log(Logger::get(fmt::format("bucket={} root={}", bucket_name, key_root))) {} TiFlashS3Client::TiFlashS3Client( const String & bucket_name_, const String & root_, - std::unique_ptr && raw_client) + std::unique_ptr && raw_client, + std::shared_ptr s3_read_limiter_, + std::shared_ptr s3_read_metrics_recorder_) : Aws::S3::S3Client(std::move(*raw_client)) , bucket_name(bucket_name_) , key_root(normalizedRoot(root_)) + , s3_read_limiter(std::move(s3_read_limiter_)) + , s3_read_metrics_recorder(std::move(s3_read_metrics_recorder_)) , log(Logger::get(fmt::format("bucket={} root={}", bucket_name, key_root))) {} @@ -329,6 +338,8 @@ void ClientFactory::init(const StorageS3Config & config_, bool mock_s3_) return; config = config_; + if (shared_s3_read_metrics_recorder == nullptr) + shared_s3_read_metrics_recorder = std::make_shared(); RUNTIME_CHECK(!config.root.starts_with("//"), config.root); config.root = normalizedRoot(config.root); @@ -342,7 +353,12 @@ void ClientFactory::init(const StorageS3Config & config_, bool mock_s3_) { auto [s3_client, vendor] = create(config, log); cloud_vendor = vendor; - shared_tiflash_client = std::make_shared(config.bucket, config.root, std::move(s3_client)); + shared_tiflash_client = std::make_shared( + config.bucket, + config.root, + std::move(s3_client), + shared_s3_read_limiter, + shared_s3_read_metrics_recorder); } else { @@ -350,7 +366,13 @@ void ClientFactory::init(const StorageS3Config & config_, bool mock_s3_) Aws::Client::ClientConfiguration cfg(true, /*defaultMode=*/"standard", /*shouldDisableIMDS=*/true); cfg.region = Aws::Region::US_EAST_1; // default region Aws::Auth::AWSCredentials cred("mock_access_key", "mock_secret_key"); - shared_tiflash_client = std::make_unique(config.bucket, config.root, cred, cfg); + shared_tiflash_client = std::make_unique( + config.bucket, + config.root, + cred, + cfg, + shared_s3_read_limiter, + shared_s3_read_metrics_recorder); } client_is_inited = true; // init finish } @@ -372,6 +394,8 @@ std::shared_ptr ClientFactory::initClientFromWriteNode() assert(kv_cluster != nullptr); const auto disagg_config = getDisaggConfigFromDisaggWriteNodes(kv_cluster, log); + if (shared_s3_read_metrics_recorder == nullptr) + shared_s3_read_metrics_recorder = std::make_shared(); // update connection fields and leave other fields unchanged config.endpoint = disagg_config.s3_config().endpoint(); config.root = normalizedRoot(disagg_config.s3_config().root()); @@ -380,7 +404,12 @@ std::shared_ptr ClientFactory::initClientFromWriteNode() auto [s3_client, vendor] = create(config, log); cloud_vendor = vendor; - shared_tiflash_client = std::make_shared(config.bucket, config.root, std::move(s3_client)); + shared_tiflash_client = std::make_shared( + config.bucket, + config.root, + std::move(s3_client), + shared_s3_read_limiter, + shared_s3_read_metrics_recorder); client_is_inited = true; // init finish return shared_tiflash_client; } diff --git a/dbms/src/Storages/S3/S3Common.h b/dbms/src/Storages/S3/S3Common.h index 456d8c002c2..9f85bc9f928 100644 --- a/dbms/src/Storages/S3/S3Common.h +++ b/dbms/src/Storages/S3/S3Common.h @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -28,6 +29,8 @@ #include #include +#include +#include namespace pingcap::kv { @@ -41,7 +44,6 @@ extern const int S3_ERROR; namespace DB::S3 { - inline String S3ErrorMessage(const Aws::S3::S3Error & e) { return fmt::format( @@ -70,12 +72,16 @@ class TiFlashS3Client : public Aws::S3::S3Client const Aws::Auth::AWSCredentials & credentials, const Aws::Client::ClientConfiguration & clientConfiguration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy signPayloads, - bool useVirtualAddressing); + bool useVirtualAddressing, + std::shared_ptr s3_read_limiter_ = nullptr, + std::shared_ptr s3_read_metrics_recorder_ = nullptr); TiFlashS3Client( const String & bucket_name_, const String & root_, - std::unique_ptr && raw_client); + std::unique_ptr && raw_client, + std::shared_ptr s3_read_limiter_ = nullptr, + std::shared_ptr s3_read_metrics_recorder_ = nullptr); const String & bucket() const { return bucket_name; } @@ -89,9 +95,39 @@ class TiFlashS3Client : public Aws::S3::S3Client req.WithBucket(bucket_name).WithKey(is_root_single_slash ? key : key_root + key); } + /// Returns the shared node-level limiter for S3 remote reads. + std::shared_ptr getS3ReadLimiter() const + { + std::lock_guard lock(s3_read_limiter_mutex); + return s3_read_limiter; + } + + /// Publish a new node-level limiter to this client. Existing and future readers share the same object. + void setS3ReadLimiter(std::shared_ptr limiter) + { + std::lock_guard lock(s3_read_limiter_mutex); + s3_read_limiter = std::move(limiter); + } + + std::shared_ptr getS3ReadMetricsRecorder() const + { + std::lock_guard lock(s3_read_metrics_recorder_mutex); + return s3_read_metrics_recorder; + } + + void setS3ReadMetricsRecorder(std::shared_ptr recorder) + { + std::lock_guard lock(s3_read_metrics_recorder_mutex); + s3_read_metrics_recorder = std::move(recorder); + } + private: const String bucket_name; String key_root; + mutable std::mutex s3_read_limiter_mutex; + std::shared_ptr s3_read_limiter; + mutable std::mutex s3_read_metrics_recorder_mutex; + std::shared_ptr s3_read_metrics_recorder; public: LoggerPtr log; @@ -146,6 +182,22 @@ class ClientFactory std::shared_ptr sharedTiFlashClient(); + void setS3ReadLimiter(const std::shared_ptr & limiter) + { + std::unique_lock lock_init(mtx_init); + shared_s3_read_limiter = limiter; + if (shared_tiflash_client != nullptr) + shared_tiflash_client->setS3ReadLimiter(shared_s3_read_limiter); + } + + void setS3ReadMetricsRecorder(const std::shared_ptr & recorder) + { + std::unique_lock lock_init(mtx_init); + shared_s3_read_metrics_recorder = recorder; + if (shared_tiflash_client != nullptr) + shared_tiflash_client->setS3ReadMetricsRecorder(shared_s3_read_metrics_recorder); + } + S3GCMethod gc_method = S3GCMethod::Lifecycle; CloudVendor cloud_vendor = CloudVendor::Unknown; @@ -171,6 +223,8 @@ class ClientFactory mutable std::mutex mtx_init; // protect `config` `shared_tiflash_client` `kv_cluster` StorageS3Config config; std::shared_ptr shared_tiflash_client; + std::shared_ptr shared_s3_read_limiter; + std::shared_ptr shared_s3_read_metrics_recorder; pingcap::kv::Cluster * kv_cluster = nullptr; LoggerPtr log; diff --git a/dbms/src/Storages/S3/S3RandomAccessFile.cpp b/dbms/src/Storages/S3/S3RandomAccessFile.cpp index e9bfa3f8208..7cf48dc1537 100644 --- a/dbms/src/Storages/S3/S3RandomAccessFile.cpp +++ b/dbms/src/Storages/S3/S3RandomAccessFile.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -53,10 +54,16 @@ namespace DB::FailPoints { extern const char force_s3_random_access_file_init_fail[]; extern const char force_s3_random_access_file_read_fail[]; +extern const char force_s3_random_access_file_seek_chunked[]; } // namespace DB::FailPoints namespace DB::S3 { +namespace +{ +constexpr size_t s3_read_limiter_preferred_chunk_size = 128 * 1024; +} + String S3RandomAccessFile::summary() const { return fmt::format("remote_fname={} cur_offset={} cur_retry={}", remote_fname, cur_offset, cur_retry); @@ -69,10 +76,14 @@ S3RandomAccessFile::S3RandomAccessFile( : client_ptr(std::move(client_ptr_)) , remote_fname(remote_fname_) , cur_offset(0) + , read_limiter(nullptr) + , read_metrics_recorder(nullptr) , log(Logger::get(remote_fname)) , scan_context(scan_context_) { RUNTIME_CHECK(client_ptr != nullptr); + read_limiter = client_ptr->getS3ReadLimiter(); + read_metrics_recorder = client_ptr->getS3ReadMetricsRecorder(); initialize("init file"); CurrentMetrics::add(CurrentMetrics::S3RandomAccessFile); } @@ -120,20 +131,63 @@ ssize_t S3RandomAccessFile::read(char * buf, size_t size) ssize_t S3RandomAccessFile::readImpl(char * buf, size_t size) { + if (read_limiter != nullptr && read_limiter->maxReadBytesPerSec() > 0) + // Charge the shared node-level budget in small chunks instead of allowing a single large `read()` to burst. + return readChunked(buf, size); + Stopwatch sw; ProfileEvents::increment(ProfileEvents::S3IORead, 1); auto & istr = read_result.GetBody(); istr.read(buf, size); - size_t gcount = istr.gcount(); + return finalizeRead(size, istr.gcount(), sw, istr); +} + +ssize_t S3RandomAccessFile::readChunked(char * buf, size_t size) +{ + Stopwatch sw; + ProfileEvents::increment(ProfileEvents::S3IORead, 1); + + auto & istr = read_result.GetBody(); + // Use the limiter-suggested step so one large logical read is split into smoother refill-period- + // sized chunks. That keeps `requestBytes()` on its strict path for normal reads and only falls + // back to borrowing semantics for requests that are unavoidably larger than one burst. + const auto chunk_size = read_limiter->getSuggestedChunkSize(s3_read_limiter_preferred_chunk_size); + size_t total_gcount = 0; + while (total_gcount < size) + { + // The limiter charges requested bytes before the actual stream read so direct reads and FileCache downloads + // compete for the same node-level remote-read budget. This is intentionally conservative: a short read still + // spends the full requested budget for this chunk. If we need tighter accounting later, we can add a + // compensation path based on the actual bytes read back from S3. + auto to_read = std::min(size - total_gcount, static_cast(chunk_size)); + read_limiter->requestBytes(to_read, S3ReadSource::DirectRead); + istr.read(buf + total_gcount, to_read); + auto gcount = istr.gcount(); + total_gcount += gcount; + if (static_cast(gcount) < to_read) + break; + } + + return finalizeRead(size, total_gcount, sw, istr); +} +ssize_t S3RandomAccessFile::finalizeRead( + size_t requested_size, + size_t actual_size, + const Stopwatch & sw, + std::istream & istr) +{ + // Keep the post-read handling shared so limiter and non-limiter paths emit identical retries, logging and + // observability signals. fiu_do_on(FailPoints::force_s3_random_access_file_read_fail, { LOG_WARNING(log, "failpoint force_s3_random_access_file_read_fail is triggered, return S3StreamError"); return S3StreamError; }); - // Theoretically, `istr.eof()` is equivalent to `cur_offset + gcount != static_cast(content_length)`. + // Theoretically, `istr.eof()` is equivalent to `cur_offset + actual_size != static_cast(content_length)`. // It's just a double check for more safety. - if (gcount < size && (!istr.eof() || cur_offset + gcount != static_cast(content_length))) + if (actual_size < requested_size + && (!istr.eof() || cur_offset + actual_size != static_cast(content_length))) { ProfileEvents::increment(ProfileEvents::S3IOReadError); auto state = istr.rdstate(); @@ -143,8 +197,8 @@ ssize_t S3RandomAccessFile::readImpl(char * buf, size_t size) log, "Cannot read from istream, size={} gcount={} state=0x{:02X} cur_offset={} content_length={} " "errno={} errmsg={} cost={:.6f}s", - size, - gcount, + requested_size, + actual_size, state, cur_offset, content_length, @@ -159,22 +213,24 @@ ssize_t S3RandomAccessFile::readImpl(char * buf, size_t size) { scan_context->disagg_s3file_read_time_ms += elapsed_secs * 1000; scan_context->disagg_s3file_read_count += 1; - scan_context->disagg_s3file_read_bytes += gcount; + scan_context->disagg_s3file_read_bytes += actual_size; } GET_METRIC(tiflash_storage_s3_request_seconds, type_read_stream).Observe(elapsed_secs); - if (elapsed_secs > 0.01) // 10ms + if (elapsed_secs > 0.01) { LOG_DEBUG( log, "gcount={} cur_offset={} content_length={} cost={:.3f}s", - gcount, + actual_size, cur_offset, content_length, elapsed_secs); } - cur_offset += gcount; - ProfileEvents::increment(ProfileEvents::S3ReadBytes, gcount); - return gcount; + cur_offset += actual_size; + ProfileEvents::increment(ProfileEvents::S3ReadBytes, actual_size); + if (read_metrics_recorder != nullptr) + read_metrics_recorder->recordBytes(actual_size, S3ReadSource::DirectRead); + return actual_size; } off_t S3RandomAccessFile::seek(off_t offset_, int whence) @@ -210,18 +266,61 @@ off_t S3RandomAccessFile::seekImpl(off_t offset_, int whence) if (offset_ < cur_offset) { ProfileEvents::increment(ProfileEvents::S3IOSeekBackward, 1); - // Backward seek, need to reset the retry count and re-initialize + // The current body stream is forward-only. Re-open from the target offset. cur_offset = offset_; cur_retry = 0; initialize("seek backward"); return cur_offset; } + if (read_limiter != nullptr && read_limiter->maxReadBytesPerSec() > 0) + return seekChunked(offset_); + // Forward seek Stopwatch sw; ProfileEvents::increment(ProfileEvents::S3IOSeek, 1); auto & istr = read_result.GetBody(); - if (!istr.ignore(offset_ - cur_offset)) + auto bytes_to_ignore = static_cast(offset_ - cur_offset); + istr.ignore(bytes_to_ignore); + return finalizeSeek(offset_, bytes_to_ignore, istr.gcount(), sw, istr); +} + +off_t S3RandomAccessFile::seekChunked(off_t offset) +{ + Stopwatch sw; + ProfileEvents::increment(ProfileEvents::S3IOSeek, 1); + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::force_s3_random_access_file_seek_chunked); + auto & istr = read_result.GetBody(); + // Use the same chunk heuristic as readChunked() so forward seeks do not turn into one oversized + // limiter request when skipping a large remote range. + const auto chunk_size = read_limiter->getSuggestedChunkSize(s3_read_limiter_preferred_chunk_size); + size_t total_ignored = 0; + const auto bytes_to_ignore = static_cast(offset - cur_offset); + while (total_ignored < bytes_to_ignore) + { + // `ignore()` still drains the response body from S3, so it must be accounted against the same byte budget. + auto to_ignore = std::min(bytes_to_ignore - total_ignored, static_cast(chunk_size)); + read_limiter->requestBytes(to_ignore, S3ReadSource::DirectRead); + istr.ignore(to_ignore); + auto ignored = istr.gcount(); + total_ignored += ignored; + if (static_cast(ignored) < to_ignore) + break; + } + + return finalizeSeek(offset, bytes_to_ignore, total_ignored, sw, istr); +} + +off_t S3RandomAccessFile::finalizeSeek( + off_t target_offset, + size_t requested_size, + size_t actual_size, + const Stopwatch & sw, + std::istream & istr) +{ + // Keep post-seek handling shared so limiter and non-limiter paths emit identical retries, logging and + // observability signals. + if (actual_size < requested_size) { ProfileEvents::increment(ProfileEvents::S3IOSeekError); auto state = istr.rdstate(); @@ -229,8 +328,16 @@ off_t S3RandomAccessFile::seekImpl(off_t offset_, int whence) GET_METRIC(tiflash_storage_s3_request_seconds, type_read_stream_err).Observe(elapsed_secs); LOG_WARNING( log, - "Cannot ignore from istream, state=0x{:02X}, errno={} errmsg={} cost={:.6f}s", + "Cannot ignore from istream, state=0x{:02X}, ignored={} expected={} target_offset={} cur_offset={} " + "content_length={} limiter_enabled={} max_read_bytes_per_sec={} errno={} errmsg={} cost={:.6f}s", state, + actual_size, + requested_size, + target_offset, + cur_offset, + content_length, + read_limiter != nullptr, + read_limiter != nullptr ? read_limiter->maxReadBytesPerSec() : 0, errno, strerror(errno), elapsed_secs); @@ -242,21 +349,23 @@ off_t S3RandomAccessFile::seekImpl(off_t offset_, int whence) { scan_context->disagg_s3file_seek_time_ms += elapsed_secs * 1000; scan_context->disagg_s3file_seek_count += 1; - scan_context->disagg_s3file_seek_bytes += offset_ - cur_offset; + scan_context->disagg_s3file_seek_bytes += actual_size; } GET_METRIC(tiflash_storage_s3_request_seconds, type_read_stream).Observe(elapsed_secs); - if (elapsed_secs > 0.01) // 10ms + if (elapsed_secs > 0.01) { LOG_DEBUG( log, "ignore_count={} cur_offset={} content_length={} cost={:.3f}s", - offset_ - cur_offset, + actual_size, cur_offset, content_length, elapsed_secs); } - ProfileEvents::increment(ProfileEvents::S3ReadBytes, offset_ - cur_offset); - cur_offset = offset_; + ProfileEvents::increment(ProfileEvents::S3ReadBytes, actual_size); + if (read_metrics_recorder != nullptr) + read_metrics_recorder->recordBytes(actual_size, S3ReadSource::DirectRead); + cur_offset = target_offset; return cur_offset; } String S3RandomAccessFile::readRangeOfObject() diff --git a/dbms/src/Storages/S3/S3RandomAccessFile.h b/dbms/src/Storages/S3/S3RandomAccessFile.h index e700c7491dd..2a6d5244014 100644 --- a/dbms/src/Storages/S3/S3RandomAccessFile.h +++ b/dbms/src/Storages/S3/S3RandomAccessFile.h @@ -16,12 +16,15 @@ #include #include +#include #include #include +#include #include #include #include +#include /// Remove the population of thread_local from Poco #ifdef thread_local @@ -31,7 +34,7 @@ namespace DB::S3 { class TiFlashS3Client; -} +} // namespace DB::S3 namespace DB::ErrorCodes { @@ -93,6 +96,15 @@ class S3RandomAccessFile final : public RandomAccessFile off_t seekImpl(off_t offset, int whence); ssize_t readImpl(char * buf, size_t size); String readRangeOfObject(); + ssize_t readChunked(char * buf, size_t size); + ssize_t finalizeRead(size_t requested_size, size_t actual_size, const Stopwatch & sw, std::istream & istr); + off_t finalizeSeek( + off_t target_offset, + size_t requested_size, + size_t actual_size, + const Stopwatch & sw, + std::istream & istr); + off_t seekChunked(off_t offset); // When reading, it is necessary to pass the extra information of file, such file size, to S3RandomAccessFile::create. // It is troublesome to pass parameters layer by layer. So currently, use thread_local global variable to pass parameters. @@ -105,6 +117,8 @@ class S3RandomAccessFile final : public RandomAccessFile off_t cur_offset; Aws::S3::Model::GetObjectResult read_result; Int64 content_length = 0; + std::shared_ptr read_limiter; + std::shared_ptr read_metrics_recorder; DB::LoggerPtr log; bool is_close = false; diff --git a/dbms/src/Storages/S3/S3ReadLimiter.cpp b/dbms/src/Storages/S3/S3ReadLimiter.cpp new file mode 100644 index 00000000000..11ec2b5090c --- /dev/null +++ b/dbms/src/Storages/S3/S3ReadLimiter.cpp @@ -0,0 +1,188 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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 +#include +#include + +#include +#include + +namespace DB::S3 +{ +namespace +{ +// We only emit wait metrics after the call actually blocked, so the hot path keeps the zero-wait case cheap. +template +void recordWaitIfNeeded(bool waited, const Stopwatch & sw, F && observe) +{ + if (!waited) + return; + observe(sw.elapsedSeconds()); +} +} // namespace + +void DB::S3::S3ReadMetricsRecorder::recordBytes(UInt64 bytes, S3ReadSource source) +{ + if (bytes == 0) + return; + + switch (source) + { + case S3ReadSource::DirectRead: + GET_METRIC(tiflash_storage_io_limiter, type_s3_direct_read_bytes).Increment(bytes); + break; + case S3ReadSource::FileCacheDownload: + GET_METRIC(tiflash_storage_io_limiter, type_s3_filecache_download_bytes).Increment(bytes); + break; + } +} + +DB::S3::S3ReadLimiter::S3ReadLimiter(UInt64 max_read_bytes_per_sec_, UInt64 refill_period_ms_) + : refill_period_ms(refill_period_ms_) + , max_read_bytes_per_sec(max_read_bytes_per_sec_) + , available_bytes(static_cast(burstBytesPerPeriod(max_read_bytes_per_sec_))) + , last_refill_time(Clock::now()) + , stop(false) + , log(Logger::get("S3ReadLimiter")) +{ + GET_METRIC(tiflash_storage_io_limiter_curr, type_s3_read_bytes).Set(max_read_bytes_per_sec_); +} + +DB::S3::S3ReadLimiter::~S3ReadLimiter() +{ + setStop(); +} + +void DB::S3::S3ReadLimiter::updateConfig(UInt64 max_read_bytes_per_sec_) +{ + { + std::lock_guard lock(bytes_mutex); + max_read_bytes_per_sec.store(max_read_bytes_per_sec_, std::memory_order_relaxed); + available_bytes = std::min(available_bytes, static_cast(burstBytesPerPeriod(max_read_bytes_per_sec_))); + if (max_read_bytes_per_sec_ == 0) + available_bytes = 0; + last_refill_time = Clock::now(); + } + GET_METRIC(tiflash_storage_io_limiter_curr, type_s3_read_bytes).Set(max_read_bytes_per_sec_); + bytes_cv.notify_all(); +} + +void DB::S3::S3ReadLimiter::requestBytes(UInt64 bytes, S3ReadSource /*source*/) +{ + if (bytes == 0) + return; + + const auto limit = max_read_bytes_per_sec.load(std::memory_order_relaxed); + if (limit == 0) + return; + + Stopwatch sw; + bool waited = false; + std::unique_lock lock(bytes_mutex); + SCOPE_EXIT({ + recordWaitIfNeeded(waited, sw, [](double seconds) { + GET_METRIC(tiflash_storage_io_limiter_pending_seconds, type_s3_read_byte).Observe(seconds); + }); + }); + while (!stop) + { + const auto current_limit = max_read_bytes_per_sec.load(std::memory_order_relaxed); + // Config reload can disable the limiter while callers are waiting. + if (current_limit == 0) + return; + + const auto now = Clock::now(); + refillBytesLocked(now); + const auto requested_bytes = static_cast(bytes); + const auto burst_bytes = static_cast(burstBytesPerPeriod(current_limit)); + if (available_bytes >= requested_bytes) + { + available_bytes -= requested_bytes; + return; + } + + // Preserve the strict token-bucket behavior for requests that fit into one burst. When one + // caller asks for more than the bucket can ever accumulate, allow it to borrow once some + // budget is available so the request still makes forward progress. Upper layers are expected + // to call getSuggestedChunkSize() and keep this branch rare. + if (requested_bytes > burst_bytes && available_bytes > 0) + { + available_bytes -= requested_bytes; + return; + } + + if (!waited) + { + GET_METRIC(tiflash_storage_io_limiter_pending_count, type_s3_read_byte).Increment(); + waited = true; + } + + // Sleep only for the missing budget instead of a fixed interval so large readers converge quickly + // after budget becomes available again. + const auto missing = requested_bytes - available_bytes; + const auto wait_us + = std::max(1, static_cast(missing * 1000000.0 / static_cast(current_limit))); + bytes_cv.wait_for(lock, std::chrono::microseconds(wait_us)); + } +} + +UInt64 DB::S3::S3ReadLimiter::getSuggestedChunkSize(UInt64 preferred_chunk_size) const +{ + const auto limit = max_read_bytes_per_sec.load(std::memory_order_relaxed); + if (limit == 0) + return preferred_chunk_size; + return std::max(1, std::min(preferred_chunk_size, burstBytesPerPeriod(limit))); +} + +void DB::S3::S3ReadLimiter::setStop() +{ + { + std::lock_guard lock_bytes(bytes_mutex); + if (stop) + return; + stop = true; + } + bytes_cv.notify_all(); +} + +void DB::S3::S3ReadLimiter::refillBytesLocked(Clock::time_point now) +{ + const auto current_limit = max_read_bytes_per_sec.load(std::memory_order_relaxed); + if (current_limit == 0) + { + available_bytes = 0; + last_refill_time = now; + return; + } + + const auto elapsed_ns = std::chrono::duration_cast(now - last_refill_time).count(); + if (elapsed_ns <= 0) + return; + + const auto burst_bytes = static_cast(burstBytesPerPeriod(current_limit)); + // Clamp to one refill-period burst so a temporarily idle reader cannot accumulate an unbounded burst. + available_bytes = std::min( + burst_bytes, + available_bytes + static_cast(current_limit) * static_cast(elapsed_ns) / 1000000000.0); + last_refill_time = now; +} + +UInt64 DB::S3::S3ReadLimiter::burstBytesPerPeriod(UInt64 max_read_bytes_per_sec_) const +{ + if (max_read_bytes_per_sec_ == 0) + return 0; + return std::max(1, max_read_bytes_per_sec_ * refill_period_ms / 1000); +} +} // namespace DB::S3 diff --git a/dbms/src/Storages/S3/S3ReadLimiter.h b/dbms/src/Storages/S3/S3ReadLimiter.h new file mode 100644 index 00000000000..55749cd7ecd --- /dev/null +++ b/dbms/src/Storages/S3/S3ReadLimiter.h @@ -0,0 +1,108 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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 DB::S3 +{ +enum class S3ReadSource : UInt8 +{ + DirectRead, + FileCacheDownload, +}; + +class S3ReadMetricsRecorder +{ +public: + /// Record remote-read bytes regardless of whether byte throttling is enabled. + static void recordBytes(UInt64 bytes, S3ReadSource source); +}; + +class S3ReadLimiter +{ +public: + /// Stream-based limiting looks attractive because a token could track one live `GetObject` + /// body stream. + /// + /// Important: such a token must not be interpreted as a safe upper bound for the number of + /// `S3RandomAccessFile` objects. One reader can hold a response body open while being idle in a + /// pipeline stage, so limiting tokens too aggressively can stall unrelated readers even when + /// there is little ongoing S3 network I/O. + /// + /// Stream-based limiting is therefore removed for now. Keep this note here so future changes do + /// not accidentally re-introduce the same unsafe hard cap on `S3RandomAccessFile` concurrency. + + /// A lightweight node-level limiter for S3 remote reads. + /// + /// It currently enforces byte-rate limiting only: + /// - total remote-read bytes consumed by direct reads and FileCache downloads + /// + /// Concurrent/open-stream limiting is not provided here. TiFlash readers may keep response + /// bodies open across scheduling gaps, so treating open streams as a hard cap can block forward + /// progress even when the node is no longer transferring many bytes. + explicit S3ReadLimiter(UInt64 max_read_bytes_per_sec_ = 0, UInt64 refill_period_ms_ = 100); + + ~S3ReadLimiter(); + + void updateConfig(UInt64 max_read_bytes_per_sec_); + + /// Charge remote-read bytes. The call blocks when the current node-level budget is exhausted. + /// + /// Requests that fit within one refill-period burst keep strict token-bucket semantics. If one + /// caller asks for more than a single burst can ever accumulate, the limiter allows that request + /// to borrow against future refills once some positive budget is available so the caller does not + /// wait forever. + void requestBytes(UInt64 bytes, S3ReadSource source); + + /// Suggest a chunk size for limiter-aware loops in upper layers. + /// + /// Callers should prefer this value before each `read()` / `ignore()` / buffer refill so large + /// remote reads are naturally split into refill-period-sized steps. Keeping chunks near one burst + /// preserves smooth throttling and makes the large-request borrowing path in `requestBytes()` a + /// rare fallback instead of the common case. + UInt64 getSuggestedChunkSize(UInt64 preferred_chunk_size) const; + + UInt64 maxReadBytesPerSec() const { return max_read_bytes_per_sec.load(std::memory_order_relaxed); } + + void setStop(); + +private: + using Clock = std::chrono::steady_clock; + + /// Refill the token bucket according to elapsed wall time. Caller must hold `bytes_mutex`. + void refillBytesLocked(Clock::time_point now); + /// Limit the instantaneous burst so long reads are naturally split into small limiter-aware chunks. + UInt64 burstBytesPerPeriod(UInt64 max_read_bytes_per_sec_) const; + + const UInt64 refill_period_ms; + std::atomic max_read_bytes_per_sec; + + mutable std::mutex bytes_mutex; + std::condition_variable bytes_cv; + // Token-bucket state for S3 byte throttling. + double available_bytes; + Clock::time_point last_refill_time; + bool stop; + + LoggerPtr log; +}; +} // namespace DB::S3 diff --git a/dbms/src/Storages/S3/S3ReadLimiter_fwd.h b/dbms/src/Storages/S3/S3ReadLimiter_fwd.h new file mode 100644 index 00000000000..d9423796a92 --- /dev/null +++ b/dbms/src/Storages/S3/S3ReadLimiter_fwd.h @@ -0,0 +1,25 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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 DB::S3 +{ +enum class S3ReadSource : UInt8; + +class S3ReadLimiter; +class S3ReadMetricsRecorder; +} // namespace DB::S3 diff --git a/dbms/src/Storages/S3/tests/gtest_filecache.cpp b/dbms/src/Storages/S3/tests/gtest_filecache.cpp index a69149e97bd..acc8194278b 100644 --- a/dbms/src/Storages/S3/tests/gtest_filecache.cpp +++ b/dbms/src/Storages/S3/tests/gtest_filecache.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -26,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -34,6 +36,7 @@ #include #include +#include #include #include #include @@ -52,6 +55,12 @@ namespace DB::ErrorCodes extern const int FILE_DOESNT_EXIST; } +namespace DB::FailPoints +{ +extern const char file_cache_bg_download_fail[]; +extern const char file_cache_bg_download_schedule_fail[]; +} // namespace DB::FailPoints + namespace DB::tests::S3 { class FileCacheTest : public ::testing::Test @@ -1126,13 +1135,298 @@ TEST_F(FileCacheTest, UpdateConfig) // small dt_filecache_max_downloading_count_scale, the queue size should be at least vcores * concurrency settings.dt_filecache_downloading_count_scale = 2.0; settings.dt_filecache_max_downloading_count_scale = 0.1; + settings.dt_filecache_wait_on_downloading_ms = 7; file_cache.updateConfig(settings); ASSERT_DOUBLE_EQ(file_cache.download_count_scale, 2.0); ASSERT_DOUBLE_EQ(file_cache.max_downloading_count_scale, 0.1); + ASSERT_EQ(file_cache.wait_on_downloading_ms.load(std::memory_order_relaxed), 7); ASSERT_EQ(S3FileCachePool::get().getMaxThreads(), vcores * 2.0); ASSERT_EQ(S3FileCachePool::get().getQueueSize(), vcores * 2.0); } +TEST_F(FileCacheTest, FileSegmentWaitForNotEmptyFor) +{ + auto file_seg = std::make_shared("/tmp/test", FileSegment::Status::Empty, 128, FileType::Merged); + ASSERT_EQ(file_seg->waitForNotEmptyFor(10ms), FileSegment::Status::Empty); + + auto complete_future = std::async(std::launch::async, [&]() { return file_seg->waitForNotEmptyFor(200ms); }); + std::this_thread::sleep_for(20ms); + file_seg->setComplete(256); + ASSERT_EQ(complete_future.get(), FileSegment::Status::Complete); + + auto failed_seg + = std::make_shared("/tmp/test_failed", FileSegment::Status::Empty, 128, FileType::Meta); + auto failed_future = std::async(std::launch::async, [&]() { return failed_seg->waitForNotEmptyFor(200ms); }); + std::this_thread::sleep_for(20ms); + failed_seg->setStatus(FileSegment::Status::Failed); + ASSERT_EQ(failed_future.get(), FileSegment::Status::Failed); +} + +TEST_F(FileCacheTest, GetWaitOnDownloadingHitAndTimeout) +{ + auto cache_dir = fmt::format("{}/wait_on_downloading", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = 100}; + + UInt16 vcores = 2; + IORateLimiter rate_limiter; + FileCache file_cache(capacity_metrics, cache_config, vcores, rate_limiter); + + Settings settings; + settings.dt_filecache_downloading_count_scale = 2.0; + settings.dt_filecache_max_downloading_count_scale = 2.0; + settings.dt_filecache_wait_on_downloading_ms = 200; + file_cache.updateConfig(settings); + + auto objects = genObjects(/*store_count*/ 1, /*table_count*/ 1, /*file_count*/ 1, {"1.merged", "2.merged"}); + auto sp_download = SyncPointCtl::enableInScope("before_FileCache::downloadImpl_download_to_local"); + auto sp_wait = SyncPointCtl::enableInScope("before_FileSegment::waitForNotEmpty_wait"); + + auto first_key = S3FilenameView::fromKey(objects[0].key); + // First request publishes the `Empty` placeholder and starts the background download. + ASSERT_EQ(file_cache.get(first_key, objects[0].size), nullptr); + sp_download.waitAndPause(); + + // With a generous bounded-wait budget, the follower should reuse the downloader result instead of returning miss. + auto wait_hit = std::async(std::launch::async, [&]() { return file_cache.get(first_key, objects[0].size); }); + sp_wait.waitAndPause(); + sp_download.next(); + sp_wait.next(); + auto hit_seg = wait_hit.get(); + ASSERT_NE(hit_seg, nullptr); + ASSERT_TRUE(hit_seg->isReadyToRead()); + + settings.dt_filecache_wait_on_downloading_ms = 30; + file_cache.updateConfig(settings); + auto second_key = S3FilenameView::fromKey(objects[1].key); + // Re-run the same pattern with a much smaller budget so the follower times out and returns miss. + ASSERT_EQ(file_cache.get(second_key, objects[1].size), nullptr); + sp_download.waitAndPause(); + auto wait_timeout = std::async(std::launch::async, [&]() { return file_cache.get(second_key, objects[1].size); }); + sp_wait.waitAndPause(); + sp_wait.next(); + ASSERT_EQ(wait_timeout.get(), nullptr); + sp_download.next(); + sp_download.disable(); + + waitForBgDownload(file_cache); +} + +TEST_F(FileCacheTest, GetWaitOnDownloadingReturnsMissWhenDownloaderFails) +{ + auto cache_dir = fmt::format("{}/wait_on_downloading_failed", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = 100}; + + UInt16 vcores = 2; + IORateLimiter rate_limiter; + FileCache file_cache(capacity_metrics, cache_config, vcores, rate_limiter); + + Settings settings; + settings.dt_filecache_downloading_count_scale = 2.0; + settings.dt_filecache_max_downloading_count_scale = 2.0; + settings.dt_filecache_wait_on_downloading_ms = 200; + file_cache.updateConfig(settings); + + auto objects = genObjects(/*store_count*/ 1, /*table_count*/ 1, /*file_count*/ 1, {"1.merged"}); + auto sp_fail = SyncPointCtl::enableInScope("before_FileCache::bgDownloadExecutor_fail_point"); + auto sp_wait = SyncPointCtl::enableInScope("before_FileSegment::waitForNotEmpty_wait"); + + auto key = S3FilenameView::fromKey(objects[0].key); + // First caller creates the `Empty` placeholder and starts the background download. + ASSERT_EQ(file_cache.get(key, objects[0].size), nullptr); + sp_fail.waitAndPause(); + + // The follower reaches `get()` while the same key is still being downloaded. Inject a failure right before + // the downloader starts copying the body so the follower wakes up with `Status::Failed` and returns miss. + { + FailPointHelper::enableFailPoint(FailPoints::file_cache_bg_download_fail); + SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::file_cache_bg_download_fail); }); + auto wait_failed = std::async(std::launch::async, [&]() { return file_cache.get(key, objects[0].size); }); + sp_wait.waitAndPause(); + sp_fail.next(); + sp_wait.next(); + ASSERT_EQ(wait_failed.get(), nullptr); + } + sp_fail.disable(); + + waitForBgDownload(file_cache); + ASSERT_EQ(file_cache.bg_download_fail_count.load(std::memory_order_relaxed), 1); + + // The failed placeholder must be removed from the cache table. Otherwise later requests would keep observing + // the stale failed entry instead of creating a fresh download task. + { + std::lock_guard lock(file_cache.mtx); + auto & table = file_cache.tables[static_cast(FileType::Merged)]; + ASSERT_EQ(table.get(objects[0].key), nullptr); + } + + // A later foreground retry should succeed, proving the failed follower path does not leave the cache stuck. + auto file_seg = file_cache.getOrWait(key, objects[0].size); + ASSERT_NE(file_seg, nullptr); + ASSERT_TRUE(file_seg->isReadyToRead()); + ASSERT_EQ(file_seg->getSize(), objects[0].size); +} + +TEST_F(FileCacheTest, BgDownloadWorksWithSharedS3ReadLimiter) +{ + auto cache_dir = fmt::format("{}/bg_download_limiter", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = 100}; + + UInt16 vcores = 2; + IORateLimiter rate_limiter; + FileCache file_cache(capacity_metrics, cache_config, vcores, rate_limiter); + Settings settings; + settings.dt_filecache_downloading_count_scale = 2.0; + settings.dt_filecache_max_downloading_count_scale = 2.0; + file_cache.updateConfig(settings); + + // This test is kept as a regression check for the shared-limiter plumbing on the FileCache download + // path. Stream-based limiting has been removed, and the byte limit is intentionally disabled here, + // so the expectation is simply that background downloads still make progress and do not deadlock when + // a shared S3ReadLimiter object is attached to the client. + auto limiter = std::make_shared(0, 1); + s3_client->setS3ReadLimiter(limiter); + SCOPE_EXIT({ s3_client->setS3ReadLimiter(nullptr); }); + + auto objects = genObjects(/*store_count*/ 1, /*table_count*/ 1, /*file_count*/ 1, {"3.merged", "4.merged"}); + auto sp_download = SyncPointCtl::enableInScope("before_FileCache::downloadImpl_download_to_local"); + + // Start one background download and pause it at the download-to-local boundary. + ASSERT_EQ(file_cache.get(S3FilenameView::fromKey(objects[0].key), objects[0].size), nullptr); + sp_download.waitAndPause(); + + // Submit a second download while the first one is paused. The test passes as long as both downloads + // complete normally after the pause is released. + ASSERT_EQ(file_cache.get(S3FilenameView::fromKey(objects[1].key), objects[1].size), nullptr); + std::this_thread::sleep_for(50ms); + + sp_download.next(); + sp_download.disable(); + waitForBgDownload(file_cache); + + auto file_seg1 = file_cache.get(S3FilenameView::fromKey(objects[0].key), objects[0].size); + auto file_seg2 = file_cache.get(S3FilenameView::fromKey(objects[1].key), objects[1].size); + ASSERT_NE(file_seg1, nullptr); + ASSERT_NE(file_seg2, nullptr); + ASSERT_TRUE(file_seg1->isReadyToRead()); + ASSERT_TRUE(file_seg2->isReadyToRead()); +} + +TEST_F(FileCacheTest, BgDownloadUsesLimiterSuggestedChunkSize) +{ + auto cache_dir = fmt::format("{}/bg_download_low_limit", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = 100}; + + UInt16 vcores = 2; + IORateLimiter rate_limiter; + FileCache file_cache(capacity_metrics, cache_config, vcores, rate_limiter); + Settings settings; + settings.dt_filecache_downloading_count_scale = 2.0; + settings.dt_filecache_max_downloading_count_scale = 2.0; + file_cache.updateConfig(settings); + + auto object_key = fmt::format("s{}/data/t_{}/dmf_{}/small_merged.merged", nextId(), nextId(), nextId()); + constexpr size_t object_size = 8 * 1024; + writeFile(object_key, '7', object_size, WriteSettings{}); + + auto limiter = std::make_shared(32 * 1024, /*refill_period_ms*/ 100); + s3_client->setS3ReadLimiter(limiter); + SCOPE_EXIT({ s3_client->setS3ReadLimiter(nullptr); }); + + constexpr Int64 expected_delay_ms = 200; + constexpr Int64 delay_tolerance_ms = 40; + + AtomicStopwatch watch; + ASSERT_EQ(file_cache.get(S3FilenameView::fromKey(object_key), object_size), nullptr); + waitForBgDownload(file_cache); + ASSERT_GE(watch.elapsedMilliseconds(), expected_delay_ms - delay_tolerance_ms); + + auto file_seg = file_cache.get(S3FilenameView::fromKey(object_key), object_size); + ASSERT_NE(file_seg, nullptr); + ASSERT_TRUE(file_seg->isReadyToRead()); + ASSERT_EQ(file_seg->getSize(), object_size); +} + +TEST_F(FileCacheTest, GetWaitOnDownloadingSupportsColDataAndOther) +{ + auto cache_dir = fmt::format("{}/wait_on_downloading_non_merged", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = 100}; + + UInt16 vcores = 2; + IORateLimiter rate_limiter; + FileCache file_cache(capacity_metrics, cache_config, vcores, rate_limiter); + + Settings settings; + settings.dt_filecache_downloading_count_scale = 2.0; + settings.dt_filecache_max_downloading_count_scale = 2.0; + settings.dt_filecache_wait_on_downloading_ms = 200; + file_cache.updateConfig(settings); + + auto objects = genObjects(/*store_count*/ 1, /*table_count*/ 1, /*file_count*/ 1, {"1.dat", "meta"}); + auto sp_download = SyncPointCtl::enableInScope("before_FileCache::downloadImpl_download_to_local"); + auto sp_wait = SyncPointCtl::enableInScope("before_FileSegment::waitForNotEmpty_wait"); + + auto run_wait_hit_case = [&](const ObjectInfo & obj, FileType expected_file_type) { + auto key = S3FilenameView::fromKey(obj.key); + // The first request creates the placeholder, and the second request should hit the same bounded-wait path + // regardless of whether the file is classified as coldata or other. + ASSERT_EQ(file_cache.get(key, obj.size), nullptr); + sp_download.waitAndPause(); + + auto wait_hit = std::async(std::launch::async, [&]() { return file_cache.get(key, obj.size); }); + sp_wait.waitAndPause(); + sp_download.next(); + sp_wait.next(); + + auto file_seg = wait_hit.get(); + ASSERT_NE(file_seg, nullptr); + ASSERT_TRUE(file_seg->isReadyToRead()); + ASSERT_EQ(file_seg->getSize(), obj.size); + ASSERT_EQ(file_seg->getFileType(), expected_file_type); + }; + + run_wait_hit_case(objects[0], FileType::ColData); + run_wait_hit_case(objects[1], FileType::Meta); + + sp_download.disable(); + waitForBgDownload(file_cache); +} + +TEST_F(FileCacheTest, BgDownloadSupportsColDataAndOther) +{ + auto cache_dir = fmt::format("{}/bg_download_non_merged", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = 100}; + + UInt16 vcores = 2; + IORateLimiter rate_limiter; + FileCache file_cache(capacity_metrics, cache_config, vcores, rate_limiter); + Settings settings; + settings.dt_filecache_downloading_count_scale = 2.0; + settings.dt_filecache_max_downloading_count_scale = 2.0; + file_cache.updateConfig(settings); + + auto objects = genObjects(/*store_count*/ 1, /*table_count*/ 1, /*file_count*/ 1, {"2.dat", "meta"}); + + for (const auto & obj : objects) + { + auto key = S3FilenameView::fromKey(obj.key); + ASSERT_EQ(file_cache.get(key, obj.size), nullptr); + } + + waitForBgDownload(file_cache); + + std::array expected_file_types = {FileType::ColData, FileType::Meta}; + size_t index = 0; + for (const auto & obj : objects) + { + auto file_seg = file_cache.get(S3FilenameView::fromKey(obj.key), obj.size); + ASSERT_NE(file_seg, nullptr); + ASSERT_TRUE(file_seg->isReadyToRead()); + ASSERT_EQ(file_seg->getSize(), obj.size); + ASSERT_EQ(file_seg->getFileType(), expected_file_types[index]); + ++index; + } +} + TEST_F(FileCacheTest, GetBeingBlock) { auto cache_dir = fmt::format("{}/update_config", tmp_dir); @@ -1178,4 +1472,73 @@ TEST_F(FileCacheTest, GetBeingBlock) waitForBgDownload(file_cache); } +TEST_F(FileCacheTest, FailedDownloadReleasesFinalizedReservedSize) +{ + auto cache_dir = fmt::format("{}/failed_download_reserved_size", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = 100}; + + UInt16 vcores = 2; + IORateLimiter rate_limiter; + FileCache file_cache(capacity_metrics, cache_config, vcores, rate_limiter); + Settings settings; + settings.dt_filecache_downloading_count_scale = 2.0; + settings.dt_filecache_max_downloading_count_scale = 2.0; + file_cache.updateConfig(settings); + + auto objects = genObjects(/*store_count*/ 1, /*table_count*/ 1, /*file_count*/ 1, {"1.merged"}); + auto key = S3FilenameView::fromKey(objects[0].key); + + auto assert_failed_download_releases_space = [&](std::optional requested_size) { + FailPointHelper::enableFailPoint(FailPoints::file_cache_bg_download_fail); + SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::file_cache_bg_download_fail); }); + + ASSERT_EQ(file_cache.cache_used, 0); + ASSERT_EQ(file_cache.get(key, requested_size), nullptr); + waitForBgDownload(file_cache); + + ASSERT_EQ(file_cache.cache_used, 0); + ASSERT_EQ(file_cache.bg_download_fail_count.load(std::memory_order_relaxed), 1); + ASSERT_EQ(file_cache.bg_download_succ_count.load(std::memory_order_relaxed), 0); + }; + + // std::nullopt uses the file-type estimate and must still release the finalized reservation correctly. + assert_failed_download_releases_space(std::nullopt); + + file_cache.bg_download_fail_count.store(0, std::memory_order_relaxed); + + // A caller-provided wrong size should also not corrupt cache_used after finalizeReservedSize rebases it. + assert_failed_download_releases_space(objects[0].size - 1024); +} + +TEST_F(FileCacheTest, ScheduleBgDownloadFailureCleansUpPlaceholder) +{ + auto cache_dir = fmt::format("{}/schedule_bg_download_failure", tmp_dir); + StorageRemoteCacheConfig cache_config{.dir = cache_dir, .capacity = cache_capacity, .dtfile_level = 100}; + + UInt16 vcores = 2; + IORateLimiter rate_limiter; + FileCache file_cache(capacity_metrics, cache_config, vcores, rate_limiter); + Settings settings; + settings.dt_filecache_wait_on_downloading_ms = 200; + file_cache.updateConfig(settings); + + auto objects = genObjects(/*store_count*/ 1, /*table_count*/ 1, /*file_count*/ 1, {"1.merged"}); + auto key = S3FilenameView::fromKey(objects[0].key); + auto file_type = FileCache::getFileType(objects[0].key); + + FailPointHelper::enableFailPoint(FailPoints::file_cache_bg_download_schedule_fail); + SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::file_cache_bg_download_schedule_fail); }); + + ASSERT_EQ(file_cache.get(key, objects[0].size), nullptr); + ASSERT_EQ(file_cache.bg_downloading_count.load(std::memory_order_relaxed), 0); + ASSERT_EQ(file_cache.bg_download_fail_count.load(std::memory_order_relaxed), 1); + ASSERT_EQ(file_cache.tables[static_cast(file_type)].get(objects[0].key, /*update_lru*/ false), nullptr); + + FailPointHelper::disableFailPoint(FailPoints::file_cache_bg_download_schedule_fail); + ASSERT_EQ(file_cache.get(key, objects[0].size), nullptr); + waitForBgDownload(file_cache); + ASSERT_EQ(file_cache.bg_download_succ_count.load(std::memory_order_relaxed), 1); + ASSERT_NE(file_cache.tables[static_cast(file_type)].get(objects[0].key, /*update_lru*/ false), nullptr); +} + } // namespace DB::tests::S3 diff --git a/dbms/src/Storages/S3/tests/gtest_s3client.cpp b/dbms/src/Storages/S3/tests/gtest_s3client.cpp index 61ac7dbc27b..b18daa41c3f 100644 --- a/dbms/src/Storages/S3/tests/gtest_s3client.cpp +++ b/dbms/src/Storages/S3/tests/gtest_s3client.cpp @@ -15,8 +15,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -252,6 +254,33 @@ try } CATCH +TEST_F(S3ClientTest, PublishS3ReadLimiter) +{ + auto prev_limiter = ClientFactory::instance().sharedTiFlashClient()->getS3ReadLimiter(); + SCOPE_EXIT({ ClientFactory::instance().setS3ReadLimiter(prev_limiter); }); + + auto limiter = std::make_shared(4096, 7); + ClientFactory::instance().setS3ReadLimiter(limiter); + ASSERT_EQ(client->getS3ReadLimiter(), limiter); + + IORateLimiter io_rate_limiter; + IORateLimitConfig cfg; + cfg.s3_max_read_bytes_per_sec = 8192; + io_rate_limiter.updateLimiterByConfig(cfg); + + auto published = io_rate_limiter.getS3ReadLimiter(); + ASSERT_NE(published, nullptr); + ClientFactory::instance().setS3ReadLimiter(published); + ASSERT_EQ(ClientFactory::instance().sharedTiFlashClient()->getS3ReadLimiter(), published); + ASSERT_EQ(published->maxReadBytesPerSec(), 8192); + + cfg.s3_max_read_bytes_per_sec = 0; + io_rate_limiter.updateLimiterByConfig(cfg); + auto disabled = io_rate_limiter.getS3ReadLimiter(); + ASSERT_EQ(disabled, published); + ASSERT_EQ(disabled->maxReadBytesPerSec(), 0); +} + TEST_F(S3ClientTest, ListPrefixEarlyStopOnTruncatedResult) try { diff --git a/dbms/src/Storages/S3/tests/gtest_s3file.cpp b/dbms/src/Storages/S3/tests/gtest_s3file.cpp index a2bf0edc7a9..a1c9849efdb 100644 --- a/dbms/src/Storages/S3/tests/gtest_s3file.cpp +++ b/dbms/src/Storages/S3/tests/gtest_s3file.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -67,6 +68,7 @@ extern const char force_set_mocked_s3_object_mtime[]; extern const char force_syncpoint_on_s3_upload[]; extern const char force_s3_random_access_file_init_fail[]; extern const char force_s3_random_access_file_read_fail[]; +extern const char force_s3_random_access_file_seek_chunked[]; } // namespace DB::FailPoints namespace DB::tests @@ -278,6 +280,37 @@ try } CATCH +TEST_P(S3FileTest, SeekSkipsChunkedPathWhenLimiterDisabled) +try +{ + const auto size = 1024 * 1024 * 10; // 10MB + const String key = "/a/b/c/seek_disabled_limiter"; + writeFile(key, size, WriteSettings{}); + + auto prev_limiter = s3_client->getS3ReadLimiter(); + auto disabled_limiter = std::make_shared(0, 1); + s3_client->setS3ReadLimiter(disabled_limiter); + SCOPE_EXIT({ s3_client->setS3ReadLimiter(prev_limiter); }); + + S3RandomAccessFile file(s3_client, key, nullptr); + std::vector tmp_buf(256); + ASSERT_EQ(file.read(tmp_buf.data(), tmp_buf.size()), tmp_buf.size()); + + FailPointHelper::enableFailPoint(FailPoints::force_s3_random_access_file_seek_chunked); + SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::force_s3_random_access_file_seek_chunked); }); + + constexpr off_t target_offset = 1024 * 1024; + off_t seek_offset = -1; + ASSERT_NO_THROW(seek_offset = file.seek(target_offset, SEEK_SET)); + ASSERT_EQ(seek_offset, target_offset); + ASSERT_EQ(file.read(tmp_buf.data(), tmp_buf.size()), tmp_buf.size()); + + std::vector expected(256); + std::iota(expected.begin(), expected.end(), 0); + ASSERT_EQ(tmp_buf, expected); +} +CATCH + TEST_P(S3FileTest, ReadAfterDel1) try { diff --git a/docs/design/2026-03-24-disagg-s3-node-level-backpressure-and-filecache-dedup.md b/docs/design/2026-03-24-disagg-s3-node-level-backpressure-and-filecache-dedup.md new file mode 100644 index 00000000000..9d9bebfc691 --- /dev/null +++ b/docs/design/2026-03-24-disagg-s3-node-level-backpressure-and-filecache-dedup.md @@ -0,0 +1,661 @@ +# TiFlash Disaggregated Read Path: Node-Level S3 Backpressure and FileCache Same-Key Deduplication + +Purpose: define a production-oriented design with deliberately constrained implementation scope to solve uncontrolled S3 traffic on compute nodes in disaggregated read workloads caused by cold reads and amplified cache misses. The target risk scenario is: a large portion of one table is cold on the current node and therefore absent from local `FileCache`; the system receives concurrent queries in a short time window; multiple queries scan roughly the same set of files on TiFlash at about the same time; S3 access latency keeps rising; node network becomes congested; queries queue for a long time; and the situation eventually turns into query failures or obvious timeouts. + +Date: 2026-03-24 + +## Summary + +This proposal recommends rolling out in two incremental phases: + +1. Introduce a node-level `S3ReadLimiter` to cap total remote-read bandwidth on one node. +2. In `FileCache::get(...)`, add one bounded wait only when the same physical key already has an in-flight download, so followers prefer to reuse the existing download result instead of immediately falling back to `S3RandomAccessFile` direct reads. + +The first version deliberately does **not** do the following: + +- no query-level async read scheduler +- no new global miss coordination table +- no sleep-retry on `RejectTooManyDownloading` +- no complicated logic-file-type-specific branching inside `FileCache` + +The reason is straightforward: the only stable, reusable, and easily rollbackable seams in the current code are `S3RandomAccessFile`, `FileCache`, `ClientFactory/TiFlashS3Client`, and `IORateLimitConfig` / `IORateLimiter`. Once we put a hard node-level traffic guardrail in place and deduplicate same-key miss fan-out, we are already able to turn “an occasional network blow-up that makes queries fail” into “a bounded, observable, tunable problem”. + +## Context + +### 1. Current direct S3 reads issue `GetObject` immediately + +The current S3 open path is: + +```text +FileProvider::newRandomAccessFile + -> S3RandomAccessFile::create + -> FileCache::getRandomAccessFile + -> cache hit: return local file + -> cache miss: return nullptr + -> fallback: construct S3RandomAccessFile + -> initialize() immediately in constructor + -> client->GetObject(req) +``` + +Verified code locations: + +- `FileProvider::newRandomAccessFile` goes directly to `S3RandomAccessFile::create(...)` on the S3 path and does not use the incoming `read_limiter`: + `dbms/src/IO/FileProvider/FileProvider.cpp:35` +- `S3RandomAccessFile::create(...)` first tries `FileCache::getRandomAccessFile(...)`, then immediately constructs a remote reader on miss: + `dbms/src/Storages/S3/S3RandomAccessFile.cpp:345` +- `S3RandomAccessFile` calls `initialize("init file")` in its constructor: + `dbms/src/Storages/S3/S3RandomAccessFile.cpp:65` +- `initialize(...)` directly calls `client_ptr->GetObject(req)`: + `dbms/src/Storages/S3/S3RandomAccessFile.cpp:278` + +This means: + +- once `FileCache` misses, the caller immediately consumes a new S3 stream +- there is currently no node-level hard cap on total bandwidth or total remote-read concurrency + +### 2. Current `FileCache::get(...)` only inserts a background download and returns immediately on miss + +`FileCache::get(...)` currently has three key branches: + +- hit an existing `Complete` segment and return it directly: + `dbms/src/Storages/S3/FileCache.cpp:342` +- hit an existing segment that is not ready yet, record a miss, return `nullptr` immediately: + `dbms/src/Storages/S3/FileCache.cpp:351` +- key does not exist and caching is allowed: insert an `Empty` segment, schedule background download, return `nullptr` immediately: + `dbms/src/Storages/S3/FileCache.cpp:366` + +Current behavior can be summarized as: + +```text +The same key is already being downloaded + -> get() still returns nullptr + -> caller keeps direct-falling back to S3 + +First miss of one key + -> get() only inserts Empty + submits bg download + -> caller keeps direct-falling back to S3 +``` + +That is, today `FileCache` does not help query hot paths reuse an already running download. + +### 3. A wait primitive already exists, but only for the foreground-download path + +`FileSegment::waitForNotEmpty()` already exists, but it is a 30-second-granularity loop and is used only by `getOrWait(...)`: + +- `dbms/src/Storages/S3/FileCache.cpp:76` +- `dbms/src/Storages/S3/FileCache.cpp:411` + +`getOrWait(...)` means: + +- if key already exists: wait for the download result +- if key does not exist: perform a foreground download directly + +This API is suitable for vector index / inverted index / full-text index paths that must land on disk and then `mmap`, but not suitable for ordinary query read paths, because it turns the very first miss on the current thread into a synchronous download. + +### 4. Existing `IORateLimiter` cannot directly limit remote S3 reads + +Current read-side `IORateLimiter` implementation depends on `/proc//io` and `/proc//task//io` to sample real local-disk bytes: + +- `dbms/src/IO/BaseFile/RateLimiter.cpp:523` +- `dbms/src/IO/BaseFile/RateLimiter.cpp:601` +- `dbms/src/IO/BaseFile/RateLimiter.cpp:644` + +This works for local disks, not remote S3 traffic, for two reasons: + +1. S3 bytes never show up in `/proc/.../io` disk statistics. +2. For S3, the application already knows how many bytes it plans to read and how many bytes it actually read, so `/proc` sampling is unnecessary. + +Therefore remote reads need a dedicated limiter implementation instead of reusing the existing local-disk `ReadLimiter`. + +### 5. `S3RandomAccessFile` does not have access to `Context` + +Today `S3RandomAccessFile` only holds: + +- `TiFlashS3Client` +- `remote_fname` +- `ScanContext` + +Code location: + +- `dbms/src/Storages/S3/S3RandomAccessFile.h:43` + +It does not have a `Context` or `IORateLimiter` reference. Therefore, if we want a node-level remote limiter, we must first solve how the runtime object becomes reachable from the S3 read path. + +### 6. Under MetaV2, many logical small files are mapped to the physical `.merged` key + +This is the key fact that shapes the phase-2 design. + +Under MetaV2 / DMFile V3: + +- min-max index is written into the merged file: + `dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp:312` +- mark is written into the merged file: + `dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp:345` +- some small data files are also merged into the merged file: + `dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp:321` + +Correspondingly, `getReadFileSize(...)` returns the size of the entire merged object, not the logical sub-file size: + +- `dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp:395` + +So the real access chain becomes: + +```text +Logically read mark / index / small data + -> resolve to merged_sub_file_infos + -> actually open N.merged + -> FileCache only sees the physical key: N.merged +``` + +The direct conclusion is: + +- `FileCache::get(...)` cannot reliably tell, from the physical key alone, whether this is a small mark/index read or a large merged-data read +- therefore the first version should not introduce complicated wait policies at the `FileCache` layer based on logical file categories + +## Goals + +- Prevent one TiFlash compute node from saturating its NIC because of cold S3 reads and thereby causing query failures. +- Put direct S3 reads and FileCache downloads under the same node-level backpressure instead of allowing the two paths to amplify independently. +- Reduce miss fan-out for the same physical key without changing the current synchronous read model. +- Keep the cache-hit fast path almost unchanged, especially with no new heap allocations or major CPU overhead when the limiter is disabled. +- Make every new behavior switchable, observable, and rollbackable online. + +## Non-Goals + +- No query-level async read scheduler. +- No cross-node coordination or global download registry. +- No sleep-retry for `RejectTooManyDownloading` in the first version. +- Do not turn the first miss into a foreground download in the first version; only deduplicate followers. +- Do not separately optimize metadata-burst during read-task build; the node-level limiter in phase 1 automatically covers these S3 reads. +- Do not add a separate timeout/cancel semantic for limiter waits in the first version. + +## Design + +### Phase 1: node-level S3 remote-read limiter + +#### Design decision + +Add a dedicated `S3ReadLimiter` that constrains: + +- total remote-read bytes per second on one node + +It covers two paths: + +- `S3RandomAccessFile` direct reads +- `FileCache::downloadImpl(...)` foreground/background downloads + +#### Why the first version keeps only byte-rate limiting + +We experimented with modeling the `GetObject` body lifetime as a `StreamToken`, and with using `s3_max_get_object_streams` to cap concurrent active streams. However, under real workloads and replay validation, this turned out to be an unsafe upper bound for the number of `S3RandomAccessFile` objects: + +- one `S3RandomAccessFile` may keep a remote body stream open across pipeline / scheduler gaps while not continuously transferring bytes +- for semantic consistency, the token must cover the entire body lifetime; once the limit is set too tightly, readers that hold a stream but are temporarily idle are still counted under the hard cap +- the result is that active streams stay pinned near the cap while real S3 throughput has already dropped, and subsequent readers remain blocked, which manifests as permanently stuck queries or no forward progress + +Therefore the first version keeps `s3_max_read_bytes_per_sec` as the deterministic node-level guardrail and removes `s3_max_get_object_streams` from both config and metrics. If we want to reintroduce a stream-dimension cap in the future, we first need a model that is closer to real network occupancy than a reader-lifetime token. + +#### Configuration surface + +Continue to use `[storage.io_rate_limit]` and add one new field: + +- `s3_max_read_bytes_per_sec` + - node-level upper bound on total remote-read bandwidth + - uniformly covers `S3RandomAccessFile` read/forward seek and `FileCache` download + - `0` means disabled + +Suggested TOML form: + +```toml +[storage.io_rate_limit] +# Existing disk IO limiter configs... +# max_bytes_per_sec = 0 + +# Limit total S3 remote-read bandwidth on one compute node. +# 0 means disabled. +# s3_max_read_bytes_per_sec = 0 +``` + +#### Runtime ownership and integration points + +Recommended runtime object relationship: + +```text +storage.io_rate_limit + -> IORateLimitConfig + -> IORateLimiter handles parse / reload + -> owns a dedicated S3ReadLimiter + -> Server publishes it to ClientFactory during initialization + -> TiFlashS3Client holds shared_ptr + -> used directly by S3RandomAccessFile / FileCache::downloadImpl +``` + +Key constraints: + +- `S3ReadLimiter` is not a subclass of the current `ReadLimiter` + - it does not depend on `/proc/.../io` + - it throttles directly from application-known byte counts +- `IORateLimiter` still owns config reload entry + - but the remote limiter does not participate in current disk IO auto tune +- `ClientFactory` is responsible for publishing the current limiter to all `TiFlashS3Client` instances + - including the client created immediately at startup + - and clients lazily initialized later by compute nodes through write nodes + +This avoids introducing an unnatural `Context` dependency into `S3RandomAccessFile`. + +#### Behavioral semantics + +##### 1. Semantics of byte budget + +`s3_max_read_bytes_per_sec` accounts and limits real remote traffic: + +- bytes read by `S3RandomAccessFile::readImpl(...)` +- bytes discarded by forward skip in `S3RandomAccessFile::seekImpl(...)` +- bytes downloaded to local storage by `FileCache::downloadImpl(...)` + +It is a throttling mechanism, not fail-fast: + +- wait when the budget is exhausted +- continue once budget becomes available again + +#### Key implementation details + +##### 1. `S3RandomAccessFile` switches to chunked mode only when the limiter is enabled + +Current implementation: + +- `readImpl(...)` performs one `istream.read(buf, size)`: + `dbms/src/Storages/S3/S3RandomAccessFile.cpp:121` +- `seekImpl(...)` performs one `istream.ignore(delta)` on forward skip: + `dbms/src/Storages/S3/S3RandomAccessFile.cpp:195` + +If we simply add one `request(size)` / `request(delta)` before these calls, there are three problems: + +1. one read can accumulate an overly large burst +2. retry / EOF / stream error can make reserved budget diverge from actual bytes +3. forward seek can burst a large amount of traffic after obtaining one large budget chunk + +Therefore the first version should: + +- keep the current one-shot `read/ignore` fast path when the limiter is disabled +- switch to fixed-size chunk loops when the limiter is enabled + - for example 64 KiB or 128 KiB + - wait for budget before each chunk advances the stream + - no new heap allocation + +This requires: + +- `readImpl(...)` to become chunked `istream.read(...)` when the limiter is enabled +- `seekImpl(...)` forward skip to become chunked `istream.ignore(...)` when the limiter is enabled +- backward seek to keep the current semantics: re-`GetObject` plus a new body stream + +##### 2. `FileCache::downloadImpl(...)` reuses the same limiter without introducing a new abstraction layer + +Current download flow: + +```text +downloadImpl + -> client->GetObject(req) + -> downloadToLocal(result.GetBody(), ...) + -> ReadBufferFromIStream + copyData(...) +``` + +Code locations: + +- `dbms/src/Storages/S3/FileCache.cpp:1013` +- `dbms/src/Storages/S3/FileCache.cpp:989` + +The first version does not need a new IO framework here. Keep it simple: + +- if limiter is disabled, continue to use the current download implementation +- if limiter is enabled, change `downloadToLocal(...)` into a fixed-buffer local loop + - read a chunk + - write to local file + - no extra heap allocations throughout the path + +This path is not the cache-hit hot path of a query, so one additional tiny branch relative to current `copyData(...)` is acceptable, but it should not introduce a new object graph. + +#### Compatibility and invariants + +- cache-hit read path stays unchanged +- when the limiter is disabled, behavior must remain identical to today +- existing `S3RandomAccessFile::initialize(...)` retry logic must be preserved +- remote limiter does not depend on query context +- remote limiter does not participate in current disk IO auto tune + +#### Observability + +At minimum, add these node-level metrics: + +- count and duration of waiting for byte budget +- charged bytes of direct reads +- charged bytes of FileCache downloads + +Keep and continue to use existing metrics: + +- `tiflash_storage_s3_request_seconds` +- `ProfileEvents::S3GetObject` +- `ProfileEvents::S3ReadBytes` +- existing disaggregated S3 statistics in `ScanContext` +- `CurrentMetrics::S3RandomAccessFile` + - this can continue to serve as an approximate guardrail for direct-read pressure + +### Phase 2: same-key deduplication in `FileCache::get(...)` + +#### Design decision + +The first version does exactly one thing: + +- when `FileCache::get(...)` finds that the same physical key already has an `Empty` segment, meaning someone is downloading it, allow the current thread to perform one very short bounded wait and try to reuse that result + +The first version intentionally does not do two things: + +- no wait strategy by logical file type +- no sleep-retry on `RejectTooManyDownloading` + +This is an intentional narrowing of scope: + +- `FileCache` can reliably see only physical keys +- under MetaV2, mark/index/small data often map to `.merged` +- queue-congestion retry is not very helpful for wide-table scenarios with many unique keys missing together +- phase 1 already solves the root cause of “the node network gets blown up”, so phase 2 can focus on the most reliable same-key deduplication only + +#### Configuration surface + +Add one dynamic setting near the existing `dt_filecache_*` settings: + +- `dt_filecache_wait_on_downloading_ms` + - `0` means disabled + - when non-zero, `FileCache::get(...)` waits up to this duration when it hits an existing `Empty` segment + +The recommended code default for the first version is `0`; when enabling in canary, start from a very small value such as `1~2ms`. + +#### Behavioral semantics + +Current behavior: + +```text +get(key) + -> found Empty segment + -> return nullptr + -> caller directly falls back to S3 +``` + +Behavior after the change: + +```text +get(key) + -> found Empty segment + -> release FileCache::mtx + -> waitForNotEmptyFor(dt_filecache_wait_on_downloading_ms) + -> Complete: return cached segment + -> Failed / timeout: return nullptr +``` + +Note two points: + +1. wait only when the same key already has an in-flight download +2. for the first miss where the current thread just inserted the `Empty` segment, keep the current behavior in the first version and return `nullptr` immediately + +In other words, the first version deduplicates followers, not leaders. + +#### Local primitive to add + +Add on `FileSegment`: + +- `waitForNotEmptyFor(timeout_ms)` + +Code locations: + +- `dbms/src/Storages/S3/FileCache.h:90` +- `dbms/src/Storages/S3/FileCache.cpp:76` + +Requirements: + +- return the state observed at timeout end +- hold no `FileCache::mtx` during the entire wait +- allow existing `waitForNotEmpty()` to share internal logic with it and avoid duplication + +#### Why the first version does not add sleep-retry on `RejectTooManyDownloading` + +We do not adopt this because: + +1. it mainly helps when the queue is only briefly jittering +2. it is not very helpful for the main problem of wide-table cold reads where many unique keys miss together +3. it expands `FileCache::get(...)` from “same-key reuse” to “congestion guess + retry”, increasing complexity and tuning surface + +Therefore the first version keeps only the most robust layer: when there is already an owner downloading one key, let followers wait briefly. + +#### Compatibility and invariants + +- `dt_filecache_wait_on_downloading_ms = 0` must preserve current behavior +- no `FileCache::mtx` is held while waiting +- only `Complete` is treated as hit; both `Failed` and timeout are treated as miss +- existing `getOrWait(...)`, vector-index / inverted-index / full-text-index foreground download paths are out of scope for this change + +#### Observability + +At least add the following counters: + +- `wait_on_downloading` +- `wait_on_downloading_hit` +- `wait_on_downloading_timeout` +- `wait_on_downloading_failed` + +To support finer-grained benefit analysis and tuning, also add a low-cardinality metric set: + +- `wait_on_downloading_result{result, file_type}` +- `wait_on_downloading_bytes{result, file_type}` +- `wait_on_downloading_wait_seconds{result, file_type}` +- `bg_download_stage_seconds{stage, file_type}` +- `bg_downloading_count` + +Suggested fixed labels: + +- `result`: `hit` / `timeout` / `failed` +- `file_type`: `merged` / `coldata` / `other` +- `stage`: `queue_wait` / `download` + +Among them: + +- `wait_on_downloading_result{...}` is used for count-based hit ratio +- `wait_on_downloading_bytes{...}` is used for bytes-weighted hit ratio, to avoid overestimating benefit from many tiny merged hits +- `wait_on_downloading_wait_seconds{...}` is used to measure actual follower waiting cost +- `bg_download_stage_seconds{stage="queue_wait"}` is the time from `bgDownload(...)` submission until `bgDownloadExecutor(...)` actually starts running +- `bg_download_stage_seconds{stage="download"}` is the time from executor start until download completes or fails +- `bg_downloading_count` is the number of currently active background downloads + +This metric set must strictly avoid high cardinality: + +- no labels by key, table name, or DMFile id +- keep only coarse-grained categories such as `Merged vs ColData vs Other`, enough to support tuning + +Current code already has natural hook points and state for these metrics: + +- the queue-full decision that causes `RejectTooManyDownloading`: + `dbms/src/Storages/S3/FileCache.cpp:887` +- current background download count `bg_downloading_count`: + `dbms/src/Storages/S3/FileCache.h:471` + `dbms/src/Storages/S3/FileCache.cpp:1106` +- stopwatch around one download: + `dbms/src/Storages/S3/FileCache.cpp:1010` + +This metric set is sufficient to answer four questions: + +- whether bounded wait really absorbs direct fallback +- whether the benefit mainly comes from `Merged` or from `ColData` +- when count hit ratio looks promising, whether bytes-weighted hit ratio still holds +- whether timeout mainly comes from queueing too long or from the object download itself being too slow + +## Rejected Directions + +### 1. Distinguish wait policy at the `FileCache` layer by `Meta / Index / Mark / Merged / ColData` + +Not adopted. Under MetaV2, this mapping does not reliably hold at the `FileCache` layer: + +- mark/index/small data often map to the physical `.merged` +- `FileCache` only sees the physical key + +If we force this into the first version, the document would become more precise than the implementation, but the implementation still would not have accurate-enough information, and the final policy and metrics would both become misleading. + +### 2. Sleep-retry on `RejectTooManyDownloading` + +Not adopted because: + +- it cannot replace a node-level traffic cap +- it is not very useful for cold reads with many unique keys +- it adds hot-path branches and new tuning knobs + +If phase 2 later shows that same-key follower dedup already works well but queue jitter is still obvious, we can evaluate a third incremental change separately. + +## Incremental Plan + +1. Add S3 remote-read config into `IORateLimitConfig`, disabled by default. +2. Add `S3ReadLimiter`; let `IORateLimiter` handle reload and `ClientFactory` publish it to `TiFlashS3Client`. +3. Wire it into `S3RandomAccessFile` and `FileCache::downloadImpl(...)`, together with phase-1 metrics and unit tests. +4. Canary phase 1 on compute nodes first, with real machine-specific limits. +5. Add bounded wait to `FileSegment` / `FileCache::get(...)`, disabled by default. +6. After phase 1 metrics become stable, enable a very small `dt_filecache_wait_on_downloading_ms` in canary. + +### Recommended initial parameters + +For the first canary, the first goal is to make sure a node cannot saturate its network because of cold-read fan-out, rather than to maximize throughput from day one. + +- `s3_max_read_bytes_per_sec` + - start from `30%~50%` of the node's sustainable outbound bandwidth budget rather than the theoretical NIC peak + - for `10GbE` nodes, start from `300~500 MiB/s`, i.e. `314572800~524288000` + - for `25GbE` nodes, start from `800~1200 MiB/s`, i.e. `838860800~1258291200` + - if the same node also serves obvious MPP exchange, page-storage, S3 background tasks, or other outbound traffic, prefer the lower end of the range + - when increasing, change by only `10%~20%` each step + +- `dt_filecache_wait_on_downloading_ms` + - keep `0` in phase 1 + - for the first phase-2 canary, start from `1` + - increase to `2` only after `wait_on_downloading_hit / wait_on_downloading` is clearly non-zero and timeout ratio is still low + - do not exceed `5` in the first version + +At phase 1, there is only one knob to stabilize: `s3_max_read_bytes_per_sec`. It directly determines how much sustained pressure a compute node can apply to S3 during a cold-read burst. + +### Rollout sequence + +Recommended four-step rollout, changing only one main variable each time: + +1. Single-node canary, phase 1 only + - suggested starting point: + - `s3_max_read_bytes_per_sec =` `30%~50%` of node budget + - `dt_filecache_wait_on_downloading_ms = 0` + - must cover at least one real cold-read peak or one reproducible wide-table cold-cache pressure test + +2. Small-batch canary, limited to one AZ, one tenant group, or no more than `5%` of compute nodes + - keep config identical to the canary + - if tuning is required, change only one parameter each time and observe at least one complete business peak window before the next change + +3. Expand to `20%~30%` of compute nodes + - only expand when phase 1 has already shown clear error reduction, node outbound traffic no longer sticks to the line, and S3 latency no longer keeps worsening + +4. Enable phase 2 on the same canary + - keep phase-1 config unchanged and add only `dt_filecache_wait_on_downloading_ms = 1` + - if phase-2 hit ratio is not obvious or timeout ratio is high, roll back to `0` without blocking phase-1 full rollout + +### Rollout criteria and rollback thresholds + +The following thresholds are recommendations for the first rollout, not contractual guarantees; their goal is to turn “whether to continue rollout” into an actionable decision. + +Before expanding phase 1, all of the following are recommended: + +- `CurrentMetrics::S3RandomAccessFile` no longer grows almost linearly with query concurrency + - “continuously above `900` for several minutes” is recommended as an alert line + - this is a trend/guardrail signal, not an exact token count +- P95/P99 of `tiflash_storage_s3_request_seconds{type="get_object"}` and `tiflash_storage_s3_request_seconds{type="read_stream"}` no longer keep worsening during bursts + - some slowdown relative to baseline is acceptable + - but if P99 degrades by more than `30%` at similar throughput and query errors do not clearly improve, rollout should not continue +- 1-minute peak node outbound traffic should stay below `85%` of the node budget + - if it still sticks near the link limit or shows obvious saw-tooth oscillation, the byte budget is still too high +- query errors caused by node network saturation should drop to `0` or near `0` + - if the error type does not clearly improve, the limiter likely did not hit the real bottleneck +- `tiflash_storage_remote_cache{type="dtfile_download_failed"}` should not rise significantly after enabling + +Phase 1 should be rolled back immediately or at least stop further rollout under any of the following conditions: + +- query error rate rises obviously, or new timeout/cancel errors increase significantly +- `CurrentMetrics::S3RandomAccessFile` still frequently approaches `1000`, indicating that direct-read pressure is still too high or there are unprotected paths +- P99 of `tiflash_storage_s3_request_seconds{type="get_object"}` or `{type="read_stream"}` keeps rising to around `2x` baseline and does not recover +- node outbound traffic still sticks to the line or turns into a new severe oscillation + +Before expanding phase 2, all of the following are recommended: + +- `wait_on_downloading_hit / wait_on_downloading >= 10%` + - if it stays below `5%` for a long time, the same-key follower dedup benefit is weak and keeping `0` is likely better +- bytes-weighted hit ratio should be stably non-zero; start with `5%` as an empirical threshold + - can be computed as `sum(wait_on_downloading_bytes{result="hit"}) / sum(wait_on_downloading_bytes)` + - if count hit ratio is high but bytes-weighted hit ratio stays below `5%`, the benefit mainly comes from small metadata / merged files and should not be expected to significantly reduce the main data traffic +- `wait_on_downloading_timeout / wait_on_downloading <= 20%` + - if timeout ratio is high, the wait window is larger than the real reusable window +- `wait_on_downloading_failed` remains at a very low level +- after enabling `1ms` wait, query P99 does not worsen by more than `10%` +- `tiflash_storage_remote_cache{type="dtfile_too_many_download"}` does not continue to rise, or direct-fallback spikes show signs of dropping +- split results between `Merged` and `ColData` match workload expectations + - if `merged` hit is high but `coldata` is low, phase 2 mainly absorbs MetaV2 small-file fallback; this still has value, but network relief is usually smaller than the count ratio suggests + - if the goal is to further reduce main data traffic, focus on `coldata` hit/timeout/bytes rather than only total hit ratio +- `bg_download_stage_seconds{stage="queue_wait"}` should not stay significantly above `bg_download_stage_seconds{stage="download"}` for a long time + - if queue wait dominates timeout, first inspect download queue size, concurrent-download config, and `RejectTooManyDownloading` pressure instead of increasing `dt_filecache_wait_on_downloading_ms` + - if actual download dominates timeout and timeout is concentrated on `coldata`, the benefit ceiling of phase 2 is inherently limited and the wait window should not keep growing in order to force more benefit + +Existing code locations for the current observability basis: + +- `CurrentMetrics::S3RandomAccessFile`: `dbms/src/Common/CurrentMetrics.cpp:88` +- `tiflash_storage_s3_request_seconds`: `dbms/src/Common/TiFlashMetrics.h:779` +- `tiflash_storage_remote_cache`: `dbms/src/Common/TiFlashMetrics.h:875` + +## Validation Strategy + +### Unit tests + +- Phase 1 + - `s3_max_read_bytes_per_sec` limits both direct reads and downloads + - behavior remains unchanged when the limiter is disabled + - lazily created S3 clients on compute nodes still pick up the current limiter + +- Phase 2 + - timeout / success / failed semantics of `waitForNotEmptyFor(...)` + - `FileCache::get(...)` can directly return the cached file via bounded wait when the key already has an `Empty` segment + - behavior remains unchanged when `dt_filecache_wait_on_downloading_ms = 0` + - no `FileCache::mtx` is held during waiting + - leader behavior on the first miss stays unchanged + - observability classification for `Merged` / `ColData` / `Other` is correct + - `wait_on_downloading_bytes{...}` matches actual waited file size accounting + - `bg_download_stage_seconds{stage="queue_wait|download"}` is accounted correctly on both success and failure paths + +### Integration tests + +- cold cache, wide table, many columns read simultaneously +- multiple threads reading the same physical `.merged` key concurrently +- node-level limiter still caps peak traffic when direct read and FileCache download coexist +- under MetaV2, mark / index / data reads all map to the `.merged` key path + +### Production success criteria + +- after phase 1 rollout, node peak outbound traffic stays stably below the budget and no longer continuously sticks near the link limit +- query failures caused by node network saturation drop to `0` or near `0` +- P95 / P99 of `tiflash_storage_s3_request_seconds{type="get_object"}` and `{type="read_stream"}` no longer keep worsening during bursts +- if phase 2 is enabled, `wait_on_downloading_hit / wait_on_downloading` should at least reach a two-digit percentage and timeout ratio should remain low +- if phase 2 is enabled, bytes-weighted hit ratio should stay stably non-zero, and the `Merged vs ColData` split should explain where the benefit comes from rather than leaving only one total hit ratio +- if timeout mainly comes from queue wait, optimize download queueing first; if timeout mainly comes from actual download, accept the benefit ceiling of phase 2 on large `ColData` files instead of treating it as a replacement for phase 1 + +## Risks and Mitigations + +- limit set too low and query latency rises + - mitigation: disabled by default; size per node; roll out in phases + +- direct read and FileCache download share the same limit and may squeeze each other + - mitigation: share one budget in the first version and do not add more layering yet; decide whether to split only after observing `source`-dimension metrics + +- bounded wait hit ratio is low and only adds tail latency + - mitigation: default `0`; start from a tiny value such as `1~2ms`; trigger only when the same key already has an in-flight download + +- new CPU overhead after enabling the limiter + - mitigation: use chunked paths only when the limiter is enabled; keep current fast path when disabled + +## Open Questions + +- if we want to constrain “active remote body streams” again in the future, what model is closer to actual network occupancy than a reader-lifetime token? +- if phase 2 hit ratio is already respectable but direct-fallback spikes still happen after `RejectTooManyDownloading`, do we need a third phase with queue-full retry? diff --git a/docs/tiflash_http_api.md b/docs/tiflash_http_api.md index a618f6e4f29..119e6221049 100644 --- a/docs/tiflash_http_api.md +++ b/docs/tiflash_http_api.md @@ -292,3 +292,55 @@ curl "http://${TIFLASH_IP}:${TIFLASH_STATUS_PORT}/tiflash/remote/cache/evict/typ "released_size":"21380742440" } ``` + +## Evict the mark cache on one TiFlash node + +```bash +curl "http://${TIFLASH_IP}:${TIFLASH_STATUS_PORT}/tiflash/cache/evict/mark" +``` + +### Response + +```json +{ + "status": "ok", + "cache": "mark" +} +``` + +If the mark cache is not enabled on the target node, the request is still +treated as a successful no-op: + +```json +{ + "status": "ok", + "cache": "mark", + "message": "cache not enabled" +} +``` + +## Evict the minmax index cache on one TiFlash node + +```bash +curl "http://${TIFLASH_IP}:${TIFLASH_STATUS_PORT}/tiflash/cache/evict/minmax" +``` + +### Response + +```json +{ + "status": "ok", + "cache": "minmax" +} +``` + +If the minmax index cache is not enabled on the target node, the request is +still treated as a successful no-op: + +```json +{ + "status": "ok", + "cache": "minmax", + "message": "cache not enabled" +} +``` diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 402ef82a129..c89d56ac423 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1774097820692, + "iteration": 1775402037950, "links": [], "panels": [ { @@ -1217,7 +1217,7 @@ "h": 7, "w": 12, "x": 0, - "y": 2 + "y": 34 }, "hiddenSeries": false, "id": 141, @@ -1329,7 +1329,7 @@ "h": 7, "w": 12, "x": 12, - "y": 2 + "y": 34 }, "hiddenSeries": false, "id": 154, @@ -1459,7 +1459,7 @@ "h": 7, "w": 12, "x": 0, - "y": 9 + "y": 41 }, "hiddenSeries": false, "id": 145, @@ -1589,7 +1589,7 @@ "h": 7, "w": 12, "x": 12, - "y": 9 + "y": 41 }, "hiddenSeries": false, "id": 147, @@ -1719,7 +1719,7 @@ "h": 7, "w": 12, "x": 0, - "y": 16 + "y": 48 }, "hiddenSeries": false, "id": 155, @@ -1849,7 +1849,7 @@ "h": 7, "w": 12, "x": 12, - "y": 16 + "y": 48 }, "hiddenSeries": false, "id": 257, @@ -1979,7 +1979,7 @@ "h": 7, "w": 12, "x": 0, - "y": 23 + "y": 55 }, "hiddenSeries": false, "id": 151, @@ -2109,7 +2109,7 @@ "h": 7, "w": 12, "x": 12, - "y": 23 + "y": 55 }, "hiddenSeries": false, "id": 156, @@ -2239,7 +2239,7 @@ "h": 7, "w": 12, "x": 0, - "y": 30 + "y": 62 }, "hiddenSeries": false, "id": 149, @@ -2369,7 +2369,7 @@ "h": 7, "w": 12, "x": 12, - "y": 30 + "y": 62 }, "hiddenSeries": false, "id": 159, @@ -2499,7 +2499,7 @@ "h": 7, "w": 12, "x": 0, - "y": 37 + "y": 69 }, "hiddenSeries": false, "id": 161, @@ -2629,7 +2629,7 @@ "h": 7, "w": 12, "x": 12, - "y": 37 + "y": 69 }, "hiddenSeries": false, "id": 256, @@ -2759,7 +2759,7 @@ "h": 7, "w": 12, "x": 0, - "y": 44 + "y": 76 }, "hiddenSeries": false, "id": 153, @@ -2889,7 +2889,7 @@ "h": 7, "w": 12, "x": 12, - "y": 44 + "y": 76 }, "hiddenSeries": false, "id": 267, @@ -3019,7 +3019,7 @@ "h": 7, "w": 12, "x": 0, - "y": 51 + "y": 83 }, "hiddenSeries": false, "id": 295, @@ -3161,7 +3161,7 @@ "h": 7, "w": 12, "x": 12, - "y": 51 + "y": 83 }, "hiddenSeries": false, "id": 268, @@ -3302,7 +3302,7 @@ "h": 7, "w": 12, "x": 0, - "y": 3 + "y": 35 }, "hiddenSeries": false, "id": 329, @@ -3415,7 +3415,7 @@ "h": 7, "w": 12, "x": 12, - "y": 3 + "y": 35 }, "hiddenSeries": false, "id": 331, @@ -3520,7 +3520,7 @@ "h": 7, "w": 12, "x": 0, - "y": 10 + "y": 42 }, "hiddenSeries": false, "id": 333, @@ -3625,7 +3625,7 @@ "h": 7, "w": 12, "x": 12, - "y": 10 + "y": 42 }, "hiddenSeries": false, "id": 335, @@ -3744,7 +3744,7 @@ "h": 7, "w": 12, "x": 0, - "y": 4 + "y": 36 }, "hiddenSeries": false, "id": 9, @@ -3845,7 +3845,7 @@ "h": 7, "w": 12, "x": 12, - "y": 4 + "y": 36 }, "hiddenSeries": false, "id": 2, @@ -3944,7 +3944,7 @@ "h": 7, "w": 12, "x": 0, - "y": 11 + "y": 43 }, "hiddenSeries": false, "id": 11, @@ -4067,7 +4067,7 @@ "h": 7, "w": 12, "x": 12, - "y": 11 + "y": 43 }, "hiddenSeries": false, "id": 12, @@ -4164,7 +4164,7 @@ "h": 7, "w": 12, "x": 0, - "y": 18 + "y": 50 }, "hiddenSeries": false, "id": 13, @@ -4282,7 +4282,7 @@ "h": 7, "w": 12, "x": 12, - "y": 18 + "y": 50 }, "hiddenSeries": false, "id": 14, @@ -4383,7 +4383,7 @@ "h": 7, "w": 12, "x": 0, - "y": 25 + "y": 57 }, "hiddenSeries": false, "id": 63, @@ -4499,7 +4499,7 @@ "h": 7, "w": 12, "x": 12, - "y": 25 + "y": 57 }, "hiddenSeries": false, "id": 165, @@ -4600,7 +4600,7 @@ "h": 7, "w": 12, "x": 0, - "y": 32 + "y": 64 }, "hiddenSeries": false, "id": 100, @@ -4699,7 +4699,7 @@ "h": 7, "w": 12, "x": 12, - "y": 32 + "y": 64 }, "hiddenSeries": false, "id": 77, @@ -4797,7 +4797,7 @@ "h": 7, "w": 12, "x": 0, - "y": 39 + "y": 71 }, "hiddenSeries": false, "id": 102, @@ -4896,7 +4896,7 @@ "h": 7, "w": 12, "x": 12, - "y": 39 + "y": 71 }, "hiddenSeries": false, "id": 101, @@ -4996,7 +4996,7 @@ "h": 7, "w": 12, "x": 0, - "y": 46 + "y": 78 }, "hiddenSeries": false, "id": 157, @@ -5095,7 +5095,7 @@ "h": 7, "w": 12, "x": 12, - "y": 46 + "y": 78 }, "hiddenSeries": false, "id": 103, @@ -5194,7 +5194,7 @@ "h": 7, "w": 12, "x": 0, - "y": 53 + "y": 85 }, "hiddenSeries": false, "id": 199, @@ -5293,7 +5293,7 @@ "h": 7, "w": 12, "x": 12, - "y": 53 + "y": 85 }, "hiddenSeries": false, "id": 166, @@ -5394,7 +5394,7 @@ "h": 7, "w": 12, "x": 0, - "y": 60 + "y": 92 }, "hiddenSeries": false, "id": 297, @@ -5494,7 +5494,7 @@ "h": 7, "w": 12, "x": 12, - "y": 60 + "y": 92 }, "hiddenSeries": false, "id": 299, @@ -5613,7 +5613,7 @@ "h": 8, "w": 12, "x": 0, - "y": 5 + "y": 37 }, "hiddenSeries": false, "id": 107, @@ -5715,7 +5715,7 @@ "h": 8, "w": 12, "x": 12, - "y": 5 + "y": 37 }, "hiddenSeries": false, "id": 109, @@ -5853,7 +5853,7 @@ "h": 8, "w": 12, "x": 0, - "y": 13 + "y": 45 }, "hiddenSeries": false, "id": 111, @@ -5967,7 +5967,7 @@ "h": 8, "w": 12, "x": 12, - "y": 13 + "y": 45 }, "hiddenSeries": false, "id": 113, @@ -6081,7 +6081,7 @@ "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 53 }, "hiddenSeries": false, "id": 117, @@ -6182,7 +6182,7 @@ "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 53 }, "hiddenSeries": false, "id": 115, @@ -6316,7 +6316,7 @@ "h": 7, "w": 12, "x": 0, - "y": 6 + "y": 38 }, "hiddenSeries": false, "id": 19, @@ -6438,7 +6438,7 @@ "h": 7, "w": 12, "x": 12, - "y": 6 + "y": 38 }, "hiddenSeries": false, "id": 18, @@ -6536,7 +6536,7 @@ "h": 7, "w": 12, "x": 0, - "y": 13 + "y": 45 }, "hiddenSeries": false, "id": 20, @@ -6689,7 +6689,7 @@ "h": 8, "w": 12, "x": 0, - "y": 7 + "y": 39 }, "hiddenSeries": false, "id": 310, @@ -6816,7 +6816,7 @@ "h": 8, "w": 12, "x": 12, - "y": 7 + "y": 39 }, "hiddenSeries": false, "id": 309, @@ -6941,7 +6941,7 @@ "h": 8, "w": 12, "x": 0, - "y": 15 + "y": 47 }, "hiddenSeries": false, "id": 316, @@ -7045,7 +7045,7 @@ "h": 8, "w": 12, "x": 12, - "y": 15 + "y": 47 }, "height": "", "hiddenSeries": false, @@ -7172,7 +7172,7 @@ "h": 8, "w": 12, "x": 0, - "y": 23 + "y": 55 }, "hiddenSeries": false, "id": 318, @@ -7286,7 +7286,7 @@ "h": 8, "w": 12, "x": 12, - "y": 23 + "y": 55 }, "height": "", "hiddenSeries": false, @@ -7423,7 +7423,7 @@ "h": 8, "w": 12, "x": 0, - "y": 8 + "y": 40 }, "hiddenSeries": false, "id": 325, @@ -7523,7 +7523,7 @@ "h": 8, "w": 12, "x": 12, - "y": 8 + "y": 40 }, "hiddenSeries": false, "id": 324, @@ -7633,7 +7633,7 @@ "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 48 }, "hiddenSeries": false, "id": 319, @@ -7743,7 +7743,7 @@ "h": 8, "w": 12, "x": 12, - "y": 16 + "y": 48 }, "hiddenSeries": false, "id": 323, @@ -7853,7 +7853,7 @@ "h": 8, "w": 12, "x": 0, - "y": 24 + "y": 56 }, "hiddenSeries": false, "id": 238, @@ -7979,7 +7979,7 @@ "h": 8, "w": 12, "x": 12, - "y": 24 + "y": 56 }, "hiddenSeries": false, "id": 169, @@ -8128,7 +8128,7 @@ "h": 8, "w": 12, "x": 0, - "y": 32 + "y": 64 }, "hiddenSeries": false, "id": 168, @@ -8252,7 +8252,7 @@ "h": 8, "w": 12, "x": 12, - "y": 32 + "y": 64 }, "hiddenSeries": false, "id": 337, @@ -8368,7 +8368,7 @@ "h": 8, "w": 12, "x": 0, - "y": 40 + "y": 72 }, "hiddenSeries": false, "id": 289, @@ -8493,7 +8493,7 @@ "h": 8, "w": 12, "x": 12, - "y": 40 + "y": 72 }, "hiddenSeries": false, "id": 272, @@ -8619,7 +8619,7 @@ "h": 8, "w": 12, "x": 0, - "y": 48 + "y": 80 }, "hiddenSeries": false, "id": 291, @@ -8728,7 +8728,7 @@ "h": 8, "w": 12, "x": 12, - "y": 48 + "y": 80 }, "hiddenSeries": false, "id": 263, @@ -8841,7 +8841,7 @@ "h": 8, "w": 12, "x": 0, - "y": 9 + "y": 41 }, "hiddenSeries": false, "id": 41, @@ -8954,7 +8954,7 @@ "h": 8, "w": 12, "x": 12, - "y": 9 + "y": 41 }, "hiddenSeries": false, "id": 38, @@ -9113,7 +9113,7 @@ "h": 8, "w": 12, "x": 0, - "y": 17 + "y": 49 }, "height": "", "hiddenSeries": false, @@ -9228,7 +9228,7 @@ "h": 8, "w": 12, "x": 12, - "y": 17 + "y": 49 }, "height": "", "hiddenSeries": false, @@ -9342,7 +9342,7 @@ "h": 5, "w": 12, "x": 0, - "y": 25 + "y": 57 }, "hiddenSeries": false, "id": 39, @@ -9445,7 +9445,7 @@ "h": 5, "w": 12, "x": 12, - "y": 25 + "y": 57 }, "hiddenSeries": false, "id": 42, @@ -9486,7 +9486,7 @@ "hide": true, "interval": "", "intervalFactor": 2, - "legendFormat": "max-{{type}} {{additional_groupby}}", + "legendFormat": "max-{{type}} {{$additional_groupby}}", "refId": "A" }, { @@ -9494,7 +9494,7 @@ "expr": "histogram_quantile(0.9999, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (le, type, $additional_groupby))", "hide": false, "interval": "", - "legendFormat": "9999-{{type}} {{additional_groupby}}", + "legendFormat": "9999-{{type}} {{$additional_groupby}}", "refId": "B" }, { @@ -9502,7 +9502,7 @@ "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (le, type, $additional_groupby))", "hide": false, "interval": "", - "legendFormat": "99-{{type}} {{additional_groupby}}", + "legendFormat": "99-{{type}} {{$additional_groupby}}", "refId": "C" } ], @@ -9565,7 +9565,7 @@ "h": 5, "w": 12, "x": 0, - "y": 30 + "y": 62 }, "hiddenSeries": false, "id": 130, @@ -9668,7 +9668,7 @@ "h": 5, "w": 12, "x": 12, - "y": 30 + "y": 62 }, "hiddenSeries": false, "id": 131, @@ -9709,7 +9709,7 @@ "hide": true, "interval": "", "intervalFactor": 2, - "legendFormat": "max-{{type}} {{additional_groupby}}", + "legendFormat": "max-{{type}} {{$additional_groupby}}", "refId": "A" }, { @@ -9717,7 +9717,7 @@ "expr": "histogram_quantile(0.9999, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (le, type, $additional_groupby))", "hide": false, "interval": "", - "legendFormat": "9999-{{type}} {{additional_groupby}}", + "legendFormat": "9999-{{type}} {{$additional_groupby}}", "refId": "B" }, { @@ -9725,7 +9725,7 @@ "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (le, type, $additional_groupby))", "hide": false, "interval": "", - "legendFormat": "99-{{type}} {{additional_groupby}}", + "legendFormat": "99-{{type}} {{$additional_groupby}}", "refId": "C" } ], @@ -9789,7 +9789,7 @@ "h": 7, "w": 24, "x": 0, - "y": 35 + "y": 67 }, "hiddenSeries": false, "id": 67, @@ -9903,7 +9903,7 @@ "h": 7, "w": 8, "x": 0, - "y": 42 + "y": 74 }, "hiddenSeries": false, "id": 50, @@ -10037,7 +10037,7 @@ "h": 7, "w": 8, "x": 8, - "y": 42 + "y": 74 }, "hiddenSeries": false, "id": 22, @@ -10151,7 +10151,7 @@ "h": 7, "w": 8, "x": 16, - "y": 42 + "y": 74 }, "hiddenSeries": false, "id": 52, @@ -10268,7 +10268,7 @@ "h": 7, "w": 12, "x": 0, - "y": 49 + "y": 81 }, "hiddenSeries": false, "id": 46, @@ -10384,7 +10384,7 @@ "h": 7, "w": 12, "x": 12, - "y": 49 + "y": 81 }, "hiddenSeries": false, "id": 47, @@ -10501,7 +10501,7 @@ "h": 8, "w": 12, "x": 0, - "y": 56 + "y": 88 }, "height": "", "hiddenSeries": false, @@ -10623,7 +10623,7 @@ "h": 8, "w": 12, "x": 12, - "y": 56 + "y": 88 }, "height": "", "hiddenSeries": false, @@ -10744,7 +10744,7 @@ "h": 7, "w": 12, "x": 0, - "y": 64 + "y": 96 }, "hiddenSeries": false, "id": 294, @@ -10850,7 +10850,7 @@ "h": 7, "w": 12, "x": 12, - "y": 64 + "y": 96 }, "hiddenSeries": false, "id": 293, @@ -10965,7 +10965,7 @@ "h": 8, "w": 12, "x": 0, - "y": 10 + "y": 42 }, "hiddenSeries": false, "id": 40, @@ -11065,7 +11065,7 @@ "h": 8, "w": 12, "x": 12, - "y": 10 + "y": 42 }, "hiddenSeries": false, "id": 88, @@ -11252,7 +11252,7 @@ "h": 8, "w": 12, "x": 0, - "y": 18 + "y": 50 }, "hiddenSeries": false, "id": 292, @@ -11381,7 +11381,7 @@ "h": 8, "w": 12, "x": 12, - "y": 18 + "y": 50 }, "hiddenSeries": false, "id": 269, @@ -11488,7 +11488,7 @@ "h": 8, "w": 8, "x": 0, - "y": 26 + "y": 58 }, "hiddenSeries": false, "id": 132, @@ -11613,7 +11613,7 @@ "h": 8, "w": 8, "x": 8, - "y": 26 + "y": 58 }, "hiddenSeries": false, "id": 361, @@ -11720,7 +11720,7 @@ "h": 8, "w": 8, "x": 16, - "y": 26 + "y": 58 }, "hiddenSeries": false, "id": 362, @@ -11839,7 +11839,7 @@ "h": 8, "w": 12, "x": 0, - "y": 34 + "y": 66 }, "hiddenSeries": false, "id": 301, @@ -11969,7 +11969,7 @@ "h": 8, "w": 12, "x": 12, - "y": 34 + "y": 66 }, "hiddenSeries": false, "id": 237, @@ -12090,7 +12090,7 @@ "h": 8, "w": 12, "x": 0, - "y": 11 + "y": 43 }, "hiddenSeries": false, "id": 128, @@ -12233,7 +12233,7 @@ "h": 8, "w": 12, "x": 12, - "y": 11 + "y": 43 }, "hiddenSeries": false, "id": 129, @@ -12350,7 +12350,7 @@ "h": 8, "w": 12, "x": 0, - "y": 19 + "y": 51 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12412,7 +12412,7 @@ "h": 8, "w": 12, "x": 12, - "y": 19 + "y": 51 }, "hiddenSeries": false, "id": 158, @@ -12548,7 +12548,7 @@ "h": 8, "w": 12, "x": 0, - "y": 27 + "y": 59 }, "hiddenSeries": false, "id": 163, @@ -12653,7 +12653,7 @@ "h": 8, "w": 12, "x": 12, - "y": 27 + "y": 59 }, "hiddenSeries": false, "id": 162, @@ -12773,7 +12773,7 @@ "h": 8, "w": 12, "x": 0, - "y": 35 + "y": 67 }, "hiddenSeries": false, "id": 164, @@ -12889,7 +12889,7 @@ "h": 8, "w": 12, "x": 12, - "y": 35 + "y": 67 }, "hiddenSeries": false, "id": 231, @@ -12996,7 +12996,7 @@ "h": 8, "w": 12, "x": 0, - "y": 43 + "y": 75 }, "height": "", "hiddenSeries": false, @@ -13109,7 +13109,7 @@ "h": 8, "w": 12, "x": 12, - "y": 43 + "y": 75 }, "hiddenSeries": false, "id": 123, @@ -13240,7 +13240,7 @@ "h": 9, "w": 24, "x": 0, - "y": 51 + "y": 83 }, "hiddenSeries": false, "id": 232, @@ -13347,7 +13347,7 @@ "h": 9, "w": 24, "x": 0, - "y": 60 + "y": 92 }, "hiddenSeries": false, "id": 345, @@ -13588,8 +13588,6 @@ "min": false, "rightSide": true, "show": true, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, @@ -13669,7 +13667,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The storage I/O limiter metrics.", + "description": "I/O Limiter current pending gauge.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -13678,12 +13676,12 @@ "fillGradient": 0, "gridPos": { "h": 8, - "w": 12, + "w": 8, "x": 0, "y": 20 }, "hiddenSeries": false, - "id": 266, + "id": 86, "legend": { "alignAsTable": true, "avg": false, @@ -13701,7 +13699,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -13712,7 +13710,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "", + "alias": "/pending/", "yaxis": 2 } ], @@ -13721,24 +13719,55 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_io_limiter_pending_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (type, instance)", + "expr": "avg(tiflash_system_current_metric_RateLimiterPendingWriteRequest{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", "format": "time_series", - "instant": false, + "hide": true, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{type}}-{{instance}}", + "intervalFactor": 1, + "legendFormat": "other-current-{{instance}}", "refId": "A" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "bgwrite-current-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "fgwrite-current-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "bgread-current-{{instance}}", + "refId": "D" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "fgread-current-{{instance}}", + "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Pending Rate", + "title": "I/O Limiter Current Pending Gauge", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -13752,11 +13781,11 @@ "yaxes": [ { "decimals": 0, - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -13779,7 +13808,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "I/O Limiter current pending count.", + "description": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -13788,16 +13817,16 @@ "fillGradient": 0, "gridPos": { "h": 8, - "w": 12, - "x": 12, + "w": 8, + "x": 8, "y": 20 }, "hiddenSeries": false, - "id": 86, + "id": 266, "legend": { "alignAsTable": true, "avg": false, - "current": false, + "current": true, "hideZero": true, "max": true, "min": false, @@ -13811,7 +13840,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -13822,7 +13851,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/pending/", + "alias": "", "yaxis": 2 } ], @@ -13831,71 +13860,24 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_system_current_metric_RateLimiterPendingWriteRequest{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_io_limiter_pending_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (type, instance)", "format": "time_series", - "hide": true, + "instant": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "other-current-{{instance}}", + "intervalFactor": 2, + "legendFormat": "{{type}}-{{instance}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "bgwrite-current-{{instance}}", - "refId": "B" - }, - { - "exemplar": true, - "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "fgwrite-current-{{instance}}", - "refId": "C" - }, - { - "exemplar": true, - "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "bgread-current-{{instance}}", - "refId": "D" - }, - { - "exemplar": true, - "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "fgread-current-{{instance}}", - "refId": "E" - }, - { - "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m]))) by (le, type) / 1000000000)", - "hide": false, - "interval": "", - "legendFormat": "{{type}}-pending-max", - "refId": "F" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (le, type))", - "hide": false, - "interval": "", - "legendFormat": "{{type}}-pending-P99", - "refId": "G" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Current Pending Count", + "title": "I/O Limiter Pending OPS", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -13909,11 +13891,11 @@ "yaxes": [ { "decimals": 0, - "format": "short", + "format": "ops", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -13929,40 +13911,159 @@ "align": false, "alignLevel": null } - } - ], - "title": "Rate Limiter", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 12 - }, - "id": 64, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The stall duration of write and delete range", + "description": "I/O Limiter pending duration.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, - "w": 24, + "w": 8, + "x": 16, + "y": 20 + }, + "hiddenSeries": false, + "id": 369, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m]))) by (le, type) / 1000000000)", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-pending-max", + "refId": "F" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (le, type))", + "hide": false, + "interval": "", + "legendFormat": "{{type}}-pending-P999", + "refId": "G" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (le, type))", + "hide": false, + "interval": "", + "legendFormat": "{{type}}-pending-P99", + "refId": "H" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "I/O Limiter Pending Duration", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 0, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Rate Limiter", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 12 + }, + "id": 64, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The stall duration of write and delete range", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 24, "x": 0, - "y": 13 + "y": 61 }, "hiddenSeries": false, "id": 62, @@ -14081,7 +14182,7 @@ "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 69 }, "height": "", "hiddenSeries": false, @@ -14200,7 +14301,7 @@ "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 69 }, "height": "", "hiddenSeries": false, @@ -14317,7 +14418,7 @@ "h": 9, "w": 24, "x": 0, - "y": 29 + "y": 77 }, "height": "", "hiddenSeries": false, @@ -14439,7 +14540,7 @@ "h": 9, "w": 24, "x": 0, - "y": 38 + "y": 86 }, "hiddenSeries": false, "id": 90, @@ -14567,7 +14668,7 @@ "h": 7, "w": 12, "x": 0, - "y": 14 + "y": 62 }, "hiddenSeries": false, "id": 167, @@ -14667,7 +14768,7 @@ "h": 7, "w": 12, "x": 12, - "y": 14 + "y": 62 }, "hiddenSeries": false, "id": 35, @@ -14765,7 +14866,7 @@ "h": 7, "w": 12, "x": 0, - "y": 21 + "y": 69 }, "hiddenSeries": false, "id": 270, @@ -14865,7 +14966,7 @@ "h": 7, "w": 12, "x": 12, - "y": 21 + "y": 69 }, "hiddenSeries": false, "id": 271, @@ -14965,7 +15066,7 @@ "h": 7, "w": 12, "x": 0, - "y": 28 + "y": 76 }, "hiddenSeries": false, "id": 37, @@ -15109,7 +15210,7 @@ "h": 7, "w": 12, "x": 12, - "y": 28 + "y": 76 }, "hiddenSeries": false, "id": 36, @@ -15237,7 +15338,7 @@ "h": 7, "w": 24, "x": 0, - "y": 35 + "y": 83 }, "hiddenSeries": false, "id": 82, @@ -15396,7 +15497,7 @@ "h": 7, "w": 12, "x": 0, - "y": 42 + "y": 90 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15466,7 +15567,7 @@ "h": 7, "w": 12, "x": 12, - "y": 42 + "y": 90 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15536,7 +15637,7 @@ "h": 7, "w": 12, "x": 0, - "y": 49 + "y": 97 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15608,7 +15709,7 @@ "h": 7, "w": 12, "x": 12, - "y": 49 + "y": 97 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15670,7 +15771,7 @@ "h": 7, "w": 12, "x": 0, - "y": 56 + "y": 104 }, "hiddenSeries": false, "id": 235, @@ -15770,7 +15871,7 @@ "h": 7, "w": 12, "x": 12, - "y": 56 + "y": 104 }, "hiddenSeries": false, "id": 241, @@ -15878,7 +15979,7 @@ "h": 7, "w": 12, "x": 0, - "y": 63 + "y": 111 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15959,7 +16060,7 @@ "h": 7, "w": 12, "x": 12, - "y": 63 + "y": 111 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16032,7 +16133,7 @@ "h": 7, "w": 12, "x": 0, - "y": 70 + "y": 118 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16115,7 +16216,7 @@ "h": 7, "w": 12, "x": 12, - "y": 70 + "y": 118 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16188,7 +16289,7 @@ "h": 7, "w": 24, "x": 0, - "y": 77 + "y": 125 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16253,7 +16354,7 @@ "h": 7, "w": 12, "x": 0, - "y": 84 + "y": 132 }, "hiddenSeries": false, "id": 240, @@ -16357,7 +16458,7 @@ "h": 7, "w": 12, "x": 12, - "y": 84 + "y": 132 }, "hiddenSeries": false, "id": 239, @@ -16495,7 +16596,7 @@ "h": 7, "w": 24, "x": 0, - "y": 91 + "y": 139 }, "height": "", "hiddenSeries": false, @@ -16605,7 +16706,7 @@ "h": 7, "w": 24, "x": 0, - "y": 98 + "y": 146 }, "height": "", "hiddenSeries": false, @@ -16721,7 +16822,7 @@ "h": 7, "w": 12, "x": 0, - "y": 105 + "y": 153 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16787,7 +16888,7 @@ "h": 7, "w": 12, "x": 12, - "y": 105 + "y": 153 }, "hiddenSeries": false, "id": 91, @@ -16915,7 +17016,7 @@ "h": 7, "w": 12, "x": 0, - "y": 112 + "y": 160 }, "hiddenSeries": false, "id": 296, @@ -17035,7 +17136,7 @@ "h": 7, "w": 24, "x": 0, - "y": 15 + "y": 63 }, "hiddenSeries": false, "id": 242, @@ -17139,7 +17240,7 @@ "h": 7, "w": 24, "x": 0, - "y": 22 + "y": 70 }, "hiddenSeries": false, "id": 75, @@ -17273,7 +17374,7 @@ "h": 7, "w": 12, "x": 0, - "y": 29 + "y": 77 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17339,7 +17440,7 @@ "h": 7, "w": 12, "x": 12, - "y": 29 + "y": 77 }, "hiddenSeries": false, "id": 249, @@ -17445,7 +17546,7 @@ "h": 7, "w": 12, "x": 0, - "y": 36 + "y": 84 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17517,7 +17618,7 @@ "h": 7, "w": 12, "x": 12, - "y": 36 + "y": 84 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17587,7 +17688,7 @@ "h": 7, "w": 12, "x": 0, - "y": 43 + "y": 91 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17659,7 +17760,7 @@ "h": 7, "w": 12, "x": 12, - "y": 43 + "y": 91 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17729,7 +17830,7 @@ "h": 7, "w": 12, "x": 0, - "y": 50 + "y": 98 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17802,7 +17903,7 @@ "h": 7, "w": 12, "x": 12, - "y": 50 + "y": 98 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17871,7 +17972,7 @@ "h": 7, "w": 12, "x": 0, - "y": 57 + "y": 105 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17948,7 +18049,7 @@ "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 64 }, "hiddenSeries": false, "id": 99, @@ -18101,7 +18202,7 @@ "h": 8, "w": 12, "x": 12, - "y": 16 + "y": 64 }, "heatmap": {}, "hideZeroBuckets": true, @@ -18179,7 +18280,7 @@ "h": 8, "w": 12, "x": 0, - "y": 25 + "y": 33 }, "hiddenSeries": false, "id": 187, @@ -18299,7 +18400,7 @@ "h": 8, "w": 12, "x": 12, - "y": 25 + "y": 33 }, "height": "", "hiddenSeries": false, @@ -18418,7 +18519,7 @@ "h": 8, "w": 12, "x": 0, - "y": 33 + "y": 41 }, "height": "", "hiddenSeries": false, @@ -18528,7 +18629,7 @@ "h": 8, "w": 12, "x": 12, - "y": 33 + "y": 41 }, "height": "", "hiddenSeries": false, @@ -18641,7 +18742,7 @@ "h": 8, "w": 12, "x": 0, - "y": 41 + "y": 49 }, "hiddenSeries": false, "id": 176, @@ -18749,7 +18850,7 @@ "h": 8, "w": 12, "x": 12, - "y": 41 + "y": 49 }, "hiddenSeries": false, "id": 175, @@ -18876,7 +18977,7 @@ "h": 8, "w": 12, "x": 0, - "y": 49 + "y": 57 }, "hiddenSeries": false, "id": 189, @@ -18978,7 +19079,7 @@ "h": 8, "w": 12, "x": 12, - "y": 49 + "y": 57 }, "hiddenSeries": false, "id": 191, @@ -19083,7 +19184,7 @@ "h": 8, "w": 8, "x": 0, - "y": 57 + "y": 65 }, "hiddenSeries": false, "id": 365, @@ -19189,7 +19290,7 @@ "h": 8, "w": 9, "x": 8, - "y": 57 + "y": 65 }, "hiddenSeries": false, "id": 193, @@ -19316,7 +19417,7 @@ "h": 8, "w": 7, "x": 17, - "y": 57 + "y": 65 }, "hiddenSeries": false, "id": 195, @@ -19427,7 +19528,7 @@ "h": 8, "w": 12, "x": 0, - "y": 65 + "y": 73 }, "hiddenSeries": false, "id": 363, @@ -19533,7 +19634,7 @@ "h": 8, "w": 12, "x": 12, - "y": 65 + "y": 73 }, "hiddenSeries": false, "id": 364, @@ -19636,7 +19737,7 @@ "h": 8, "w": 12, "x": 0, - "y": 73 + "y": 81 }, "hiddenSeries": false, "id": 251, @@ -19743,7 +19844,7 @@ "h": 8, "w": 12, "x": 12, - "y": 73 + "y": 81 }, "hiddenSeries": false, "id": 252, @@ -19850,7 +19951,7 @@ "h": 8, "w": 12, "x": 0, - "y": 81 + "y": 89 }, "hiddenSeries": false, "id": 254, @@ -19957,7 +20058,7 @@ "h": 8, "w": 12, "x": 12, - "y": 81 + "y": 89 }, "hiddenSeries": false, "id": 253, @@ -20078,7 +20179,7 @@ "h": 8, "w": 24, "x": 0, - "y": 18 + "y": 34 }, "hiddenSeries": false, "id": 173, @@ -20179,7 +20280,7 @@ "h": 8, "w": 12, "x": 0, - "y": 26 + "y": 42 }, "hiddenSeries": false, "id": 185, @@ -20309,7 +20410,7 @@ "h": 8, "w": 12, "x": 12, - "y": 26 + "y": 42 }, "hiddenSeries": false, "id": 186, @@ -20403,7 +20504,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Remote Cache Usage", "fieldConfig": { "defaults": {}, "overrides": [] @@ -20414,34 +20514,33 @@ "h": 8, "w": 12, "x": 0, - "y": 34 + "y": 50 }, "hiddenSeries": false, - "id": 188, + "id": 373, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideZero": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, - "sort": "current", + "sideWidth": null, + "sort": "max", "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -20451,43 +20550,36 @@ "targets": [ { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_DTFileCacheCapacity{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", - "format": "time_series", - "hide": false, + "expr": "histogram_quantile(0.999, sum(rate(tiflash_storage_remote_cache_bg_download_stage_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (le, stage, file_type, $additional_groupby))", + "hide": true, "interval": "", - "legendFormat": "DTFileCapacity-{{instance}}", + "legendFormat": "999%-{{stage}}-{{file_type}} {{$additional_groupby}}", + "queryType": "randomWalk", "refId": "B" }, { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_DTFileCacheUsed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "DTFileUsed-{{instance}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_PageCacheCapacity{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_remote_cache_bg_download_stage_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (le, stage, file_type, $additional_groupby))", "hide": false, "interval": "", - "legendFormat": "PageCapacity-{{instance}}", - "refId": "C" + "legendFormat": "99%-{{stage}}-{{file_type}} {{$additional_groupby}}", + "queryType": "randomWalk", + "refId": "D" }, { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_PageCacheUsed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "expr": "(sum(rate(\n tiflash_storage_remote_cache_bg_download_stage_seconds_sum\n {k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}\n [$__rate_interval]\n)) by (stage, file_type, $additional_groupby) / sum(rate(\n tiflash_storage_remote_cache_bg_download_stage_seconds_count\n {k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}\n [$__rate_interval]\n)) by (stage, file_type, $additional_groupby) )", "hide": false, "interval": "", - "legendFormat": "PageUsed-{{instance}}", - "refId": "D" + "legendFormat": "avg-{{stage}}-{{file_type}} {{$additional_groupby}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote Cache Usage", + "title": "Remote Cache BG Download Duration", "tooltip": { "shared": true, "sort": 2, @@ -20503,8 +20595,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -20512,12 +20603,12 @@ "show": true }, { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": false + "min": null, + "show": true } ], "yaxis": { @@ -20531,7 +20622,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Memory Usage of Storage Tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -20542,7 +20632,677 @@ "h": 8, "w": 12, "x": 12, - "y": 34 + "y": 50 + }, + "hiddenSeries": false, + "id": 375, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(tiflash_storage_remote_cache_wait_on_downloading_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (le, result, file_type, $additional_groupby))", + "hide": true, + "interval": "", + "legendFormat": "999%-{{result}}-{{file_type}} {{$additional_groupby}}", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_remote_cache_wait_on_downloading_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (le, result, file_type, $additional_groupby))", + "hide": false, + "interval": "", + "legendFormat": "99%-{{result}}-{{file_type}} {{$additional_groupby}}", + "queryType": "randomWalk", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote Cache Wait on Downloading Duration", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 58 + }, + "hiddenSeries": false, + "id": 371, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_remote_cache_wait_on_downloading_result{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (result, file_type , $additional_groupby)", + "format": "time_series", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{result}}-{{file_type}} {{$additional_groupby}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote Cache Wait on Downloading OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 0, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 58 + }, + "hiddenSeries": false, + "id": 372, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_remote_cache_wait_on_downloading_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (result, file_type , $additional_groupby)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{result}}-{{file_type}} {{$additional_groupby}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote Cache Wait on Downloading Flow", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 66 + }, + "hiddenSeries": false, + "id": 370, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(tiflash_storage_remote_cache_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (type, instance)", + "format": "time_series", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote Cache Gauge", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 0, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 66 + }, + "hiddenSeries": false, + "id": 374, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_remote_cache_reject{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}[1m])) by (reason, file_type, $additional_groupby)", + "format": "time_series", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{reason}}-{{file_type}} {{$additional_groupby}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote Cache Reject Download Type OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 0, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Remote Cache Usage", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 74 + }, + "hiddenSeries": false, + "id": 188, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_DTFileCacheCapacity{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "legendFormat": "DTFileCapacity-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_DTFileCacheUsed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "DTFileUsed-{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_PageCacheCapacity{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "PageCapacity-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_PageCacheUsed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", instance=~\"$tiflash_role\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "PageUsed-{{instance}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote Cache Usage", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Memory Usage of Storage Tasks", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 74 }, "hiddenSeries": false, "id": 233, @@ -20667,7 +21427,7 @@ "h": 7, "w": 12, "x": 0, - "y": 42 + "y": 82 }, "hiddenSeries": false, "id": 236, @@ -20783,7 +21543,7 @@ "h": 7, "w": 12, "x": 12, - "y": 42 + "y": 82 }, "hiddenSeries": false, "id": 356, @@ -20903,7 +21663,7 @@ "h": 7, "w": 12, "x": 0, - "y": 49 + "y": 89 }, "hiddenSeries": false, "id": 353, @@ -21013,7 +21773,7 @@ "h": 7, "w": 12, "x": 12, - "y": 49 + "y": 89 }, "hiddenSeries": false, "id": 358, @@ -22300,7 +23060,7 @@ "h": 8, "w": 12, "x": 0, - "y": 20 + "y": 162 }, "hiddenSeries": false, "id": 224, @@ -22400,7 +23160,7 @@ "h": 8, "w": 12, "x": 12, - "y": 20 + "y": 162 }, "hiddenSeries": false, "id": 226, @@ -22508,7 +23268,7 @@ "h": 8, "w": 12, "x": 0, - "y": 28 + "y": 170 }, "hiddenSeries": false, "id": 228, @@ -22611,7 +23371,7 @@ "h": 8, "w": 12, "x": 12, - "y": 28 + "y": 170 }, "hiddenSeries": false, "id": 222, @@ -22784,7 +23544,7 @@ "h": 8, "w": 12, "x": 0, - "y": 36 + "y": 178 }, "hiddenSeries": false, "id": 230, @@ -22939,7 +23699,7 @@ "h": 8, "w": 12, "x": 12, - "y": 36 + "y": 178 }, "hiddenSeries": false, "id": 218, @@ -23069,7 +23829,7 @@ "h": 8, "w": 12, "x": 0, - "y": 44 + "y": 186 }, "hiddenSeries": false, "id": 220, @@ -23199,7 +23959,7 @@ "h": 8, "w": 12, "x": 12, - "y": 44 + "y": 186 }, "hiddenSeries": false, "id": 216, @@ -23325,7 +24085,7 @@ "h": 8, "w": 12, "x": 0, - "y": 52 + "y": 194 }, "hiddenSeries": false, "id": 300, @@ -23448,7 +24208,7 @@ "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 203 }, "hiddenSeries": false, "id": 246, @@ -23615,7 +24375,7 @@ "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 203 }, "hiddenSeries": false, "id": 201, @@ -23773,7 +24533,7 @@ "h": 7, "w": 12, "x": 0, - "y": 22 + "y": 204 }, "hiddenSeries": false, "id": 338, @@ -23893,7 +24653,7 @@ "h": 7, "w": 12, "x": 12, - "y": 22 + "y": 204 }, "hiddenSeries": false, "id": 341, @@ -24023,7 +24783,7 @@ "h": 8, "w": 12, "x": 0, - "y": 23 + "y": 205 }, "hiddenSeries": false, "id": 286, @@ -24144,7 +24904,7 @@ "h": 8, "w": 12, "x": 12, - "y": 23 + "y": 205 }, "hiddenSeries": false, "id": 288, @@ -24273,7 +25033,7 @@ "h": 8, "w": 12, "x": 0, - "y": 31 + "y": 213 }, "hiddenSeries": false, "id": 282, @@ -24400,7 +25160,7 @@ "h": 8, "w": 12, "x": 12, - "y": 31 + "y": 213 }, "hiddenSeries": false, "id": 284, @@ -24496,7 +25256,7 @@ "type": "row" } ], - "refresh": "30s", + "refresh": "1m", "schemaVersion": 27, "style": "dark", "tags": [], @@ -24613,7 +25373,7 @@ { "allValue": null, "current": { - "selected": true, + "selected": false, "text": "none", "value": "none" }, @@ -24644,7 +25404,7 @@ { "allValue": null, "current": { - "selected": true, + "selected": false, "text": "All", "value": ".*" },