From b955a5e81c6a9d5d0c63e727fa53f717b6abd817 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Tue, 1 Jul 2025 22:46:59 +0800 Subject: [PATCH 01/10] Save changes Signed-off-by: JaySon-Huang --- .../DeltaMerge/ColumnFile/ColumnFile.h | 1 + .../ColumnFile/ColumnFileInMemory.cpp | 11 ++ .../ColumnFile/ColumnFileInMemory.h | 1 + .../DeltaMerge/Delta/DeltaValueSpace.cpp | 7 + .../DeltaMerge/Delta/DeltaValueSpace.h | 3 +- .../Storages/DeltaMerge/Delta/MemTableSet.cpp | 10 +- .../Storages/DeltaMerge/Delta/MemTableSet.h | 12 +- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 99 +------------ .../DeltaMerge/DeltaMergeStore_Statistics.cpp | 9 +- .../DeltaMerge/DeltaMergeStore_Statistics.h | 121 ++++++++++++++++ .../Storages/DeltaMerge/File/DMFileMetaV2.cpp | 2 +- .../DeltaMerge/StoragePool/StoragePool.cpp | 14 +- .../DeltaMerge/tests/gtest_segment.cpp | 135 ++++++++++++++++++ .../tests/gtest_segment_test_basic.cpp | 25 +++- .../System/StorageSystemDTSegments.cpp | 1 + dbms/src/TestUtils/gtests_dbms_main.cpp | 2 +- 16 files changed, 331 insertions(+), 122 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h index 0142e343a46..bea914daca0 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h @@ -94,6 +94,7 @@ class ColumnFile virtual size_t getRows() const { return 0; } virtual size_t getBytes() const { return 0; } + virtual size_t getAllocateBytes() const { return 0; } virtual size_t getDeletes() const { return 0; } virtual Type getType() const = 0; diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp index 515a4a3c705..29dd392367d 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp @@ -83,16 +83,27 @@ bool ColumnFileInMemory::append( || cache->block.bytes() >= context.delta_cache_limit_bytes) return false; + size_t new_alloc_block_bytes = 0; for (size_t i = 0; i < cache->block.columns(); ++i) { const auto & col = data.getByPosition(i).column; const auto & cache_col = *cache->block.getByPosition(i).column; auto * mutable_cache_col = const_cast(&cache_col); + size_t alloc_bytes = mutable_cache_col->allocatedBytes(); mutable_cache_col->insertRangeFrom(*col, offset, limit); + new_alloc_block_bytes += mutable_cache_col->allocatedBytes() - alloc_bytes; } rows += limit; bytes += data_bytes; + LOG_INFO( + Logger::get(), + "Append rows to ColumnFileInMemory, new_rows={} new_bytes={} new_alloc_bytes={} total_rows={} total_bytes={}", + limit - offset, + data_bytes, + new_alloc_block_bytes, + rows, + bytes); return true; } diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h index 5728ef585d6..fb19ae691ac 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h @@ -67,6 +67,7 @@ class ColumnFileInMemory : public ColumnFile size_t getRows() const override { return rows; } size_t getBytes() const override { return bytes; } + size_t getAllocateBytes() const override { return cache->block.allocatedBytes(); } CachePtr getCache() { return cache; } diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp index 808184a3082..250b673ece7 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp @@ -277,8 +277,15 @@ size_t DeltaValueSpace::getTotalCacheBytes() const return mem_table_set->getBytes(); } +size_t DeltaValueSpace::getTotalAllocatedBytes() const +{ + std::scoped_lock lock(mutex); + return mem_table_set->getAllocatedBytes(); +} + size_t DeltaValueSpace::getValidCacheRows() const { + // FIXME: Seems that this function is the same as getTotalCacheRows(). std::scoped_lock lock(mutex); return mem_table_set->getRows(); } diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h index 7d7fdbfa12d..7063fcd28fe 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -85,7 +85,7 @@ class DeltaValueSpace /// Note that it's safe to do multiple flush concurrently but only one of them can succeed, /// and other thread's work is just a waste of resource. - /// So we only allow one flush task running at any time to aviod waste resource. + /// So we only allow one flush task running at any time to avoid waste resource. std::atomic_bool is_flushing = false; std::atomic last_try_flush_rows = 0; @@ -220,6 +220,7 @@ class DeltaValueSpace size_t getTotalCacheRows() const; size_t getTotalCacheBytes() const; + size_t getTotalAllocatedBytes() const; size_t getValidCacheRows() const; bool isFlushing() const { return is_flushing; } diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp index 8f36ed01dfc..8f421224fca 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp @@ -41,6 +41,7 @@ void MemTableSet::appendColumnFileInner(const ColumnFilePtr & column_file) rows += column_file->getRows(); bytes += column_file->getBytes(); + allocated_bytes += column_file->getAllocateBytes(); deletes += column_file->getDeletes(); } @@ -181,7 +182,6 @@ void MemTableSet::appendColumnFile(const ColumnFilePtr & column_file) void MemTableSet::appendToCache(DMContext & context, const Block & block, size_t offset, size_t limit) { // If the `column_files` is not empty, and the last `column_file` is a `ColumnInMemoryFile`, we will merge the newly block into the last `column_file`. - // Otherwise, create a new `ColumnInMemoryFile` and write into it. bool success = false; size_t append_bytes = block.bytes(offset, limit); if (!column_files.empty()) @@ -193,8 +193,10 @@ void MemTableSet::appendToCache(DMContext & context, const Block & block, size_t if (!success) { - auto schema = getSharedBlockSchemas(context)->getOrCreate(block); + /// Otherwise, create a new `ColumnInMemoryFile` and write into it. + // Try to reuse the global shared schema block. + auto schema = getSharedBlockSchemas(context)->getOrCreate(block); // Create a new column file. auto new_column_file = std::make_shared(schema); // Must append the empty `new_column_file` to `column_files` before appending data to it, @@ -206,6 +208,7 @@ void MemTableSet::appendToCache(DMContext & context, const Block & block, size_t } rows += limit; bytes += append_bytes; + } void MemTableSet::appendDeleteRange(const RowKeyRange & delete_range) @@ -340,6 +343,7 @@ void MemTableSet::removeColumnFilesInFlushTask(const ColumnFileFlushTask & flush size_t new_rows = 0; size_t new_bytes = 0; + size_t new_alloc_bytes = 0; size_t new_deletes = 0; for (size_t i = tasks.size(); i < column_files.size(); ++i) { @@ -347,12 +351,14 @@ void MemTableSet::removeColumnFilesInFlushTask(const ColumnFileFlushTask & flush new_column_files.emplace_back(column_file); new_rows += column_file->getRows(); new_bytes += column_file->getBytes(); + new_alloc_bytes += column_file->getAllocateBytes(); new_deletes += column_file->getDeletes(); } column_files.swap(new_column_files); column_files_count = column_files.size(); rows = new_rows; bytes = new_bytes; + allocated_bytes = new_alloc_bytes; deletes = new_deletes; } diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h index 2b91f5855c9..e68cd98c49b 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h @@ -18,9 +18,7 @@ #include #include -namespace DB -{ -namespace DM +namespace DB::DM { class MemTableSet; using MemTableSetPtr = std::shared_ptr; @@ -41,11 +39,13 @@ class MemTableSet #endif // Note that we must update `column_files_count` for outer thread-safe after `column_files` changed ColumnFiles column_files; + + // In order to avoid data-race, we use atomic variables to track the state of this MemTableSet. // TODO: check the proper memory_order when use this atomic variable std::atomic column_files_count; - std::atomic rows = 0; std::atomic bytes = 0; + std::atomic allocated_bytes = 0; std::atomic deletes = 0; LoggerPtr log; @@ -88,6 +88,7 @@ class MemTableSet size_t getColumnFileCount() const { return column_files_count.load(); } size_t getRows() const { return rows.load(); } size_t getBytes() const { return bytes.load(); } + size_t getAllocatedBytes() const { return allocated_bytes.load(); } size_t getDeletes() const { return deletes.load(); } /// Thread safe part end @@ -153,5 +154,4 @@ class MemTableSet void removeColumnFilesInFlushTask(const ColumnFileFlushTask & flush_task); }; -} // namespace DM -} // namespace DB +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index ab289840ec3..4e216d1ffa1 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -76,104 +77,6 @@ namespace tests class DeltaMergeStoreTest; } -struct SegmentStats -{ - UInt64 segment_id = 0; - RowKeyRange range; - UInt64 epoch = 0; - UInt64 rows = 0; - UInt64 size = 0; - - Float64 delta_rate = 0; - UInt64 delta_memtable_rows = 0; - UInt64 delta_memtable_size = 0; - UInt64 delta_memtable_column_files = 0; - UInt64 delta_memtable_delete_ranges = 0; - UInt64 delta_persisted_page_id = 0; - UInt64 delta_persisted_rows = 0; - UInt64 delta_persisted_size = 0; - UInt64 delta_persisted_column_files = 0; - UInt64 delta_persisted_delete_ranges = 0; - UInt64 delta_cache_size = 0; - UInt64 delta_index_size = 0; - - UInt64 stable_page_id = 0; - UInt64 stable_rows = 0; - UInt64 stable_size = 0; - UInt64 stable_dmfiles = 0; - UInt64 stable_dmfiles_id_0 = 0; - UInt64 stable_dmfiles_rows = 0; - UInt64 stable_dmfiles_size = 0; - UInt64 stable_dmfiles_size_on_disk = 0; - UInt64 stable_dmfiles_packs = 0; -}; -using SegmentsStats = std::vector; - -struct StoreStats -{ - UInt64 segment_count = 0; - - UInt64 total_rows = 0; - UInt64 total_size = 0; - UInt64 total_delete_ranges = 0; - - Float64 delta_rate_rows = 0; - Float64 delta_rate_segments = 0; - - Float64 delta_placed_rate = 0; - UInt64 delta_cache_size = 0; - Float64 delta_cache_rate = 0; - Float64 delta_cache_wasted_rate = 0; - - UInt64 delta_index_size = 0; - - Float64 avg_segment_rows = 0; - Float64 avg_segment_size = 0; - - UInt64 delta_count = 0; - UInt64 total_delta_rows = 0; - UInt64 total_delta_size = 0; - Float64 avg_delta_rows = 0; - Float64 avg_delta_size = 0; - Float64 avg_delta_delete_ranges = 0; - - UInt64 stable_count = 0; - UInt64 total_stable_rows = 0; - UInt64 total_stable_size = 0; - UInt64 total_stable_size_on_disk = 0; - Float64 avg_stable_rows = 0; - Float64 avg_stable_size = 0; - - // statistics about column file in delta - UInt64 total_pack_count_in_delta = 0; - UInt64 max_pack_count_in_delta = 0; - Float64 avg_pack_count_in_delta = 0; - Float64 avg_pack_rows_in_delta = 0; - Float64 avg_pack_size_in_delta = 0; - - UInt64 total_pack_count_in_stable = 0; - Float64 avg_pack_count_in_stable = 0; - Float64 avg_pack_rows_in_stable = 0; - Float64 avg_pack_size_in_stable = 0; - - UInt64 storage_stable_num_snapshots = 0; - Float64 storage_stable_oldest_snapshot_lifetime = 0.0; - UInt64 storage_stable_oldest_snapshot_thread_id = 0; - String storage_stable_oldest_snapshot_tracing_id; - - UInt64 storage_delta_num_snapshots = 0; - Float64 storage_delta_oldest_snapshot_lifetime = 0.0; - UInt64 storage_delta_oldest_snapshot_thread_id = 0; - String storage_delta_oldest_snapshot_tracing_id; - - UInt64 storage_meta_num_snapshots = 0; - Float64 storage_meta_oldest_snapshot_lifetime = 0.0; - UInt64 storage_meta_oldest_snapshot_thread_id = 0; - String storage_meta_oldest_snapshot_tracing_id; - - UInt64 background_tasks_length = 0; -}; - struct LocalIndexStats { UInt64 column_id{}; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp index 887d0aaaf03..d6b9fb5633e 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -13,15 +13,14 @@ // limitations under the License. #include +#include #include #include #include #include #include -namespace DB -{ -namespace DM +namespace DB::DM { StoreStats DeltaMergeStore::getStoreStats() @@ -176,6 +175,7 @@ SegmentsStats DeltaMergeStore::getSegmentsStats() stat.delta_persisted_column_files = delta_persisted->getColumnFileCount(); stat.delta_persisted_delete_ranges = delta_persisted->getDeletes(); stat.delta_cache_size = delta->getTotalCacheBytes(); + stat.delta_cache_alloc_size = delta->getTotalAllocatedBytes(); stat.delta_index_size = delta->getDeltaIndexBytes(); stat.stable_page_id = stable->getId(); @@ -294,5 +294,4 @@ LocalIndexesStats DeltaMergeStore::getLocalIndexStats() return stats; } -} // namespace DM -} // namespace DB +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h new file mode 100644 index 00000000000..1d26ecc19de --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h @@ -0,0 +1,121 @@ +// 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 + +namespace DB::DM +{ + +struct SegmentStats +{ + UInt64 segment_id = 0; + RowKeyRange range; + UInt64 epoch = 0; + UInt64 rows = 0; + UInt64 size = 0; + + Float64 delta_rate = 0; + UInt64 delta_memtable_rows = 0; + UInt64 delta_memtable_size = 0; + UInt64 delta_memtable_column_files = 0; + UInt64 delta_memtable_delete_ranges = 0; + UInt64 delta_persisted_page_id = 0; + UInt64 delta_persisted_rows = 0; + UInt64 delta_persisted_size = 0; + UInt64 delta_persisted_column_files = 0; + UInt64 delta_persisted_delete_ranges = 0; + UInt64 delta_cache_size = 0; + UInt64 delta_cache_alloc_size = 0; + UInt64 delta_index_size = 0; + + UInt64 stable_page_id = 0; + UInt64 stable_rows = 0; + UInt64 stable_size = 0; + UInt64 stable_dmfiles = 0; + UInt64 stable_dmfiles_id_0 = 0; + UInt64 stable_dmfiles_rows = 0; + UInt64 stable_dmfiles_size = 0; + UInt64 stable_dmfiles_size_on_disk = 0; + UInt64 stable_dmfiles_packs = 0; +}; +using SegmentsStats = std::vector; + +struct StoreStats +{ + UInt64 segment_count = 0; + + UInt64 total_rows = 0; + UInt64 total_size = 0; + UInt64 total_delete_ranges = 0; + + Float64 delta_rate_rows = 0; + Float64 delta_rate_segments = 0; + + Float64 delta_placed_rate = 0; + UInt64 delta_cache_size = 0; + Float64 delta_cache_rate = 0; + Float64 delta_cache_wasted_rate = 0; + + UInt64 delta_index_size = 0; + + Float64 avg_segment_rows = 0; + Float64 avg_segment_size = 0; + + UInt64 delta_count = 0; + UInt64 total_delta_rows = 0; + UInt64 total_delta_size = 0; + Float64 avg_delta_rows = 0; + Float64 avg_delta_size = 0; + Float64 avg_delta_delete_ranges = 0; + + UInt64 stable_count = 0; + UInt64 total_stable_rows = 0; + UInt64 total_stable_size = 0; + UInt64 total_stable_size_on_disk = 0; + Float64 avg_stable_rows = 0; + Float64 avg_stable_size = 0; + + // statistics about column file in delta + UInt64 total_pack_count_in_delta = 0; + UInt64 max_pack_count_in_delta = 0; + Float64 avg_pack_count_in_delta = 0; + Float64 avg_pack_rows_in_delta = 0; + Float64 avg_pack_size_in_delta = 0; + + UInt64 total_pack_count_in_stable = 0; + Float64 avg_pack_count_in_stable = 0; + Float64 avg_pack_rows_in_stable = 0; + Float64 avg_pack_size_in_stable = 0; + + UInt64 storage_stable_num_snapshots = 0; + Float64 storage_stable_oldest_snapshot_lifetime = 0.0; + UInt64 storage_stable_oldest_snapshot_thread_id = 0; + String storage_stable_oldest_snapshot_tracing_id; + + UInt64 storage_delta_num_snapshots = 0; + Float64 storage_delta_oldest_snapshot_lifetime = 0.0; + UInt64 storage_delta_oldest_snapshot_thread_id = 0; + String storage_delta_oldest_snapshot_tracing_id; + + UInt64 storage_meta_num_snapshots = 0; + Float64 storage_meta_oldest_snapshot_lifetime = 0.0; + UInt64 storage_meta_oldest_snapshot_thread_id = 0; + String storage_meta_oldest_snapshot_tracing_id; + + UInt64 background_tasks_length = 0; +}; +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp index 700c6bb214a..2c92f319c41 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp @@ -284,7 +284,7 @@ void DMFileMetaV2::read(const FileProviderPtr & file_provider, const ReadMode & { break; } - LOG_WARNING(log, "{}'s size is larger than {}", metaPath(), buf.size()); + // LOG_WARNING(log, "{}'s size is larger than {}", metaPath(), buf.size()); buf.resize(buf.size() + meta_buffer_size); } buf.resize(read_bytes); diff --git a/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp b/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp index fc1abb9df02..bb757c308a9 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp +++ b/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp @@ -867,7 +867,7 @@ void StoragePool::drop() } } -PageIdU64 StoragePool::newDataPageIdForDTFile(StableDiskDelegator & delegator, const char * who) const +PageIdU64 StoragePool::newDataPageIdForDTFile(StableDiskDelegator & delegator, [[maybe_unused]] const char * who) const { // In case that there is a DTFile created on disk but TiFlash crashes without persisting the ID. // After TiFlash process restored, the ID will be inserted into the stable delegator, but we may @@ -891,12 +891,12 @@ PageIdU64 StoragePool::newDataPageIdForDTFile(StableDiskDelegator & delegator, c break; } // else there is a DTFile with that id, continue to acquire a new ID. - LOG_WARNING( - logger, - "The DTFile is already exists, continute to acquire another ID. call={} path={} file_id={}", - who, - existed_path, - dtfile_id); + // LOG_WARNING( + // logger, + // "The DTFile is already exists, continue to acquire another ID. call={} path={} file_id={}", + // who, + // existed_path, + // dtfile_id); } while (true); return dtfile_id; } diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp index 5cdf249fe0f..d7db0b01a5f 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp @@ -607,6 +607,141 @@ try } CATCH +namespace +{ +struct ProcessMemoryUsage +{ + double resident_mb; + Int64 cur_proc_num_threads; + double cur_virt_mb; +}; + +bool process_mem_usage(double & resident_set, Int64 & cur_proc_num_threads, UInt64 & cur_virt_size) +{ + resident_set = 0.0; + + // 'file' stat seems to give the most reliable results + std::ifstream stat_stream("/proc/self/stat", std::ios_base::in); + // if "/proc/self/stat" is not supported + if (!stat_stream.is_open()) + return false; + + // dummy vars for leading entries in stat that we don't care about + std::string pid, comm, state, ppid, pgrp, session, tty_nr; + std::string tpgid, flags, minflt, cminflt, majflt, cmajflt; + std::string utime, stime, cutime, cstime, priority, nice; + std::string itrealvalue, starttime; + + // the field we want + Int64 rss; + + stat_stream >> pid >> comm >> state >> ppid >> pgrp >> session >> tty_nr >> tpgid >> flags >> minflt >> cminflt + >> majflt >> cmajflt >> utime >> stime >> cutime >> cstime >> priority >> nice >> cur_proc_num_threads + >> itrealvalue >> starttime >> cur_virt_size >> rss; // don't care about the rest + + stat_stream.close(); + + Int64 page_size_kb = sysconf(_SC_PAGE_SIZE) / 1024; // in case x86-64 is configured to use 2MB pages + resident_set = rss * page_size_kb; + return true; +} +ProcessMemoryUsage get_process_mem_usage() +{ + double resident_set; + Int64 cur_proc_num_threads = 1; + UInt64 cur_virt_size = 0; + process_mem_usage(resident_set, cur_proc_num_threads, cur_virt_size); + resident_set *= 1024; // unit: byte + return ProcessMemoryUsage{ + resident_set / 1024.0 / 1024, + cur_proc_num_threads, + cur_virt_size / 1024.0 / 1024, + }; +} +} // namespace + +TEST_F(SegmentOperationTest, TestMassiveSegment) +try +{ + const size_t level = 5; + for (size_t lvl = 0; lvl < level; ++lvl) + { + size_t num_expected_segs = 1000; + // size_t num_expected_segs = 10; + size_t progress_interval = 100; + const auto lvl_beg_seg_id = segments.rbegin()->first; + { + auto seg = segments[lvl_beg_seg_id]; + LOG_INFO(log, "lvl={} beg_seg_id={} rowkey={}", lvl, lvl_beg_seg_id, seg->getRowKeyRange().toString()); + } + auto next_split_seg_id = lvl_beg_seg_id; + for (size_t i = 0; i < num_expected_segs; ++i) + { + auto split_point = (lvl * num_expected_segs + 1 + i) * 500; + auto n_seg_id = splitSegmentAt(next_split_seg_id, split_point, Segment::SplitMode::Logical); + ASSERT_TRUE(n_seg_id.has_value()) << fmt::format("i={} sp={}", i, split_point); + next_split_seg_id = *n_seg_id; + if (i % progress_interval == 0) + { + auto mu = get_process_mem_usage(); + LOG_INFO( + log, + "lvl={} round={} split_point={} next_seg_id={} mem_resident_set={:.3f}MB)", + lvl, + i, + split_point, + *n_seg_id, + mu.resident_mb); + } + } + { + auto mu = get_process_mem_usage(); + LOG_INFO(log, "lvl={} round={} mem_resident_set={:.3f}MB", lvl, num_expected_segs, mu.resident_mb); + } + + size_t round = 0; + for (auto && [seg_id, seg] : segments) + { + // next_split_seg_id is the last segment created in this level, skip it + if (seg_id == next_split_seg_id) + continue; + + if (seg_id < lvl_beg_seg_id) + continue; // skip segments created in previous levels + + auto write_rows = 500; + if (round % progress_interval == 0) + { + auto mu = get_process_mem_usage(); + LOG_INFO( + log, + "lvl={} round={} written_rows={} mem_resident_set={:.3f}MB", + lvl, + round, + write_rows * round, + mu.resident_mb); + } + writeToCache( + seg_id, + write_rows, + /* start_at */ lvl * num_expected_segs * write_rows + round * write_rows, + false, + std::nullopt); + round++; + } + { + auto mu = get_process_mem_usage(); + LOG_INFO( + log, + "TestMassiveSegment done, segments.size()={} lvl={} mem_resident_set={:.3f}MB", + lvl, + segments.size(), + mu.resident_mb); + } + } +} +CATCH + class SegmentEnableLogicalSplitTest : public SegmentOperationTest { protected: diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp index 76540fd0949..495914f0fe8 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -92,8 +92,18 @@ void SegmentTestBasic::reloadWithOptions(SegmentTestOptions config) TiFlashStorageTestBasic::SetUp(); options = config; table_columns = std::make_shared(); + ColumnDefinesPtr cols = DMTestEnv::getDefaultColumns(); + for (Int64 i = 0; i < 1000; ++i) + { + // add a new column + cols->emplace_back(ColumnDefine{ + i + 100, + fmt::format("field_{}", i), + DB::tests::typeFromString("Nullable(String)"), + }); + } - root_segment = reload(config.is_common_handle, nullptr, std::move(config.db_settings)); + root_segment = reload(config.is_common_handle, cols, std::move(config.db_settings)); ASSERT_EQ(root_segment->segmentId(), DELTA_MERGE_FIRST_SEGMENT_ID); segments.clear(); segments[DELTA_MERGE_FIRST_SEGMENT_ID] = root_segment; @@ -543,6 +553,19 @@ Block SegmentTestBasic::prepareWriteBlockInSegmentRange( RUNTIME_CHECK(write_end_key_this_round <= segment_end_key); Block block = prepareWriteBlock(*write_start_key, write_end_key_this_round, is_deleted, including_right_boundary, ts); + for (Int64 i = 0; i < 1000; ++i) + { + auto null_col = ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()); + for (size_t j = 0; j < write_rows_this_round; ++j) + { + null_col->insertDefault(); + } + block.insert(ColumnWithTypeAndName{ + std::move(null_col), + std::make_shared(std::make_shared()), + fmt::format("field_{}", i), + }); + } blocks.emplace_back(block); remaining_rows -= write_rows_this_round + including_right_boundary; diff --git a/dbms/src/Storages/System/StorageSystemDTSegments.cpp b/dbms/src/Storages/System/StorageSystemDTSegments.cpp index d87d38d0cec..2a457768342 100644 --- a/dbms/src/Storages/System/StorageSystemDTSegments.cpp +++ b/dbms/src/Storages/System/StorageSystemDTSegments.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/TestUtils/gtests_dbms_main.cpp b/dbms/src/TestUtils/gtests_dbms_main.cpp index 4543302464d..086ffa2007c 100644 --- a/dbms/src/TestUtils/gtests_dbms_main.cpp +++ b/dbms/src/TestUtils/gtests_dbms_main.cpp @@ -66,7 +66,7 @@ int main(int argc, char ** argv) install_fault_signal_handlers({SIGSEGV, SIGILL, SIGFPE, SIGABRT, SIGTERM}); bool enable_colors = isatty(STDERR_FILENO) && isatty(STDOUT_FILENO); - DB::tests::TiFlashTestEnv::setupLogger("trace", std::cerr, enable_colors); + DB::tests::TiFlashTestEnv::setupLogger("information", std::cerr, enable_colors); auto run_mode = DB::PageStorageRunMode::ONLY_V3; DB::tests::TiFlashTestEnv::initializeGlobalContext(/*testdata_path*/ {}, run_mode); DB::ServerInfo server_info; From 92bfd28ddfd815dea47cb7e09b6495d519763066 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Wed, 2 Jul 2025 09:17:45 +0800 Subject: [PATCH 02/10] Update to tiflash system table Signed-off-by: JaySon-Huang --- .../DeltaMerge/DeltaMergeStore_Statistics.cpp | 33 +++++++++++-------- .../DeltaMerge/DeltaMergeStore_Statistics.h | 1 + .../System/StorageSystemDTSegments.cpp | 2 ++ .../Storages/System/StorageSystemDTTables.cpp | 2 ++ 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp index d6b9fb5633e..e591fffde42 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -32,7 +32,8 @@ StoreStats DeltaMergeStore::getStoreStats() Int64 total_placed_rows = 0; Int64 total_delta_cache_rows = 0; - Float64 total_delta_cache_size = 0; + UInt64 total_delta_cache_size = 0; + UInt64 total_delta_cache_alloc_size = 0; Int64 total_delta_valid_cache_rows = 0; { std::shared_lock lock(read_write_mutex); @@ -65,6 +66,7 @@ StoreStats DeltaMergeStore::getStoreStats() total_delta_cache_rows += delta->getTotalCacheRows(); total_delta_cache_size += delta->getTotalCacheBytes(); + total_delta_cache_alloc_size += delta->getTotalAllocatedBytes(); total_delta_valid_cache_rows += delta->getValidCacheRows(); } @@ -88,6 +90,7 @@ StoreStats DeltaMergeStore::getStoreStats() stat.delta_placed_rate = static_cast(total_placed_rows) / stat.total_delta_rows; stat.delta_cache_size = total_delta_cache_size; + stat.delta_cache_alloc_size = total_delta_cache_alloc_size; stat.delta_cache_rate = static_cast(total_delta_valid_cache_rows) / stat.total_delta_rows; stat.delta_cache_wasted_rate = static_cast(total_delta_cache_rows - total_delta_valid_cache_rows) / total_delta_valid_cache_rows; @@ -154,8 +157,6 @@ SegmentsStats DeltaMergeStore::getSegmentsStats() SegmentStats stat; const auto & delta = segment->getDelta(); - const auto & delta_memtable = delta->getMemTableSet(); - const auto & delta_persisted = delta->getPersistedFileSet(); const auto & stable = segment->getStable(); stat.segment_id = segment->segmentId(); @@ -165,17 +166,21 @@ SegmentsStats DeltaMergeStore::getSegmentsStats() stat.size = segment->getEstimatedBytes(); stat.delta_rate = static_cast(delta->getRows()) / stat.rows; - stat.delta_memtable_rows = delta_memtable->getRows(); - stat.delta_memtable_size = delta_memtable->getBytes(); - stat.delta_memtable_column_files = delta_memtable->getColumnFileCount(); - stat.delta_memtable_delete_ranges = delta_memtable->getDeletes(); - stat.delta_persisted_page_id = delta_persisted->getId(); - stat.delta_persisted_rows = delta_persisted->getRows(); - stat.delta_persisted_size = delta_persisted->getBytes(); - stat.delta_persisted_column_files = delta_persisted->getColumnFileCount(); - stat.delta_persisted_delete_ranges = delta_persisted->getDeletes(); - stat.delta_cache_size = delta->getTotalCacheBytes(); - stat.delta_cache_alloc_size = delta->getTotalAllocatedBytes(); + { + const auto & delta_memtable = delta->getMemTableSet(); + const auto & delta_persisted = delta->getPersistedFileSet(); + stat.delta_memtable_rows = delta_memtable->getRows(); + stat.delta_memtable_size = delta_memtable->getBytes(); + stat.delta_memtable_column_files = delta_memtable->getColumnFileCount(); + stat.delta_memtable_delete_ranges = delta_memtable->getDeletes(); + stat.delta_persisted_page_id = delta_persisted->getId(); + stat.delta_persisted_rows = delta_persisted->getRows(); + stat.delta_persisted_size = delta_persisted->getBytes(); + stat.delta_persisted_column_files = delta_persisted->getColumnFileCount(); + stat.delta_persisted_delete_ranges = delta_persisted->getDeletes(); + stat.delta_cache_size = delta_memtable->getBytes(); // FIXME: this is the same as delta_memtable_size + stat.delta_cache_alloc_size = delta_memtable->getAllocatedBytes(); + } stat.delta_index_size = delta->getDeltaIndexBytes(); stat.stable_page_id = stable->getId(); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h index 1d26ecc19de..cab67f8b916 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h @@ -67,6 +67,7 @@ struct StoreStats Float64 delta_placed_rate = 0; UInt64 delta_cache_size = 0; + UInt64 delta_cache_alloc_size = 0; Float64 delta_cache_rate = 0; Float64 delta_cache_wasted_rate = 0; diff --git a/dbms/src/Storages/System/StorageSystemDTSegments.cpp b/dbms/src/Storages/System/StorageSystemDTSegments.cpp index 2a457768342..37802f177c4 100644 --- a/dbms/src/Storages/System/StorageSystemDTSegments.cpp +++ b/dbms/src/Storages/System/StorageSystemDTSegments.cpp @@ -61,6 +61,7 @@ StorageSystemDTSegments::StorageSystemDTSegments(const std::string & name_) {"delta_persisted_column_files", std::make_shared()}, {"delta_persisted_delete_ranges", std::make_shared()}, {"delta_cache_size", std::make_shared()}, + {"delta_cache_alloc_size", std::make_shared()}, {"delta_index_size", std::make_shared()}, {"stable_page_id", std::make_shared()}, @@ -153,6 +154,7 @@ BlockInputStreams StorageSystemDTSegments::read( res_columns[j++]->insert(stat.delta_persisted_column_files); res_columns[j++]->insert(stat.delta_persisted_delete_ranges); res_columns[j++]->insert(stat.delta_cache_size); + res_columns[j++]->insert(stat.delta_cache_alloc_size); res_columns[j++]->insert(stat.delta_index_size); res_columns[j++]->insert(stat.stable_page_id); diff --git a/dbms/src/Storages/System/StorageSystemDTTables.cpp b/dbms/src/Storages/System/StorageSystemDTTables.cpp index 5143a0290b0..0a0097337e7 100644 --- a/dbms/src/Storages/System/StorageSystemDTTables.cpp +++ b/dbms/src/Storages/System/StorageSystemDTTables.cpp @@ -55,6 +55,7 @@ StorageSystemDTTables::StorageSystemDTTables(const std::string & name_) {"delta_placed_rate", std::make_shared()}, {"delta_cache_size", std::make_shared()}, + {"delta_cache_alloc_size", std::make_shared()}, {"delta_cache_rate", std::make_shared()}, {"delta_cache_wasted_rate", std::make_shared()}, @@ -178,6 +179,7 @@ BlockInputStreams StorageSystemDTTables::read( res_columns[j++]->insert(stat.delta_placed_rate); res_columns[j++]->insert(stat.delta_cache_size); + res_columns[j++]->insert(stat.delta_cache_alloc_size); res_columns[j++]->insert(stat.delta_cache_rate); res_columns[j++]->insert(stat.delta_cache_wasted_rate); From 908c195ff15a777f61394b9c2688d571f1625907 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Fri, 4 Jul 2025 15:44:53 +0800 Subject: [PATCH 03/10] Add metrics Signed-off-by: JaySon-Huang --- dbms/src/Common/CurrentMetrics.cpp | 5 ++- .../DeltaMerge/ColumnFile/ColumnFile.h | 21 ++++------- .../ColumnFile/ColumnFileInMemory.cpp | 25 +++++++------ .../ColumnFile/ColumnFileInMemory.h | 24 +++++++++++-- .../Storages/DeltaMerge/Delta/MemTableSet.cpp | 35 +++++++++++++++---- .../Storages/DeltaMerge/Delta/MemTableSet.h | 17 +++------ .../src/Storages/DeltaMerge/DeltaMergeStore.h | 2 +- .../Storages/DeltaMerge/Remote/Serializer.cpp | 2 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 4 ++- dbms/src/Storages/DeltaMerge/Segment.h | 4 +++ 10 files changed, 87 insertions(+), 52 deletions(-) diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index c6b888da9e0..c30da051f3e 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -88,7 +88,10 @@ M(ConnectionPoolSize) \ M(MemoryTrackingQueryStorageTask) \ M(MemoryTrackingFetchPages) \ - M(MemoryTrackingSharedColumnData) + M(MemoryTrackingSharedColumnData) \ + M(DT_NumSegments) \ + M(DT_NumMemTable) \ + M(DT_BytesMemTable) namespace CurrentMetrics { diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h index bea914daca0..1b9e5a70669 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h @@ -73,19 +73,6 @@ class ColumnFile INMEMORY_FILE = 4, }; - struct Cache - { - explicit Cache(const Block & header) - : block(header.cloneWithColumns(header.cloneEmptyColumns())) - {} - explicit Cache(Block && block) - : block(std::move(block)) - {} - - std::mutex mutex; - Block block; - }; - using CachePtr = std::shared_ptr; using ColIdToOffset = std::unordered_map; public: @@ -138,7 +125,13 @@ class ColumnFile /// been persisted in the disk and their data will be immutable. virtual bool isAppendable() const { return false; } virtual void disableAppend() {} - virtual bool append( + + struct AppendResult + { + bool success = false; // whether the append is successful + size_t new_alloc_bytes = 0; // the new allocated bytes after append + }; + virtual AppendResult append( const DMContext & /*dm_context*/, const Block & /*data*/, size_t /*offset*/, diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp index 29dd392367d..57f249d436c 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp @@ -41,6 +41,7 @@ void ColumnFileInMemory::fillColumns(const ColumnDefines & col_defs, size_t col_ // Copy data from cache const auto & type = getDataType(cd.id); auto col_data = type->createColumn(); + col_data->reserve(rows); col_data->insertRangeFrom(*(cache->block.getByPosition(col_offset).column), 0, rows); // Cast if need auto col_converted = convertColumnByColumnDefineIfNeed(type, std::move(col_data), cd); @@ -64,7 +65,13 @@ ColumnFileReaderPtr ColumnFileInMemory::getReader( return std::make_shared(*this, col_defs); } -bool ColumnFileInMemory::append( +void ColumnFileInMemory::disableAppend() +{ + disable_append = true; + // TODO: Call shrinkToFit() to release the extra memory of the cache block. +} + +ColumnFile::AppendResult ColumnFileInMemory::append( const DMContext & context, const Block & data, size_t offset, @@ -72,16 +79,16 @@ bool ColumnFileInMemory::append( size_t data_bytes) { if (disable_append) - return false; + return AppendResult{false, 0}; std::scoped_lock lock(cache->mutex); if (!isSameSchema(cache->block, data)) - return false; + return AppendResult{false, 0}; // check whether this instance overflows if (cache->block.rows() >= context.delta_cache_limit_rows || cache->block.bytes() >= context.delta_cache_limit_bytes) - return false; + return AppendResult{false, 0}; size_t new_alloc_block_bytes = 0; for (size_t i = 0; i < cache->block.columns(); ++i) @@ -96,15 +103,7 @@ bool ColumnFileInMemory::append( rows += limit; bytes += data_bytes; - LOG_INFO( - Logger::get(), - "Append rows to ColumnFileInMemory, new_rows={} new_bytes={} new_alloc_bytes={} total_rows={} total_bytes={}", - limit - offset, - data_bytes, - new_alloc_block_bytes, - rows, - bytes); - return true; + return AppendResult{true, new_alloc_block_bytes}; } Block ColumnFileInMemory::readDataForFlush() const diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h index fb19ae691ac..257b3327434 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h @@ -30,6 +30,20 @@ class ColumnFileInMemory : public ColumnFile friend class ColumnFileInMemoryReader; friend struct Remote::Serializer; + struct Cache + { + explicit Cache(const Block & header) + : block(header.cloneWithColumns(header.cloneEmptyColumns())) + {} + explicit Cache(Block && block) + : block(std::move(block)) + {} + + std::mutex mutex; + Block block; + }; + using CachePtr = std::shared_ptr; + private: ColumnFileSchemaPtr schema; @@ -83,9 +97,13 @@ class ColumnFileInMemory : public ColumnFile ReadTag) const override; bool isAppendable() const override { return !disable_append; } - void disableAppend() override { disable_append = true; } - bool append(const DMContext & dm_context, const Block & data, size_t offset, size_t limit, size_t data_bytes) - override; + void disableAppend() override; + AppendResult append( + const DMContext & dm_context, + const Block & data, + size_t offset, + size_t limit, + size_t data_bytes) override; Block readDataForFlush() const; diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp index 8f421224fca..69635369c5b 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp @@ -22,9 +22,32 @@ #include #include +namespace CurrentMetrics +{ +extern const Metric DT_NumMemTable; +extern const Metric DT_BytesMemTable; +} // namespace CurrentMetrics + namespace DB::DM { +MemTableSet::MemTableSet(const ColumnFiles & in_memory_files) + : holder_counter(CurrentMetrics::DT_NumMemTable, 1) + , holder_allocated_bytes(CurrentMetrics::DT_BytesMemTable, 0) + , column_files(in_memory_files) + , log(Logger::get()) +{ + column_files_count = column_files.size(); + for (const auto & file : column_files) + { + rows += file->getRows(); + bytes += file->getBytes(); + allocated_bytes += file->getAllocateBytes(); + deletes += file->getDeletes(); + } + holder_allocated_bytes.changeTo(allocated_bytes.load()); +} + void MemTableSet::appendColumnFileInner(const ColumnFilePtr & column_file) { if (!column_files.empty()) @@ -182,16 +205,16 @@ void MemTableSet::appendColumnFile(const ColumnFilePtr & column_file) void MemTableSet::appendToCache(DMContext & context, const Block & block, size_t offset, size_t limit) { // If the `column_files` is not empty, and the last `column_file` is a `ColumnInMemoryFile`, we will merge the newly block into the last `column_file`. - bool success = false; + ColumnFile::AppendResult append_res; size_t append_bytes = block.bytes(offset, limit); if (!column_files.empty()) { auto & last_column_file = column_files.back(); if (last_column_file->isAppendable()) - success = last_column_file->append(context, block, offset, limit, append_bytes); + append_res = last_column_file->append(context, block, offset, limit, append_bytes); } - if (!success) + if (!append_res.success) { /// Otherwise, create a new `ColumnInMemoryFile` and write into it. @@ -202,13 +225,13 @@ void MemTableSet::appendToCache(DMContext & context, const Block & block, size_t // Must append the empty `new_column_file` to `column_files` before appending data to it, // because `appendColumnFileInner` will update stats related to `column_files` but we will update stats relate to `new_column_file` here. appendColumnFileInner(new_column_file); - success = new_column_file->append(context, block, offset, limit, append_bytes); - if (unlikely(!success)) + append_res = new_column_file->append(context, block, offset, limit, append_bytes); + if (unlikely(!append_res.success)) throw Exception("Write to MemTableSet failed", ErrorCodes::LOGICAL_ERROR); } rows += limit; bytes += append_bytes; - + allocated_bytes += append_res.new_alloc_bytes; } void MemTableSet::appendDeleteRange(const RowKeyRange & delete_range) diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h index e68cd98c49b..72cd10f879d 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h @@ -37,6 +37,10 @@ class MemTableSet #else public: #endif + // Keep track of the number of mem-table in memory. + CurrentMetrics::Increment holder_counter; + CurrentMetrics::Increment holder_allocated_bytes; + // Note that we must update `column_files_count` for outer thread-safe after `column_files` changed ColumnFiles column_files; @@ -54,18 +58,7 @@ class MemTableSet void appendColumnFileInner(const ColumnFilePtr & column_file); public: - explicit MemTableSet(const ColumnFiles & in_memory_files = {}) - : column_files(in_memory_files) - , log(Logger::get()) - { - column_files_count = column_files.size(); - for (const auto & file : column_files) - { - rows += file->getRows(); - bytes += file->getBytes(); - deletes += file->getDeletes(); - } - } + explicit MemTableSet(const ColumnFiles & in_memory_files = {}); /** * Resets the logger by using the one from the segment. diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 4e216d1ffa1..3b9e3c2f0d1 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -27,8 +27,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp index 147c9c2af91..d809f16f418 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp @@ -309,7 +309,7 @@ ColumnFileInMemoryPtr Serializer::deserializeCFInMemory(const RemotePb::ColumnFi } auto block = block_schema->cloneWithColumns(std::move(columns)); - auto cache = std::make_shared(std::move(block)); + auto cache = std::make_shared(std::move(block)); // We do not try to reuse the CFSchema from `SharedBlockSchemas`, because the ColumnFile will be freed immediately after the request. auto schema = std::make_shared(*block_schema); diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 72bbb97d23c..9b93fbb14d3 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -128,6 +128,7 @@ extern const Metric DT_SnapshotOfPlaceIndex; extern const Metric DT_SnapshotOfReplayVersionChain; extern const Metric DT_SnapshotOfSegmentIngest; extern const Metric DT_SnapshotOfBitmapFilter; +extern const Metric DT_NumSegments; } // namespace CurrentMetrics namespace DB @@ -295,7 +296,8 @@ Segment::Segment( // PageIdU64 next_segment_id_, const DeltaValueSpacePtr & delta_, const StableValueSpacePtr & stable_) - : epoch(epoch_) + : holder_counter(CurrentMetrics::DT_NumSegments) + , epoch(epoch_) , rowkey_range(rowkey_range_) , is_common_handle(rowkey_range.is_common_handle) , rowkey_column_size(rowkey_range.rowkey_column_size) diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index 8b3615125fe..97bada605a3 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -837,6 +837,10 @@ class Segment #else public: #endif + + // Keep track of the number of segments in memory. + CurrentMetrics::Increment holder_counter; + /// The version of this segment. After split / merge / mergeDelta / replaceData, epoch got increased by 1. const UInt64 epoch; From a080bc68a841734d8cb7ad48ac77da21576dc06a Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Fri, 4 Jul 2025 16:31:54 +0800 Subject: [PATCH 04/10] Add pushdown filter to scan_details Signed-off-by: JaySon-Huang --- .../DeltaMerge/DeltaMergeStore_Statistics.cpp | 2 ++ .../DeltaMerge/DeltaMergeStore_Statistics.h | 1 + .../DeltaMerge/Filter/PushDownExecutor.cpp | 16 ++++++++++++++++ .../DeltaMerge/Filter/PushDownExecutor.h | 9 ++++++--- dbms/src/Storages/DeltaMerge/ScanContext.cpp | 6 ++++++ dbms/src/Storages/DeltaMerge/ScanContext.h | 5 ++++- 6 files changed, 35 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp index e591fffde42..51d213c6e1a 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -38,6 +38,7 @@ StoreStats DeltaMergeStore::getStoreStats() { std::shared_lock lock(read_write_mutex); stat.segment_count = segments.size(); + stat.column_count = original_table_columns.size(); for (const auto & [handle, segment] : segments) { @@ -85,6 +86,7 @@ StoreStats DeltaMergeStore::getStoreStats() } } // access to `segments` end + stat.delta_rate_rows = static_cast(stat.total_delta_rows) / stat.total_rows; stat.delta_rate_segments = static_cast(stat.delta_count) / stat.segment_count; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h index cab67f8b916..9dbbf7d702f 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h @@ -56,6 +56,7 @@ using SegmentsStats = std::vector; struct StoreStats { + UInt64 column_count = 0; UInt64 segment_count = 0; UInt64 total_rows = 0; diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp index 4003ebb4810..84daec58c59 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -229,4 +230,19 @@ PushDownExecutorPtr PushDownExecutor::build( context, tracing_logger); } + +Poco::JSON::Object::Ptr PushDownExecutor::toJSONObject() const +{ + Poco::JSON::Object::Ptr json = new Poco::JSON::Object(); + if (rs_operator) + { + json->set("rs_operator", rs_operator->toDebugString()); + } + if (ann_query_info) + { + json->set("ann_query_info", ann_query_info->ShortDebugString()); + } + return json; +} + } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h index 2a324089838..9d631cf7927 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -39,7 +40,7 @@ class PushDownExecutor const RSOperatorPtr & rs_operator_, const ANNQueryInfoPtr & ann_query_info_, const FTSQueryInfoPtr & fts_query_info_, - const ExpressionActionsPtr & beofre_where_, + const ExpressionActionsPtr & before_where_, const ExpressionActionsPtr & project_after_where_, const ColumnDefinesPtr & filter_columns_, const String filter_column_name_, @@ -47,7 +48,7 @@ class PushDownExecutor const ColumnDefinesPtr & columns_after_cast_, const ColumnRangePtr & column_range_) : rs_operator(rs_operator_) - , before_where(beofre_where_) + , before_where(before_where_) , project_after_where(project_after_where_) , filter_column_name(std::move(filter_column_name_)) , filter_columns(filter_columns_) @@ -77,6 +78,8 @@ class PushDownExecutor : fts_query_info(fts_query_info_) {} + Poco::JSON::Object::Ptr toJSONObject() const; + // Use by StorageDisaggregated. static PushDownExecutorPtr build( const DM::RSOperatorPtr & rs_operator, @@ -105,7 +108,7 @@ class PushDownExecutor const ExpressionActionsPtr before_where; // The projection after the filter, used to remove the tmp filter column // Used to construct the ExpressionBlockInputStream - // Note: ususally we will remove the tmp filter column in the LateMaterializationBlockInputStream, this only used for unexpected cases + // Note: usually we will remove the tmp filter column in the LateMaterializationBlockInputStream, this only used for unexpected cases const ExpressionActionsPtr project_after_where; const String filter_column_name; // The columns needed by the filter expression diff --git a/dbms/src/Storages/DeltaMerge/ScanContext.cpp b/dbms/src/Storages/DeltaMerge/ScanContext.cpp index 10403032c73..8fba0164bf9 100644 --- a/dbms/src/Storages/DeltaMerge/ScanContext.cpp +++ b/dbms/src/Storages/DeltaMerge/ScanContext.cpp @@ -18,6 +18,7 @@ #include #pragma GCC diagnostic pop #include +#include #include #include @@ -182,6 +183,11 @@ String ScanContext::toJson() const json->set("vector_idx", vec_idx); } + if (pushdown_executor) + { + json->set("pushdown", pushdown_executor->toJSONObject()); + } + std::stringstream buf; json->stringify(buf); return buf.str(); diff --git a/dbms/src/Storages/DeltaMerge/ScanContext.h b/dbms/src/Storages/DeltaMerge/ScanContext.h index d5e5db0ad29..a152e2ae091 100644 --- a/dbms/src/Storages/DeltaMerge/ScanContext.h +++ b/dbms/src/Storages/DeltaMerge/ScanContext.h @@ -30,6 +30,8 @@ namespace DB::DM { +class PushDownExecutor; +using PushDownExecutorPtr = std::shared_ptr; /// ScanContext is used to record statistical information in table scan for current query. /// For each table scan(one executor id), there is only one ScanContext. /// ScanContext helps to collect the statistical information of the table scan to show in `EXPLAIN ANALYZE`. @@ -69,7 +71,6 @@ class ScanContext std::atomic delta_rows{0}; std::atomic delta_bytes{0}; - ReadMode read_mode = ReadMode::Normal; // - read_mode == Normal, apply mvcc to all read blocks // - read_mode == Bitmap, it will apply mvcc to get the bitmap @@ -135,7 +136,9 @@ class ScanContext std::atomic fts_brute_total_search_ms{0}; const KeyspaceID keyspace_id; + ReadMode read_mode = ReadMode::Normal; // note: share struct padding with keyspace_id const String resource_group_name; + PushDownExecutorPtr pushdown_executor; explicit ScanContext(const KeyspaceID & keyspace_id_ = NullspaceID, const String & name = "") : keyspace_id(keyspace_id_) From 13fa3dfd1791bf8e8f38387a31a38ab34563c5f0 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Fri, 4 Jul 2025 16:40:52 +0800 Subject: [PATCH 05/10] Normalize logging level Signed-off-by: JaySon-Huang --- dbms/src/TestUtils/gtests_dbms_main.cpp | 4 ++- libs/libcommon/include/common/logger_util.h | 36 +++++++++++++++++++++ libs/libdaemon/src/BaseDaemon.cpp | 18 ++--------- 3 files changed, 41 insertions(+), 17 deletions(-) create mode 100644 libs/libcommon/include/common/logger_util.h diff --git a/dbms/src/TestUtils/gtests_dbms_main.cpp b/dbms/src/TestUtils/gtests_dbms_main.cpp index 086ffa2007c..ae598b8d067 100644 --- a/dbms/src/TestUtils/gtests_dbms_main.cpp +++ b/dbms/src/TestUtils/gtests_dbms_main.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include @@ -66,7 +67,8 @@ int main(int argc, char ** argv) install_fault_signal_handlers({SIGSEGV, SIGILL, SIGFPE, SIGABRT, SIGTERM}); bool enable_colors = isatty(STDERR_FILENO) && isatty(STDOUT_FILENO); - DB::tests::TiFlashTestEnv::setupLogger("information", std::cerr, enable_colors); + const auto log_level = Utils::normalizeLogLevel(Poco::Environment::get("LOG_LEVEL", "trace")); + DB::tests::TiFlashTestEnv::setupLogger(log_level, std::cerr, enable_colors); auto run_mode = DB::PageStorageRunMode::ONLY_V3; DB::tests::TiFlashTestEnv::initializeGlobalContext(/*testdata_path*/ {}, run_mode); DB::ServerInfo server_info; diff --git a/libs/libcommon/include/common/logger_util.h b/libs/libcommon/include/common/logger_util.h new file mode 100644 index 00000000000..f311a447fb8 --- /dev/null +++ b/libs/libcommon/include/common/logger_util.h @@ -0,0 +1,36 @@ +// Copyright 2025 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 + +namespace Utils +{ + +inline std::string normalizeLogLevel(const std::string & log_level) +{ + std::string norm = Poco::toLower(log_level); + // normalize + // info -> information + // warn -> warning + if (norm == "info") + return "information"; + else if (norm == "warn") + return "warning"; + else + return norm; +} + +} // namespace Utils diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 72c72d00054..1e1f4f113f2 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -77,6 +77,7 @@ #include #include #include +#include #include #include #include @@ -666,21 +667,6 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path return false; } -static std::string normalize(const std::string & log_level) -{ - std::string norm = Poco::toLower(log_level); - // normalize - // info -> information - // warn -> warning - if (norm == "info") - return "information"; - else if (norm == "warn") - return "warning"; - else - return norm; -} - - void BaseDaemon::reloadConfiguration() { // when config-file is not specified and config.toml does not exist, we do not load config. @@ -755,7 +741,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) // Split log, error log and tracing log. Poco::AutoPtr split = new Poco::ReloadableSplitterChannel; - auto log_level = normalize(config.getString("logger.level", "info")); + auto log_level = Utils::normalizeLogLevel(config.getString("logger.level", "info")); const auto log_path = config.getString("logger.log", ""); if (!log_path.empty()) { From 8ddf0bd8c203827ba3c26e2a1a8a30e8d3e7bd4f Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Fri, 4 Jul 2025 16:43:06 +0800 Subject: [PATCH 06/10] Remove debug codes Signed-off-by: JaySon-Huang --- .../Storages/DeltaMerge/File/DMFileMetaV2.cpp | 2 +- .../DeltaMerge/StoragePool/StoragePool.cpp | 12 +- .../DeltaMerge/tests/gtest_segment.cpp | 134 ------------------ .../tests/gtest_segment_test_basic.cpp | 25 +--- 4 files changed, 8 insertions(+), 165 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp index 2c92f319c41..700c6bb214a 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileMetaV2.cpp @@ -284,7 +284,7 @@ void DMFileMetaV2::read(const FileProviderPtr & file_provider, const ReadMode & { break; } - // LOG_WARNING(log, "{}'s size is larger than {}", metaPath(), buf.size()); + LOG_WARNING(log, "{}'s size is larger than {}", metaPath(), buf.size()); buf.resize(buf.size() + meta_buffer_size); } buf.resize(read_bytes); diff --git a/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp b/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp index bb757c308a9..cc002fa59e1 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp +++ b/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp @@ -891,12 +891,12 @@ PageIdU64 StoragePool::newDataPageIdForDTFile(StableDiskDelegator & delegator, [ break; } // else there is a DTFile with that id, continue to acquire a new ID. - // LOG_WARNING( - // logger, - // "The DTFile is already exists, continue to acquire another ID. call={} path={} file_id={}", - // who, - // existed_path, - // dtfile_id); + LOG_WARNING( + logger, + "The DTFile is already exists, continue to acquire another ID. call={} path={} file_id={}", + who, + existed_path, + dtfile_id); } while (true); return dtfile_id; } diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp index d7db0b01a5f..2d2b4d38503 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp @@ -607,140 +607,6 @@ try } CATCH -namespace -{ -struct ProcessMemoryUsage -{ - double resident_mb; - Int64 cur_proc_num_threads; - double cur_virt_mb; -}; - -bool process_mem_usage(double & resident_set, Int64 & cur_proc_num_threads, UInt64 & cur_virt_size) -{ - resident_set = 0.0; - - // 'file' stat seems to give the most reliable results - std::ifstream stat_stream("/proc/self/stat", std::ios_base::in); - // if "/proc/self/stat" is not supported - if (!stat_stream.is_open()) - return false; - - // dummy vars for leading entries in stat that we don't care about - std::string pid, comm, state, ppid, pgrp, session, tty_nr; - std::string tpgid, flags, minflt, cminflt, majflt, cmajflt; - std::string utime, stime, cutime, cstime, priority, nice; - std::string itrealvalue, starttime; - - // the field we want - Int64 rss; - - stat_stream >> pid >> comm >> state >> ppid >> pgrp >> session >> tty_nr >> tpgid >> flags >> minflt >> cminflt - >> majflt >> cmajflt >> utime >> stime >> cutime >> cstime >> priority >> nice >> cur_proc_num_threads - >> itrealvalue >> starttime >> cur_virt_size >> rss; // don't care about the rest - - stat_stream.close(); - - Int64 page_size_kb = sysconf(_SC_PAGE_SIZE) / 1024; // in case x86-64 is configured to use 2MB pages - resident_set = rss * page_size_kb; - return true; -} -ProcessMemoryUsage get_process_mem_usage() -{ - double resident_set; - Int64 cur_proc_num_threads = 1; - UInt64 cur_virt_size = 0; - process_mem_usage(resident_set, cur_proc_num_threads, cur_virt_size); - resident_set *= 1024; // unit: byte - return ProcessMemoryUsage{ - resident_set / 1024.0 / 1024, - cur_proc_num_threads, - cur_virt_size / 1024.0 / 1024, - }; -} -} // namespace - -TEST_F(SegmentOperationTest, TestMassiveSegment) -try -{ - const size_t level = 5; - for (size_t lvl = 0; lvl < level; ++lvl) - { - size_t num_expected_segs = 1000; - // size_t num_expected_segs = 10; - size_t progress_interval = 100; - const auto lvl_beg_seg_id = segments.rbegin()->first; - { - auto seg = segments[lvl_beg_seg_id]; - LOG_INFO(log, "lvl={} beg_seg_id={} rowkey={}", lvl, lvl_beg_seg_id, seg->getRowKeyRange().toString()); - } - auto next_split_seg_id = lvl_beg_seg_id; - for (size_t i = 0; i < num_expected_segs; ++i) - { - auto split_point = (lvl * num_expected_segs + 1 + i) * 500; - auto n_seg_id = splitSegmentAt(next_split_seg_id, split_point, Segment::SplitMode::Logical); - ASSERT_TRUE(n_seg_id.has_value()) << fmt::format("i={} sp={}", i, split_point); - next_split_seg_id = *n_seg_id; - if (i % progress_interval == 0) - { - auto mu = get_process_mem_usage(); - LOG_INFO( - log, - "lvl={} round={} split_point={} next_seg_id={} mem_resident_set={:.3f}MB)", - lvl, - i, - split_point, - *n_seg_id, - mu.resident_mb); - } - } - { - auto mu = get_process_mem_usage(); - LOG_INFO(log, "lvl={} round={} mem_resident_set={:.3f}MB", lvl, num_expected_segs, mu.resident_mb); - } - - size_t round = 0; - for (auto && [seg_id, seg] : segments) - { - // next_split_seg_id is the last segment created in this level, skip it - if (seg_id == next_split_seg_id) - continue; - - if (seg_id < lvl_beg_seg_id) - continue; // skip segments created in previous levels - - auto write_rows = 500; - if (round % progress_interval == 0) - { - auto mu = get_process_mem_usage(); - LOG_INFO( - log, - "lvl={} round={} written_rows={} mem_resident_set={:.3f}MB", - lvl, - round, - write_rows * round, - mu.resident_mb); - } - writeToCache( - seg_id, - write_rows, - /* start_at */ lvl * num_expected_segs * write_rows + round * write_rows, - false, - std::nullopt); - round++; - } - { - auto mu = get_process_mem_usage(); - LOG_INFO( - log, - "TestMassiveSegment done, segments.size()={} lvl={} mem_resident_set={:.3f}MB", - lvl, - segments.size(), - mu.resident_mb); - } - } -} -CATCH class SegmentEnableLogicalSplitTest : public SegmentOperationTest { diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp index 495914f0fe8..76540fd0949 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -92,18 +92,8 @@ void SegmentTestBasic::reloadWithOptions(SegmentTestOptions config) TiFlashStorageTestBasic::SetUp(); options = config; table_columns = std::make_shared(); - ColumnDefinesPtr cols = DMTestEnv::getDefaultColumns(); - for (Int64 i = 0; i < 1000; ++i) - { - // add a new column - cols->emplace_back(ColumnDefine{ - i + 100, - fmt::format("field_{}", i), - DB::tests::typeFromString("Nullable(String)"), - }); - } - root_segment = reload(config.is_common_handle, cols, std::move(config.db_settings)); + root_segment = reload(config.is_common_handle, nullptr, std::move(config.db_settings)); ASSERT_EQ(root_segment->segmentId(), DELTA_MERGE_FIRST_SEGMENT_ID); segments.clear(); segments[DELTA_MERGE_FIRST_SEGMENT_ID] = root_segment; @@ -553,19 +543,6 @@ Block SegmentTestBasic::prepareWriteBlockInSegmentRange( RUNTIME_CHECK(write_end_key_this_round <= segment_end_key); Block block = prepareWriteBlock(*write_start_key, write_end_key_this_round, is_deleted, including_right_boundary, ts); - for (Int64 i = 0; i < 1000; ++i) - { - auto null_col = ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()); - for (size_t j = 0; j < write_rows_this_round; ++j) - { - null_col->insertDefault(); - } - block.insert(ColumnWithTypeAndName{ - std::move(null_col), - std::make_shared(std::make_shared()), - fmt::format("field_{}", i), - }); - } blocks.emplace_back(block); remaining_rows -= write_rows_this_round + including_right_boundary; From 1f4560d889ad0fbb0542e557924de5e951427bf6 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Mon, 7 Jul 2025 12:15:09 +0800 Subject: [PATCH 07/10] Update current metrics correctly Signed-off-by: JaySon-Huang --- dbms/src/Common/CurrentMetrics.cpp | 12 +- .../Storages/DeltaMerge/Delta/MemTableSet.cpp | 114 +++++++++++++----- .../Storages/DeltaMerge/Delta/MemTableSet.h | 73 +++++++---- .../DeltaMerge/DeltaMergeStore_Statistics.cpp | 13 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 4 +- dbms/src/Storages/IStorage.cpp | 12 ++ dbms/src/Storages/IStorage.h | 6 + dbms/src/Storages/ITableDeclaration.h | 2 +- dbms/src/Storages/StorageDeltaMerge.cpp | 5 + dbms/src/Storages/StorageDeltaMerge.h | 6 +- .../Storages/System/StorageSystemDTTables.cpp | 2 + dbms/src/TiDB/Schema/SchemaSyncService.cpp | 8 ++ 12 files changed, 191 insertions(+), 66 deletions(-) diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index c30da051f3e..c2cbbba2d80 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -65,6 +65,13 @@ M(DT_SnapshotOfReplayVersionChain) \ M(DT_SnapshotOfBitmapFilter) \ M(DT_SnapshotOfDisaggReadNodeRead) \ + M(NumKeyspace) \ + M(NumIStorage) \ + M(DT_NumStorageDeltaMerge) \ + M(DT_NumSegment) \ + M(DT_NumMemTable) \ + M(DT_BytesMemTable) \ + M(DT_BytesMemTableAllocated) \ M(IOLimiterPendingBgWriteReq) \ M(IOLimiterPendingFgWriteReq) \ M(IOLimiterPendingBgReadReq) \ @@ -88,10 +95,7 @@ M(ConnectionPoolSize) \ M(MemoryTrackingQueryStorageTask) \ M(MemoryTrackingFetchPages) \ - M(MemoryTrackingSharedColumnData) \ - M(DT_NumSegments) \ - M(DT_NumMemTable) \ - M(DT_BytesMemTable) + M(MemoryTrackingSharedColumnData) namespace CurrentMetrics { diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp index 69635369c5b..0e89d674222 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp @@ -26,26 +26,72 @@ namespace CurrentMetrics { extern const Metric DT_NumMemTable; extern const Metric DT_BytesMemTable; +extern const Metric DT_BytesMemTableAllocated; } // namespace CurrentMetrics namespace DB::DM { +/// Member functions of MemTableSet::Statistic /// + +MemTableSet::Statistic::Statistic() + : holder_bytes(CurrentMetrics::DT_BytesMemTable, 0) + , holder_allocated_bytes(CurrentMetrics::DT_BytesMemTableAllocated, 0) +{} + +void MemTableSet::Statistic::append( + size_t rows_added, + size_t bytes_added, + size_t allocated_bytes_added, + size_t deletes_added, + size_t files_added) +{ + column_files_count += files_added; + rows += rows_added; + bytes += bytes_added; + allocated_bytes += allocated_bytes_added; + deletes += deletes_added; + // update the current metrics + holder_bytes.changeTo(bytes.load()); + holder_allocated_bytes.changeTo(allocated_bytes.load()); +} + +void MemTableSet::Statistic::resetTo( + size_t new_column_files_count, + size_t new_rows, + size_t new_bytes, + size_t new_allocated_bytes, + size_t new_deletes) +{ + column_files_count = new_column_files_count; + rows = new_rows; + bytes = new_bytes; + allocated_bytes = new_allocated_bytes; + deletes = new_deletes; + // update the current metrics + holder_bytes.changeTo(bytes.load()); + holder_allocated_bytes.changeTo(allocated_bytes.load()); +} + +/// Member functions of MemTableSet /// + MemTableSet::MemTableSet(const ColumnFiles & in_memory_files) : holder_counter(CurrentMetrics::DT_NumMemTable, 1) - , holder_allocated_bytes(CurrentMetrics::DT_BytesMemTable, 0) , column_files(in_memory_files) , log(Logger::get()) { - column_files_count = column_files.size(); + size_t new_rows = 0; + size_t new_bytes = 0; + size_t new_alloc_bytes = 0; + size_t new_deletes = 0; for (const auto & file : column_files) { - rows += file->getRows(); - bytes += file->getBytes(); - allocated_bytes += file->getAllocateBytes(); - deletes += file->getDeletes(); + new_rows += file->getRows(); + new_bytes += file->getBytes(); + new_alloc_bytes += file->getAllocateBytes(); + new_deletes += file->getDeletes(); } - holder_allocated_bytes.changeTo(allocated_bytes.load()); + stat.resetTo(column_files.size(), new_rows, new_bytes, new_alloc_bytes, new_deletes); } void MemTableSet::appendColumnFileInner(const ColumnFilePtr & column_file) @@ -60,12 +106,12 @@ void MemTableSet::appendColumnFileInner(const ColumnFilePtr & column_file) } column_files.push_back(column_file); - column_files_count = column_files.size(); - - rows += column_file->getRows(); - bytes += column_file->getBytes(); - allocated_bytes += column_file->getAllocateBytes(); - deletes += column_file->getDeletes(); + stat.append( + column_file->getRows(), + column_file->getBytes(), + column_file->getAllocateBytes(), + column_file->getDeletes(), + /*files_added=*/1); } std::pair MemTableSet::diffColumnFiles( @@ -229,9 +275,13 @@ void MemTableSet::appendToCache(DMContext & context, const Block & block, size_t if (unlikely(!append_res.success)) throw Exception("Write to MemTableSet failed", ErrorCodes::LOGICAL_ERROR); } - rows += limit; - bytes += append_bytes; - allocated_bytes += append_res.new_alloc_bytes; + + stat.append( // + limit, + append_bytes, + append_res.new_alloc_bytes, + /*deletes_added*/ 0, + /*files_added*/ 0); } void MemTableSet::appendDeleteRange(const RowKeyRange & delete_range) @@ -294,13 +344,18 @@ ColumnFileSetSnapshotPtr MemTableSet::createSnapshot( // This may indicate that you forget to acquire a lock -- there are modifications // while this function is still running... RUNTIME_CHECK( - total_rows == rows && total_deletes == deletes, + total_rows == stat.rows && total_deletes == stat.deletes, total_rows, - rows.load(), + stat.rows.load(), total_deletes, - deletes.load()); - - return std::make_shared(data_provider, std::move(column_files_snap), rows, bytes, deletes); + stat.deletes.load()); + + return std::make_shared( + data_provider, + std::move(column_files_snap), + stat.rows, + stat.bytes, + stat.deletes); } ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( @@ -333,7 +388,7 @@ ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( cur_rows_offset += column_file->getRows(); cur_deletes_offset += column_file->getDeletes(); } - if (unlikely(flush_task->getFlushRows() != rows || flush_task->getFlushDeletes() != deletes)) + if (unlikely(flush_task->getFlushRows() != stat.rows || flush_task->getFlushDeletes() != stat.deletes)) { LOG_ERROR( log, @@ -341,8 +396,8 @@ ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( "Files: {}", flush_task->getFlushRows(), flush_task->getFlushDeletes(), - rows.load(), - deletes.load(), + stat.rows.load(), + stat.deletes.load(), ColumnFile::filesToString(column_files)); throw Exception("Rows and deletes check failed.", ErrorCodes::LOGICAL_ERROR); } @@ -378,11 +433,12 @@ void MemTableSet::removeColumnFilesInFlushTask(const ColumnFileFlushTask & flush new_deletes += column_file->getDeletes(); } column_files.swap(new_column_files); - column_files_count = column_files.size(); - rows = new_rows; - bytes = new_bytes; - allocated_bytes = new_alloc_bytes; - deletes = new_deletes; + stat.resetTo( // + column_files.size(), + new_rows, + new_bytes, + new_alloc_bytes, + new_deletes); } diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h index 72cd10f879d..bef950dd628 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h @@ -32,6 +32,46 @@ class MemTableSet : public std::enable_shared_from_this , private boost::noncopyable { +private: + struct Statistic + { + // TODO: check the proper memory_order when use this atomic variable + std::atomic column_files_count = 0; + std::atomic rows = 0; + std::atomic bytes = 0; + std::atomic allocated_bytes = 0; + std::atomic deletes = 0; + + CurrentMetrics::Increment holder_bytes; + CurrentMetrics::Increment holder_allocated_bytes; + + Statistic(); + + String info() const + { + return fmt::format( + "MemTableSet: {} column files, {} rows, {} bytes, {} deletes", + column_files_count.load(), + rows.load(), + bytes.load(), + deletes.load()); + } + + void append( + size_t rows_added, + size_t bytes_added, + size_t allocated_bytes_added, + size_t deletes_added, + size_t files_added); + + void resetTo( + size_t new_column_files_count, + size_t new_rows, + size_t new_bytes, + size_t new_allocated_bytes, + size_t new_deletes); + }; + #ifndef DBMS_PUBLIC_GTEST private: #else @@ -39,18 +79,13 @@ class MemTableSet #endif // Keep track of the number of mem-table in memory. CurrentMetrics::Increment holder_counter; - CurrentMetrics::Increment holder_allocated_bytes; // Note that we must update `column_files_count` for outer thread-safe after `column_files` changed ColumnFiles column_files; - // In order to avoid data-race, we use atomic variables to track the state of this MemTableSet. - // TODO: check the proper memory_order when use this atomic variable - std::atomic column_files_count; - std::atomic rows = 0; - std::atomic bytes = 0; - std::atomic allocated_bytes = 0; - std::atomic deletes = 0; + // In order to avoid data-race and make it lightweight for accessing the statistic + // of mem-table, we use atomic variables to track the state of this MemTableSet. + Statistic stat; LoggerPtr log; @@ -68,21 +103,13 @@ class MemTableSet void resetLogger(const LoggerPtr & segment_log) { log = segment_log; } /// Thread safe part start - String info() const - { - return fmt::format( - "MemTableSet: {} column files, {} rows, {} bytes, {} deletes", - column_files_count.load(), - rows.load(), - bytes.load(), - deletes.load()); - } - - size_t getColumnFileCount() const { return column_files_count.load(); } - size_t getRows() const { return rows.load(); } - size_t getBytes() const { return bytes.load(); } - size_t getAllocatedBytes() const { return allocated_bytes.load(); } - size_t getDeletes() const { return deletes.load(); } + String info() const { return stat.info(); } + + size_t getColumnFileCount() const { return stat.column_files_count.load(); } + size_t getRows() const { return stat.rows.load(); } + size_t getBytes() const { return stat.bytes.load(); } + size_t getAllocatedBytes() const { return stat.allocated_bytes.load(); } + size_t getDeletes() const { return stat.deletes.load(); } /// Thread safe part end /** diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp index 51d213c6e1a..2205de5a2cb 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -86,7 +86,6 @@ StoreStats DeltaMergeStore::getStoreStats() } } // access to `segments` end - stat.delta_rate_rows = static_cast(stat.total_delta_rows) / stat.total_rows; stat.delta_rate_segments = static_cast(stat.delta_count) / stat.segment_count; @@ -169,19 +168,23 @@ SegmentsStats DeltaMergeStore::getSegmentsStats() stat.delta_rate = static_cast(delta->getRows()) / stat.rows; { - const auto & delta_memtable = delta->getMemTableSet(); - const auto & delta_persisted = delta->getPersistedFileSet(); + // Keep a copy to the shared_ptr of MemTableSet + const auto delta_memtable = delta->getMemTableSet(); stat.delta_memtable_rows = delta_memtable->getRows(); stat.delta_memtable_size = delta_memtable->getBytes(); stat.delta_memtable_column_files = delta_memtable->getColumnFileCount(); stat.delta_memtable_delete_ranges = delta_memtable->getDeletes(); + stat.delta_cache_size = delta_memtable->getBytes(); // FIXME: this is the same as delta_memtable_size + stat.delta_cache_alloc_size = delta_memtable->getAllocatedBytes(); + } + { + // Keep a copy to the shared_ptr of PersistedFileSet + const auto delta_persisted = delta->getPersistedFileSet(); stat.delta_persisted_page_id = delta_persisted->getId(); stat.delta_persisted_rows = delta_persisted->getRows(); stat.delta_persisted_size = delta_persisted->getBytes(); stat.delta_persisted_column_files = delta_persisted->getColumnFileCount(); stat.delta_persisted_delete_ranges = delta_persisted->getDeletes(); - stat.delta_cache_size = delta_memtable->getBytes(); // FIXME: this is the same as delta_memtable_size - stat.delta_cache_alloc_size = delta_memtable->getAllocatedBytes(); } stat.delta_index_size = delta->getDeltaIndexBytes(); diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 9b93fbb14d3..f3a9fb73802 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -128,7 +128,7 @@ extern const Metric DT_SnapshotOfPlaceIndex; extern const Metric DT_SnapshotOfReplayVersionChain; extern const Metric DT_SnapshotOfSegmentIngest; extern const Metric DT_SnapshotOfBitmapFilter; -extern const Metric DT_NumSegments; +extern const Metric DT_NumSegment; } // namespace CurrentMetrics namespace DB @@ -296,7 +296,7 @@ Segment::Segment( // PageIdU64 next_segment_id_, const DeltaValueSpacePtr & delta_, const StableValueSpacePtr & stable_) - : holder_counter(CurrentMetrics::DT_NumSegments) + : holder_counter(CurrentMetrics::DT_NumSegment) , epoch(epoch_) , rowkey_range(rowkey_range_) , is_common_handle(rowkey_range.is_common_handle) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index cd7ed03b5ea..f63c0d98488 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -14,6 +14,10 @@ #include +namespace CurrentMetrics +{ +extern const Metric NumIStorage; +} // namespace CurrentMetrics namespace DB { @@ -24,6 +28,14 @@ extern const int DEADLOCK_AVOIDED; extern const int TABLE_IS_DROPPED; } // namespace ErrorCodes +IStorage::IStorage() + : holder_counter(CurrentMetrics::NumIStorage, 1) +{} + +IStorage::IStorage(ColumnsDescription columns_) + : ITableDeclaration(std::move(columns_)) + , holder_counter(CurrentMetrics::NumIStorage, 1) +{} RWLock::LockHolder IStorage::tryLockTimed( const RWLockPtr & rwlock, diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index d1af315a918..bb23b339d31 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -63,6 +63,10 @@ class IStorage , public ITableDeclaration { public: + IStorage(); + + explicit IStorage(ColumnsDescription columns_); + /// The main name of the table type (for example, StorageDeltaMerge). virtual std::string getName() const = 0; @@ -329,6 +333,8 @@ class IStorage /// DROP-like queries take this lock for write (lockExclusively), to be sure /// that all table threads finished. mutable RWLockPtr drop_lock = RWLock::create(); + + CurrentMetrics::Increment holder_counter; }; /// table name -> table diff --git a/dbms/src/Storages/ITableDeclaration.h b/dbms/src/Storages/ITableDeclaration.h index 8635da352f7..8296ada0da9 100644 --- a/dbms/src/Storages/ITableDeclaration.h +++ b/dbms/src/Storages/ITableDeclaration.h @@ -37,7 +37,7 @@ class ITableDeclaration Block getSampleBlockNonMaterialized() const; Block getSampleBlockForColumns(const Names & column_names) const; - /** The hidden coloumns will not be returned. Mainly for INSERT query. + /** The hidden columns will not be returned. Mainly for INSERT query. */ Block getSampleBlockNoHidden() const; Block getSampleBlockNonMaterializedNoHidden() const; diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 1f974f2d744..94b70fc5057 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -62,6 +62,10 @@ #include #include +namespace CurrentMetrics +{ +extern const Metric DT_NumStorageDeltaMerge; +} // namespace CurrentMetrics namespace DB { @@ -88,6 +92,7 @@ StorageDeltaMerge::StorageDeltaMerge( Timestamp tombstone, Context & global_context_) : IManageableStorage{columns_, tombstone} + , holder_counter(CurrentMetrics::DT_NumStorageDeltaMerge, 1) , store_inited(false) , max_column_id_used(0) , data_path_contains_database_name(db_engine != "TiFlash") diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 13b46f02c5b..48ed1502e95 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -275,6 +275,9 @@ class StorageDeltaMerge DM::ColumnDefine handle_column_define; }; + // Keep track of the number of StorageDeltaMerge in memory. + CurrentMetrics::Increment holder_counter; + mutable std::mutex store_mutex; std::unique_ptr table_column_info; // After create DeltaMergeStore object, it is deprecated. @@ -303,12 +306,11 @@ class StorageDeltaMerge // Used to allocate new column-id when this table is NOT synced from TiDB ColumnID max_column_id_used; - std::atomic shutdown_called{false}; - // TODO: remove the following two members, which are only used for debug from ch-client. Strings pk_column_names; std::atomic next_version = 1; + std::atomic shutdown_called{false}; bool is_common_handle = false; bool pk_is_handle = false; diff --git a/dbms/src/Storages/System/StorageSystemDTTables.cpp b/dbms/src/Storages/System/StorageSystemDTTables.cpp index 0a0097337e7..0ffa571281a 100644 --- a/dbms/src/Storages/System/StorageSystemDTTables.cpp +++ b/dbms/src/Storages/System/StorageSystemDTTables.cpp @@ -44,6 +44,7 @@ StorageSystemDTTables::StorageSystemDTTables(const std::string & name_) {"belonging_table_id", std::make_shared()}, {"is_tombstone", std::make_shared()}, + {"column_count", std::make_shared()}, {"segment_count", std::make_shared()}, {"total_rows", std::make_shared()}, @@ -168,6 +169,7 @@ BlockInputStreams StorageSystemDTTables::read( res_columns[j++]->insert(table_info.belonging_table_id); res_columns[j++]->insert(dm_storage->getTombstone()); + res_columns[j++]->insert(stat.column_count); res_columns[j++]->insert(stat.segment_count); res_columns[j++]->insert(stat.total_rows); diff --git a/dbms/src/TiDB/Schema/SchemaSyncService.cpp b/dbms/src/TiDB/Schema/SchemaSyncService.cpp index 3a85b66b45d..f755ededd8a 100644 --- a/dbms/src/TiDB/Schema/SchemaSyncService.cpp +++ b/dbms/src/TiDB/Schema/SchemaSyncService.cpp @@ -29,6 +29,11 @@ #include +namespace CurrentMetrics +{ +extern const Metric NumKeyspace; +} // namespace CurrentMetrics + namespace DB { namespace ErrorCodes @@ -73,6 +78,7 @@ void SchemaSyncService::addKeyspaceGCTasks() std::unique_lock lock(keyspace_map_mutex); for (auto const & iter : keyspaces) { + // Already exist auto keyspace = iter.first; if (keyspace_handle_map.contains(keyspace)) continue; @@ -126,6 +132,7 @@ void SchemaSyncService::addKeyspaceGCTasks() keyspace_handle_map.emplace(keyspace, task_handle); num_add_tasks += 1; + CurrentMetrics::add(CurrentMetrics::NumKeyspace, 1); } auto log_level = num_add_tasks > 0 ? Poco::Message::PRIO_INFORMATION : Poco::Message::PRIO_DEBUG; @@ -158,6 +165,7 @@ void SchemaSyncService::removeKeyspaceGCTasks() PDClientHelper::removeKeyspaceGCSafepoint(keyspace); keyspace_gc_context.erase(keyspace); // clear the last gc safepoint num_remove_tasks += 1; + CurrentMetrics::sub(CurrentMetrics::NumKeyspace, 1); } auto log_level = num_remove_tasks > 0 ? Poco::Message::PRIO_INFORMATION : Poco::Message::PRIO_DEBUG; From a2d2be50f9eb52b1e5b05f68147b175678e0dbf4 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Mon, 7 Jul 2025 13:08:25 +0800 Subject: [PATCH 08/10] Reset the S3GC owner metrics Signed-off-by: JaySon-Huang --- dbms/src/Interpreters/AsynchronousMetrics.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index e2c77312d3b..b3b67b969f3 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -312,6 +312,11 @@ void AsynchronousMetrics::update() { GET_METRIC(tiflash_storage_s3_gc_status, type_owner).Set(1.0); } + else + { + // If the current node is not the owner, we reset the metric to 0 + GET_METRIC(tiflash_storage_s3_gc_status, type_owner).Set(0.0); + } } #if USE_MIMALLOC From adc685247df0758673c5566f5baac742336609d2 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Mon, 7 Jul 2025 14:49:36 +0800 Subject: [PATCH 09/10] Update tiflash_summary Signed-off-by: JaySon-Huang --- metrics/grafana/tiflash_summary.json | 4878 +++++++++++++++++--------- 1 file changed, 3258 insertions(+), 1620 deletions(-) diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 9af86542b41..12845fb01cb 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -2916,6 +2916,7 @@ "alertThreshold": true }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -3153,7 +3154,7 @@ "x": 0, "y": 2 }, - "id": 6, + "id": 327, "panels": [ { "aliasColors": {}, @@ -3161,11 +3162,12 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, @@ -3174,7 +3176,7 @@ "y": 3 }, "hiddenSeries": false, - "id": 9, + "id": 329, "legend": { "alignAsTable": true, "avg": false, @@ -3183,20 +3185,23 @@ "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "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, - "points": false, + "pointradius": 2, + "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -3204,18 +3209,30 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_coprocessor_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "exemplar": true, + "expr": "sum(tiflash_proxy_threads_state{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (instance, state)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" + "legendFormat": "{{instance}}-{{state}}", + "refId": "A", + "step": 4 + }, + { + "exemplar": true, + "expr": "sum(tiflash_proxy_threads_state{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}-total", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request QPS", + "title": "Threads state", "tooltip": { "shared": true, "sort": 0, @@ -3231,16 +3248,15 @@ }, "yaxes": [ { - "decimals": null, "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3259,11 +3275,12 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, @@ -3272,7 +3289,7 @@ "y": 3 }, "hiddenSeries": false, - "id": 2, + "id": 331, "legend": { "alignAsTable": true, "avg": false, @@ -3281,20 +3298,23 @@ "min": false, "rightSide": true, "show": true, + "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, - "points": false, + "pointradius": 2, + "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -3302,18 +3322,22 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_coprocessor_executor_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "exemplar": true, + "expr": "sum(rate(tiflash_proxy_threads_io_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name, io) > 1024", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" + "legendFormat": "{{name}}-{{io}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Executor QPS", + "title": "Threads IO", "tooltip": { "shared": true, "sort": 0, @@ -3329,15 +3353,15 @@ }, "yaxes": [ { - "format": "none", + "format": "Bps", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3356,6 +3380,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] @@ -3369,29 +3394,32 @@ "y": 10 }, "hiddenSeries": false, - "id": 11, + "id": 333, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, "total": false, - "values": 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, - "points": false, + "pointradius": 2, + "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -3400,43 +3428,21 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_proxy_thread_voluntary_context_switches{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (instance, name) > 200", "format": "time_series", + "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "999-{{type}}", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "99-{{type}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "95-{{type}}", - "refId": "C" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" + "legendFormat": "{{instance}} - {{name}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request Duration", + "title": "Thread Voluntary Context Switches", "tooltip": { "shared": true, "sort": 0, @@ -3452,11 +3458,11 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -3479,11 +3485,12 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, @@ -3492,7 +3499,7 @@ "y": 10 }, "hiddenSeries": false, - "id": 12, + "id": 335, "legend": { "alignAsTable": true, "avg": false, @@ -3501,20 +3508,23 @@ "min": false, "rightSide": true, "show": true, + "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, - "points": false, + "pointradius": 2, + "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -3522,18 +3532,22 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_coprocessor_request_error{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (reason)", + "exemplar": true, + "expr": "sum(rate(tiflash_proxy_thread_nonvoluntary_context_switches{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (instance, name) > 50", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "{{reason}}", - "refId": "A" + "legendFormat": "{{instance}} - {{name}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Error QPS", + "title": "Thread Nonvoluntary Context Switches", "tooltip": { "shared": true, "sort": 0, @@ -3553,11 +3567,11 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3569,7 +3583,22 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Threads", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 3 + }, + "id": 6, + "panels": [ { "aliasColors": {}, "bars": false, @@ -3580,26 +3609,26 @@ "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 17 + "y": 4 }, "hiddenSeries": false, - "id": 13, + "id": 9, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -3619,39 +3648,18 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_coprocessor_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "999-{{type}}", + "legendFormat": "{{type}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "99-{{type}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "95-{{type}}", - "refId": "C" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request Handle Duration", + "title": "Request QPS", "tooltip": { "shared": true, "sort": 0, @@ -3667,7 +3675,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": null, + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3675,7 +3684,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3704,10 +3713,10 @@ "h": 7, "w": 12, "x": 12, - "y": 17 + "y": 4 }, "hiddenSeries": false, - "id": 14, + "id": 2, "legend": { "alignAsTable": true, "avg": false, @@ -3737,10 +3746,8 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_coprocessor_response_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_coprocessor_executor_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "interval": "", "intervalFactor": 1, "legendFormat": "{{type}}", "refId": "A" @@ -3750,7 +3757,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Response Bytes/Seconds", + "title": "Executor QPS", "tooltip": { "shared": true, "sort": 0, @@ -3766,7 +3773,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3774,7 +3781,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3803,15 +3810,17 @@ "h": 7, "w": 12, "x": 0, - "y": 24 + "y": 11 }, "hiddenSeries": false, - "id": 63, + "id": 11, "legend": { + "alignAsTable": false, "avg": false, "current": false, "max": false, "min": false, + "rightSide": false, "show": true, "total": false, "values": false @@ -3825,7 +3834,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3834,29 +3843,34 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "interval": "", "intervalFactor": 1, "legendFormat": "999-{{type}}", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "hide": true, "intervalFactor": 1, "legendFormat": "99-{{type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "hide": true, "intervalFactor": 1, "legendFormat": "95-{{type}}", "refId": "C" }, { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "hide": true, "intervalFactor": 1, "legendFormat": "80-{{type}}", "refId": "D" @@ -3866,7 +3880,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Cop task memory usage", + "title": "Request Duration", "tooltip": { "shared": true, "sort": 0, @@ -3882,7 +3896,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -3919,10 +3933,10 @@ "h": 7, "w": 12, "x": 12, - "y": 24 + "y": 11 }, "hiddenSeries": false, - "id": 165, + "id": 12, "legend": { "alignAsTable": true, "avg": false, @@ -3952,12 +3966,10 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_exchange_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_coprocessor_request_error{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (reason)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{reason}}", "refId": "A" } ], @@ -3965,7 +3977,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Exchange Bytes/Seconds", + "title": "Error QPS", "tooltip": { "shared": true, "sort": 0, @@ -3981,7 +3993,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3989,7 +4001,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4012,26 +4024,26 @@ "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 31 + "y": 18 }, "hiddenSeries": false, - "id": 100, + "id": 13, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -4051,20 +4063,39 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type!~\".*max\"}) by (instance, type)", + "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "999-{{type}}", "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Threads of Rpc", + "title": "Request Handle Duration", "tooltip": { "shared": true, "sort": 0, @@ -4080,7 +4111,7 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -4117,10 +4148,10 @@ "h": 7, "w": 12, "x": 12, - "y": 31 + "y": 18 }, "hiddenSeries": false, - "id": 77, + "id": 14, "legend": { "alignAsTable": true, "avg": false, @@ -4150,8 +4181,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tiflash_coprocessor_handling_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", + "exemplar": true, + "expr": "sum(rate(tiflash_coprocessor_response_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", + "interval": "", "intervalFactor": 1, "legendFormat": "{{type}}", "refId": "A" @@ -4161,7 +4194,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Handling Request Number", + "title": "Response Bytes/Seconds", "tooltip": { "shared": true, "sort": 0, @@ -4177,8 +4210,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4186,7 +4218,7 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -4209,26 +4241,24 @@ "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 38 + "y": 25 }, "hiddenSeries": false, - "id": 102, + "id": 63, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -4239,7 +4269,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4248,20 +4278,39 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\".*max\", type!~\"rpc.*\"}) by (instance, type)", + "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "999-{{type}}", "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Threads", + "title": "Cop task memory usage", "tooltip": { "shared": true, "sort": 0, @@ -4277,7 +4326,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4314,10 +4363,10 @@ "h": 7, "w": 12, "x": 12, - "y": 38 + "y": 25 }, "hiddenSeries": false, - "id": 101, + "id": 165, "legend": { "alignAsTable": true, "avg": false, @@ -4348,11 +4397,11 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type=~\".*max\"}) by (instance, type)", + "expr": "sum(rate(tiflash_exchange_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{type}}", "refId": "A" } ], @@ -4360,7 +4409,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Max Threads of Rpc", + "title": "Exchange Bytes/Seconds", "tooltip": { "shared": true, "sort": 0, @@ -4376,7 +4425,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4403,7 +4452,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The MPP query count in TiFlash", "fieldConfig": { "defaults": {}, "overrides": [] @@ -4414,10 +4462,10 @@ "h": 7, "w": 12, "x": 0, - "y": 45 + "y": 32 }, "hiddenSeries": false, - "id": 157, + "id": 100, "legend": { "alignAsTable": true, "avg": false, @@ -4448,7 +4496,7 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_mpp_task_manager{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance, type)", + "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type!~\".*max\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -4460,7 +4508,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MPP Query count", + "title": "Threads of Rpc", "tooltip": { "shared": true, "sort": 0, @@ -4513,10 +4561,10 @@ "h": 7, "w": 12, "x": 12, - "y": 45 + "y": 32 }, "hiddenSeries": false, - "id": 103, + "id": 77, "legend": { "alignAsTable": true, "avg": false, @@ -4546,12 +4594,10 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\".*max\", type!~\"rpc.*\"}) by (instance, type)", + "expr": "sum(tiflash_coprocessor_handling_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{type}}", "refId": "A" } ], @@ -4559,7 +4605,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Max Threads", + "title": "Handling Request Number", "tooltip": { "shared": true, "sort": 0, @@ -4575,6 +4621,7 @@ }, "yaxes": [ { + "decimals": null, "format": "none", "label": null, "logBase": 1, @@ -4583,7 +4630,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4612,10 +4659,10 @@ "h": 7, "w": 12, "x": 0, - "y": 52 + "y": 39 }, "hiddenSeries": false, - "id": 199, + "id": 102, "legend": { "alignAsTable": true, "avg": false, @@ -4646,11 +4693,11 @@ "targets": [ { "exemplar": true, - "expr": "tiflash_mpp_task_monitor{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\".*max\", type!~\"rpc.*\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4658,7 +4705,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Time of the Longest Live MPP Task", + "title": "Threads", "tooltip": { "shared": true, "sort": 0, @@ -4674,7 +4721,7 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4711,10 +4758,10 @@ "h": 7, "w": 12, "x": 12, - "y": 52 + "y": 39 }, "hiddenSeries": false, - "id": 166, + "id": 101, "legend": { "alignAsTable": true, "avg": false, @@ -4745,11 +4792,11 @@ "targets": [ { "exemplar": true, - "expr": "tiflash_exchange_queueing_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type=~\".*max\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4757,7 +4804,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Data size in send and receive queue", + "title": "Max Threads of Rpc", "tooltip": { "shared": true, "sort": 0, @@ -4773,7 +4820,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4800,6 +4847,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The MPP query count in TiFlash", "fieldConfig": { "defaults": {}, "overrides": [] @@ -4810,10 +4858,10 @@ "h": 7, "w": 12, "x": 0, - "y": 59 + "y": 46 }, "hiddenSeries": false, - "id": 297, + "id": 157, "legend": { "alignAsTable": true, "avg": false, @@ -4844,11 +4892,11 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_network_transmission_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "max(tiflash_mpp_task_manager{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4856,7 +4904,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Network Transmission", + "title": "MPP Query count", "tooltip": { "shared": true, "sort": 0, @@ -4872,7 +4920,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4899,7 +4947,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The establish calldata details", "fieldConfig": { "defaults": {}, "overrides": [] @@ -4910,10 +4957,10 @@ "h": 7, "w": 12, "x": 12, - "y": 59 + "y": 46 }, "hiddenSeries": false, - "id": 299, + "id": 103, "legend": { "alignAsTable": true, "avg": false, @@ -4944,7 +4991,7 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_establish_calldata_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type != \"new_request_calldata\"}) by (instance, type)", + "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\".*max\", type!~\"rpc.*\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -4956,7 +5003,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Establish calldata details", + "title": "Max Threads", "tooltip": { "shared": true, "sort": 0, @@ -4992,66 +5039,49 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Coprocessor", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 3 - }, - "id": 105, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the min_tso of each instance", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 4 + "y": 53 }, "hiddenSeries": false, - "id": 107, + "id": 199, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, "total": false, "values": true }, - "lines": false, + "lines": true, "linewidth": 1, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { - "alertThreshold": false + "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 1, - "points": true, + "pointradius": 5, + "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -5060,10 +5090,11 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"min_tso\"}) by (instance, resource_group)", + "expr": "tiflash_mpp_task_monitor{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}", "refId": "A" } ], @@ -5071,7 +5102,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Min TSO", + "title": "Time of the Longest Live MPP Task", "tooltip": { "shared": true, "sort": 0, @@ -5087,13 +5118,12 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "TSO", + "format": "s", + "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true }, { "format": "short", @@ -5115,29 +5145,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "estimated thread usage in min-tso scheduler, and the sort/hard limit of estimated thread in scheduler.", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 4 + "y": 53 }, "hiddenSeries": false, - "id": 109, + "id": 166, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5146,13 +5172,14 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 1, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -5162,54 +5189,19 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_soft_limit\"}) by (instance, type, resource_group)", + "expr": "tiflash_exchange_queueing_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{type}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"estimated_thread_usage\"}) by (instance, type, resource_group)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", - "refId": "B" - }, - { - "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_hard_limit\"}) by (instance, type, resource_group)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", - "refId": "C" - }, - { - "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"global_estimated_thread_usage\"}) by (instance, type, resource_group)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", - "refId": "D" - }, - { - "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"group_entry_count\"}) by (instance, type)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-{{type}}", - "queryType": "randomWalk", - "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Estimated Thread Usage and Limit", + "title": "Data size in send and receive queue", "tooltip": { "shared": true, "sort": 0, @@ -5225,12 +5217,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "Threads", - "logBase": 10, + "format": "bytes", + "label": null, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5253,29 +5244,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the count of active/ waiting queries", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 12 + "y": 60 }, "hiddenSeries": false, - "id": 111, + "id": 297, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5284,13 +5271,14 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 1, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -5300,27 +5288,19 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_queries_count\"}) by (instance, type, resource_group)", + "expr": "sum(rate(tiflash_network_transmission_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{type}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_queries_count\"}) by (instance, type, resource_group)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Active and Waiting Queries Count", + "title": "Network Transmission", "tooltip": { "shared": true, "sort": 0, @@ -5336,12 +5316,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "Queries", + "format": "bytes", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5364,29 +5343,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the count of active/ waiting tasks", + "description": "The establish calldata details", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 12 + "y": 60 }, "hiddenSeries": false, - "id": 113, + "id": 299, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5395,13 +5371,14 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 1, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -5411,27 +5388,19 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_tasks_count\"}) by (instance, type, resource_group)", + "expr": "max(tiflash_establish_calldata_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type != \"new_request_calldata\"}) by (instance, type)", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_tasks_count\"}) by (instance, type, resource_group)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Active and Waiting Tasks Count", + "title": "Establish calldata details", "tooltip": { "shared": true, "sort": 0, @@ -5447,12 +5416,11 @@ }, "yaxes": [ { - "decimals": null, "format": "none", - "label": "Tasks", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5468,7 +5436,23 @@ "align": false, "alignLevel": null } - }, + } + ], + "repeat": null, + "title": "Coprocessor", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 105, + "panels": [ { "aliasColors": {}, "bars": false, @@ -5476,23 +5460,21 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": null, - "description": "the usage of estimated threads exceeded the hard limit where errors occur.", + "description": "the min_tso of each instance", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 20 + "y": 5 }, "hiddenSeries": false, - "id": 117, + "id": 107, "legend": { "alignAsTable": false, "avg": false, @@ -5504,16 +5486,16 @@ "total": false, "values": true }, - "lines": true, + "lines": false, "linewidth": 1, - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { - "alertThreshold": true + "alertThreshold": false }, "percentage": false, "pluginVersion": "7.5.11", "pointradius": 1, - "points": false, + "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -5522,19 +5504,18 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"hard_limit_exceeded_count\"}) by (instance, type, resource_group)", - "hide": false, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"min_tso\"}) by (instance, resource_group)", "interval": "", "legendFormat": "{{instance}}-{{resource_group}}", "queryType": "randomWalk", - "refId": "B" + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Hard Limit Exceeded Count", + "title": "Min TSO", "tooltip": { "shared": true, "sort": 0, @@ -5552,11 +5533,11 @@ { "decimals": null, "format": "none", - "label": "", + "label": "TSO", "logBase": 1, "max": null, "min": null, - "show": true + "show": false }, { "format": "short", @@ -5579,9 +5560,11 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": null, - "description": "the time of waiting for schedule", + "description": "estimated thread usage in min-tso scheduler, and the sort/hard limit of estimated thread in scheduler.", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, @@ -5590,10 +5573,10 @@ "h": 8, "w": 12, "x": 12, - "y": 20 + "y": 5 }, "hiddenSeries": false, - "id": 115, + "id": 109, "legend": { "alignAsTable": false, "avg": false, @@ -5623,37 +5606,54 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.80, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", - "hide": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_soft_limit\"}) by (instance, type, resource_group)", "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}-80", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "A" }, { "exemplar": true, - "expr": "histogram_quantile(0.90, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", - "hide": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"estimated_thread_usage\"}) by (instance, type, resource_group)", + "hide": false, "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}-90", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "B" }, { "exemplar": true, - "expr": "histogram_quantile(1.00, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_hard_limit\"}) by (instance, type, resource_group)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}-100", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "C" + }, + { + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"global_estimated_thread_usage\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", + "refId": "D" + }, + { + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"group_entry_count\"}) by (instance, type)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "queryType": "randomWalk", + "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Task Waiting Duration", + "title": "Estimated Thread Usage and Limit", "tooltip": { "shared": true, "sort": 0, @@ -5670,9 +5670,9 @@ "yaxes": [ { "decimals": null, - "format": "s", - "label": "Time", - "logBase": 1, + "format": "none", + "label": "Threads", + "logBase": 10, "max": null, "min": null, "show": true @@ -5690,65 +5690,51 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Task Scheduler", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 4 - }, - "id": 16, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Executed DDL jobs per minute", + "decimals": null, + "description": "the count of active/ waiting queries", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 5 + "y": 13 }, "hiddenSeries": false, - "id": 19, + "id": 111, "legend": { "alignAsTable": false, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -5757,41 +5743,28 @@ "steppedLine": false, "targets": [ { - "expr": "avg(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{type}}", + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_queries_count\"}) by (instance, type, resource_group)", + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "A" }, { - "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "total", + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_queries_count\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "B" - }, - { - "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type,instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{type}}-{{instance}}", - "refId": "C" - }, - { - "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "total-{{instance}}", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Internal DDL OPM", + "title": "Active and Waiting Queries Count", "tooltip": { "shared": true, "sort": 0, @@ -5808,15 +5781,15 @@ "yaxes": [ { "decimals": null, - "format": "opm", - "label": null, + "format": "none", + "label": "Queries", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5835,42 +5808,44 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Executed DDL apply jobs per minute", + "decimals": null, + "description": "the count of active/ waiting tasks", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 5 + "y": 13 }, "hiddenSeries": false, - "id": 18, + "id": 113, "legend": { "alignAsTable": false, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -5879,18 +5854,28 @@ "steppedLine": false, "targets": [ { - "expr": "avg(increase(tiflash_schema_trigger_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "triggle-by-{{type}}", + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_tasks_count\"}) by (instance, type, resource_group)", + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "A" + }, + { + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_tasks_count\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Apply OPM", + "title": "Active and Waiting Tasks Count", "tooltip": { "shared": true, "sort": 0, @@ -5907,15 +5892,15 @@ "yaxes": [ { "decimals": null, - "format": "opm", - "label": null, + "format": "none", + "label": "Tasks", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5934,97 +5919,66 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": null, + "description": "the usage of estimated threads exceeded the hard limit where errors occur.", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 12 + "y": 21 }, "hiddenSeries": false, - "id": 20, + "id": 117, "legend": { "alignAsTable": false, "avg": false, - "current": false, - "hideZero": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/^applying/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "999-{{type}}", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "99-{{type}}", + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"hard_limit_exceeded_count\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "95-{{type}}", - "refId": "C" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", keyspace!=\"\"}[1m])) by (le, type, keyspace))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" - }, - { - "expr": "sum(tiflash_sync_schema_applying{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"$type\"}) by (instance)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "applying-{{instance}}", - "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Apply Duration", + "title": "Hard Limit Exceeded Count", "tooltip": { "shared": true, "sort": 0, @@ -6040,19 +5994,20 @@ }, "yaxes": [ { - "format": "s", - "label": null, + "decimals": null, + "format": "none", + "label": "", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { "format": "short", "label": null, "logBase": 1, - "max": "2", - "min": "0", + "max": null, + "min": null, "show": true } ], @@ -6060,30 +6015,15 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "DDL", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 5 - }, - "id": 25, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total count of different kinds of commands received", + "decimals": null, + "description": "the time of waiting for schedule", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6093,66 +6033,71 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 6 + "x": 12, + "y": 21 }, "hiddenSeries": false, - "id": 41, + "id": 115, "legend": { "alignAsTable": false, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/delete_range|ingest/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "{{type}}", + "exemplar": true, + "expr": "histogram_quantile(0.80, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", + "hide": true, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}-80", + "queryType": "randomWalk", "refId": "A" }, { - "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "write block", + "exemplar": true, + "expr": "histogram_quantile(0.90, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", + "hide": true, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}-90", + "queryType": "randomWalk", "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(1.00, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}-100", + "queryType": "randomWalk", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Command OPS", + "title": "Task Waiting Duration", "tooltip": { "shared": true, "sort": 0, @@ -6169,19 +6114,19 @@ "yaxes": [ { "decimals": null, - "format": "ops", - "label": null, + "format": "s", + "label": "Time", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -6189,14 +6134,30 @@ "align": false, "alignLevel": null } - }, + } + ], + "repeat": null, + "title": "Task Scheduler", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 16, + "panels": [ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Executed DDL jobs per minute", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6204,28 +6165,28 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, + "x": 0, "y": 6 }, "hiddenSeries": false, - "id": 38, + "id": 19, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, + "current": false, "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -6234,88 +6195,52 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/fs|write/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum by (instance) (\ntiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} +\ntiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}\n)\n/\nsum by (instance) (\ntiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}\n)", + "expr": "avg(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "hide": false, - "interval": "", "intervalFactor": 1, - "legendFormat": "amp-total-{{instance}}", + "legendFormat": "{{type}}", "refId": "A" }, { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m])\n)\n/\nsum by (instance) (\nrate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[5m])\n)", + "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", - "hide": false, - "interval": "", "intervalFactor": 1, - "legendFormat": "amp-5min-{{instance}}", + "legendFormat": "total", "refId": "B" }, { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[10m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[10m])\n)\n/\nsum by (instance) (\nrate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[10m])\n)", + "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type,instance)", "format": "time_series", "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "amp-10min-{{instance}}", + "legendFormat": "{{type}}-{{instance}}", "refId": "C" }, { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30m])\n)\n/\nsum by (instance) (\nrate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[30m])\n)", + "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "amp-30min-{{instance}}", + "legendFormat": "total-{{instance}}", "refId": "D" - }, - { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m])\n)", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "fs-5min-{{instance}}", - "refId": "E" - }, - { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[5m])\n)", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "write-5min-{{instance}}", - "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Amplification", + "title": "Schema Internal DDL OPM", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, - "transformations": [], "type": "graph", "xaxis": { "buckets": null, @@ -6327,15 +6252,15 @@ "yaxes": [ { "decimals": null, - "format": "short", + "format": "opm", "label": null, "logBase": 1, - "max": "20", + "max": null, "min": "0", "show": true }, { - "format": "binBps", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -6354,7 +6279,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage engine read tasks", + "description": "Executed DDL apply jobs per minute", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6362,23 +6287,23 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, - "w": 24, - "x": 0, - "y": 14 + "h": 7, + "w": 12, + "x": 12, + "y": 6 }, "hiddenSeries": false, - "id": 40, + "id": 18, "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -6398,11 +6323,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_read_tasks_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "avg(increase(tiflash_schema_trigger_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}", + "intervalFactor": 1, + "legendFormat": "triggle-by-{{type}}", "refId": "A" } ], @@ -6410,7 +6334,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Tasks OPS", + "title": "Schema Apply OPM", "tooltip": { "shared": true, "sort": 0, @@ -6427,8 +6351,8 @@ "yaxes": [ { "decimals": null, - "format": "ops", - "label": "", + "format": "opm", + "label": null, "logBase": 1, "max": null, "min": "0", @@ -6454,43 +6378,36 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of (maybe foreground) tasks of storage in bytes", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 22 + "y": 13 }, - "height": "", "hiddenSeries": false, - "id": 258, + "id": 20, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "hideEmpty": false, + "current": false, "hideZero": false, - "max": true, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -6499,10 +6416,9 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/total/", + "alias": "/^applying/", "yaxis": 2 } ], @@ -6511,25 +6427,51 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_subtask_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "histogram_quantile(0.999, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", - "hide": false, - "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A", - "step": 10 + "legendFormat": "999-{{type}}", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", keyspace!=\"\"}[1m])) by (le, type, keyspace))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "refId": "D" + }, + { + "expr": "sum(tiflash_sync_schema_applying{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"$type\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "applying-{{instance}}", + "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "SubTasks Write Throughput (bytes)", + "title": "Schema Apply Duration", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6542,7 +6484,7 @@ }, "yaxes": [ { - "format": "binBps", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -6550,48 +6492,63 @@ "show": true }, { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], + "max": "2", + "min": "0", + "show": true + } + ], "yaxis": { "align": false, "alignLevel": null } - }, + } + ], + "repeat": null, + "title": "DDL", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 6 + }, + "id": 307, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of (maybe foreground) tasks of storage in rows", + "description": "TiFlash CPU usage calculated with process CPU running seconds.", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 22 + "x": 0, + "y": 7 }, - "height": "", "hiddenSeries": false, - "id": 259, + "id": 310, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, "max": true, "min": false, "rightSide": true, @@ -6614,11 +6571,13 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/total/", - "yaxis": 2 + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2 } ], "spaceLength": 10, @@ -6627,24 +6586,33 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_subtask_throughput_rows{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "irate(tiflash_proxy_process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tiflash\"}[1m])", "format": "time_series", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "{{type}}", + "intervalFactor": 2, + "legendFormat": "{{instance}}", "refId": "A", - "step": 10 + "step": 40 + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_LogicalCPUCores{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "interval": "", + "intervalFactor": 1, + "legendFormat": "limit-{{instance}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "SubTasks Write Throughput (rows)", + "title": "CPU Usage (irate)", "tooltip": { + "msResolution": false, "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6657,7 +6625,8 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": 1, + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -6665,7 +6634,7 @@ "show": true }, { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -6684,37 +6653,43 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage's internal sub tasks", + "decimals": null, + "description": "", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { - "h": 5, + "h": 8, "w": 12, - "x": 0, - "y": 30 + "x": 12, + "y": 7 }, "hiddenSeries": false, - "id": 39, + "id": 309, "legend": { "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -6723,28 +6698,49 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "Limit", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2, + "nullPointMode": "connected" + } + ], "spaceLength": 10, "stack": false, - "steppedLine": true, + "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", + "expr": "sum by (instance) (rate(tiflash_proxy_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"SegmentReader.*\"}[1m]))", "format": "time_series", "hide": false, + "instant": false, "interval": "", "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A" + "legendFormat": "{{name}} {{instance}}", + "refId": "A", + "step": 40 + }, + { + "exemplar": true, + "expr": "count by (instance) (tiflash_proxy_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"SegmentReader.*\"})", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "Limit", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Small Internal Tasks OPS", + "title": "Segment Reader", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -6760,7 +6756,7 @@ "yaxes": [ { "decimals": 1, - "format": "ops", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -6768,11 +6764,11 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": false } ], @@ -6787,7 +6783,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6795,13 +6790,13 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 8, "w": 12, - "x": 12, - "y": 30 + "x": 0, + "y": 15 }, "hiddenSeries": false, - "id": 42, + "id": 316, "legend": { "alignAsTable": true, "avg": false, @@ -6810,8 +6805,8 @@ "min": false, "rightSide": true, "show": true, - "sort": null, - "sortDesc": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -6833,13 +6828,12 @@ "steppedLine": false, "targets": [ { - "exemplar": false, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval]))) by (le,type) / 1000000000)", + "exemplar": true, + "expr": "sum(rate(tiflash_coprocessor_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type, instance)", "format": "time_series", - "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "max-{{type}}", + "intervalFactor": 1, + "legendFormat": "{{type}}-{{instance}}", "refId": "A" } ], @@ -6847,10 +6841,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Small Internal Tasks Duration", + "title": "Request QPS by instance", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -6863,8 +6857,8 @@ }, "yaxes": [ { - "decimals": 1, - "format": "s", + "decimals": null, + "format": "none", "label": null, "logBase": 1, "max": null, @@ -6872,12 +6866,12 @@ "show": true }, { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": false + "min": null, + "show": true } ], "yaxis": { @@ -6891,37 +6885,43 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage's internal sub tasks", + "decimals": 1, + "description": "The flow of different kinds of read operations", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 8, "w": 12, - "x": 0, - "y": 35 + "x": 12, + "y": 15 }, + "height": "", "hiddenSeries": false, - "id": 130, + "id": 314, "legend": { "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -6930,30 +6930,50 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": true, + "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", + "expr": "sum(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A" + "legendFormat": "File Descriptor-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_PSMReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Page-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "PageBackGround-{{instance}}", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Large Internal Tasks OPS", + "title": "Read Throughput by instance", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -6966,8 +6986,7 @@ }, "yaxes": [ { - "decimals": 1, - "format": "ops", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -6975,12 +6994,12 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, "min": "0", - "show": false + "show": true } ], "yaxis": { @@ -6994,7 +7013,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of storage's internal sub tasks", + "description": "The total count of different kinds of commands received", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7002,13 +7021,13 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 8, "w": 12, - "x": 12, - "y": 35 + "x": 0, + "y": 23 }, "hiddenSeries": false, - "id": 131, + "id": 318, "legend": { "alignAsTable": true, "avg": false, @@ -7017,8 +7036,6 @@ "min": false, "rightSide": true, "show": true, - "sort": null, - "sortDesc": null, "total": false, "values": true }, @@ -7034,29 +7051,40 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/delete_range|ingest/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval]))) by (le,type) / 1000000000)", + "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", "format": "time_series", "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "max-{{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Large Internal Tasks Duration", - "tooltip": { - "shared": true, + "intervalFactor": 1, + "legendFormat": "write block-{{instance}}", + "refId": "C" + }, + { + "expr": "sum(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{type}}-{{instance}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Command OPS By Instance", + "tooltip": { + "shared": true, "sort": 0, "value_type": "individual" }, @@ -7070,8 +7098,8 @@ }, "yaxes": [ { - "decimals": 1, - "format": "s", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -7079,12 +7107,12 @@ "show": true }, { - "format": "s", + "format": "opm", "label": null, "logBase": 1, "max": null, "min": "0", - "show": false + "show": true } ], "yaxis": { @@ -7098,7 +7126,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of currently opened file descriptors.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", + "decimals": 1, + "description": "The throughput of write by instance", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7106,13 +7135,14 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 40 + "h": 8, + "w": 12, + "x": 12, + "y": 23 }, + "height": "", "hiddenSeries": false, - "id": 50, + "id": 312, "legend": { "alignAsTable": true, "avg": false, @@ -7141,55 +7171,42 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ - { - "expr": "tiflash_proxy_process_open_fds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tiflash\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{instance}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_OpenFileForWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "W-{{instance}}", - "refId": "B" - }, { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_OpenFileForRead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write\"}[1m])) by (instance)", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "R-{{instance}}", - "refId": "C" + "legendFormat": "write-{{instance}}", + "refId": "A", + "step": 10 }, { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_OpenFileForReadWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "format": "time_series", + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest\"}[1m])) by (instance)", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "RW-{{instance}}", - "refId": "D" + "legendFormat": "ingest-{{instance}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Opened File Count", + "title": "Write Throughput By Instance", "tooltip": { "shared": true, "sort": 0, @@ -7205,7 +7222,7 @@ }, "yaxes": [ { - "format": "none", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -7213,7 +7230,7 @@ "show": true }, { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -7225,41 +7242,51 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Imbalance read/write", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 7 + }, + "id": 321, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of open file descriptors action.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 40 + "h": 8, + "w": 12, + "x": 0, + "y": 8 }, "hiddenSeries": false, - "id": 22, + "id": 325, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, @@ -7272,7 +7299,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -7282,31 +7309,22 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_FileOpen{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "Open-{{instance}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_FileOpenFailed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "tiflash_system_current_metric_NumKeyspace{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "OpenFail-{{instance}}", - "refId": "B" + "legendFormat": "keyspace-{{instance}}", + "refId": "I" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "File Open OPS", + "title": "Number of Keyspaces", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -7319,15 +7337,15 @@ }, "yaxes": [ { - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -7346,21 +7364,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "OPS and duration of fsync operations.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 40 + "h": 8, + "w": 12, + "x": 12, + "y": 8 }, "hiddenSeries": false, - "id": 52, + "id": 324, "legend": { "alignAsTable": true, "avg": false, @@ -7369,60 +7387,54 @@ "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "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": [ - { - "alias": "/max-fsync/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_FileFSync{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "tiflash_system_current_metric_DT_NumStorageDeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "ops-fsync-{{instance}}", - "refId": "A" + "legendFormat": "tables-{{instance}}", + "refId": "I" }, { "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_system_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"fsync\"}[$__rate_interval]))) by (le, instance) / 1000000000)", - "hide": false, + "expr": "tiflash_system_current_metric_NumIStorage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, "interval": "", - "legendFormat": "max-fsync-{{instance}}", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "tables-all-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "FSync Status", + "title": "Number of Physical Tables", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -7435,12 +7447,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -7449,7 +7460,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7463,27 +7474,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of different kinds of read operations", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 47 + "y": 16 }, "hiddenSeries": false, - "id": 46, + "id": 319, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -7494,13 +7503,13 @@ "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": [], @@ -7509,33 +7518,30 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_PSMWriteIOCalls{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "Page", - "refId": "A" - }, - { - "expr": "sum(rate(tiflash_system_profile_event_PSMWritePages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "exemplar": true, + "expr": "tiflash_system_current_metric_DT_NumSegment{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "PageFile", - "refId": "C" + "interval": "", + "intervalFactor": 1, + "legendFormat": "segments-{{instance}}", + "refId": "I" }, { - "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "exemplar": true, + "expr": "tiflash_system_current_metric_DT_NumMemTable{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "File Descriptor", - "refId": "D" + "legendFormat": "mem_table-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Disk Write OPS", + "title": "Number of Segments", "tooltip": { "shared": true, "sort": 2, @@ -7551,21 +7557,20 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7579,27 +7584,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of different kinds of read operations", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 47 + "y": 16 }, "hiddenSeries": false, - "id": 47, + "id": 323, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -7610,13 +7613,13 @@ "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": [], @@ -7625,33 +7628,30 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_PSMReadIOCalls{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "Page", - "refId": "A" - }, - { - "expr": "sum(rate(tiflash_system_profile_event_PSMReadPages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "exemplar": true, + "expr": "tiflash_system_current_metric_DT_BytesMemTable{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "PageFile", - "refId": "C" + "interval": "", + "intervalFactor": 1, + "legendFormat": "bytes-{{instance}}", + "refId": "I" }, { - "expr": "sum(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorRead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "exemplar": true, + "expr": "tiflash_system_current_metric_DT_BytesMemTableAllocated{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "File Descriptor", - "refId": "D" + "legendFormat": "bytes-allocated-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Disk Read OPS", + "title": "Bytes of MemTables", "tooltip": { "shared": true, "sort": 2, @@ -7667,21 +7667,20 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7695,35 +7694,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of different kinds of write operations", + "description": "The memory usage of mark cache and minmax index cache", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 54 + "y": 24 }, - "height": "", "hiddenSeries": false, - "id": 60, + "id": 238, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", + "sideWidth": 250, + "sort": "max", "sortDesc": true, "total": false, "values": true @@ -7740,44 +7735,52 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_MarkCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "hide": false, - "intervalFactor": 2, - "legendFormat": "File Descriptor", - "refId": "A", - "step": 10 + "interval": "", + "legendFormat": "mark_cache_{{instance}}", + "refId": "L" }, { - "expr": "sum(rate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "Page", - "refId": "B" + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_MinMaxIndexFiles{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "hide": false, + "interval": "", + "legendFormat": "minmax_index_cache_{{instance}}", + "refId": "A" }, { - "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "PageBackGround", - "refId": "C" + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_RNMVCCIndexCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "hide": false, + "interval": "", + "legendFormat": "rn_mvcc_index_cache_{{instance}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write flow", + "title": "Mark Cache and Minmax Index Cache Memory Usage", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -7790,7 +7793,7 @@ }, "yaxes": [ { - "format": "binBps", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -7802,8 +7805,8 @@ "label": null, "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { @@ -7817,8 +7820,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of different kinds of read operations", + "description": "cache misses or cache hits of mark_cache.\nBased on this infactor, we can check whether mark_cache is large enough", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7829,79 +7831,107 @@ "h": 8, "w": 12, "x": 12, - "y": 54 + "y": 24 }, - "height": "", "hiddenSeries": false, - "id": 59, + "id": 169, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "File Descriptor", - "refId": "A", - "step": 10 + "exemplar": true, + "expr": "max(tiflash_system_profile_event_MarkCacheMisses{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "interval": "", + "legendFormat": "mark cache misses", + "queryType": "randomWalk", + "refId": "A" }, { - "expr": "sum(rate(tiflash_system_profile_event_PSMReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "Page", + "exemplar": true, + "expr": "max(tiflash_system_profile_event_MarkCacheHits{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "mark cache hits", "refId": "B" - }, - { - "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "PageBackGround", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read flow", + "title": "Effectiveness of Mark Cache", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, + "transformations": [ + { + "id": "calculateField", + "options": { + "alias": "mark cache count total", + "binary": { + "left": "mark cache misses", + "operator": "+", + "reducer": "sum", + "right": "mark cache hits" + }, + "mode": "binary", + "reduce": { + "reducer": "sum" + } + } + }, + { + "id": "calculateField", + "options": { + "alias": "mark cache effectiveness", + "binary": { + "left": "mark cache hits", + "operator": "/", + "reducer": "sum", + "right": "mark cache count total" + }, + "mode": "binary", + "reduce": { + "reducer": "sum" + } + } + }, + { + "id": "filterFieldsByName", + "options": { + "include": { + "names": [ + "Time", + "mark cache effectiveness" + ] + } + } + } + ], "type": "graph", "xaxis": { "buckets": null, @@ -7912,20 +7942,20 @@ }, "yaxes": [ { - "format": "binBps", + "format": "percentunit", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "short", + "format": "percent", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { @@ -7936,25 +7966,24 @@ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The current processing number of segments' background management", + "description": "Information about schema of column file, to learn the memory usage of schema", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 62 + "y": 32 }, "hiddenSeries": false, - "id": 67, + "id": 168, "legend": { "alignAsTable": true, "avg": false, @@ -7963,19 +7992,20 @@ "min": false, "rightSide": true, "show": true, + "sort": "current", + "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": [], @@ -7984,36 +8014,46 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_system_current_metric_DT_DeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "delta_merge-{{instance}}", + "exemplar": true, + "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"current_size\"}) by (instance)", + "interval": "", + "legendFormat": "current_size-{{instance}}", + "queryType": "randomWalk", "refId": "A" }, { - "expr": "avg(tiflash_system_current_metric_DT_SegmentSplit{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "seg_split-{{instance}}", + "exemplar": true, + "expr": "sum(rate(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"hit_count\"}[1m])) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "hit_count_ops-{{instance}}", "refId": "B" }, { - "expr": "avg(tiflash_system_current_metric_DT_SegmentMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "seg_merge-{{instance}}", - "refId": "C" - } - ], + "exemplar": true, + "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"still_used_when_evict\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "still_used_when_evict-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"miss_count\"}[1m])) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "miss_count_ops-{{instance}}", + "refId": "D" + } + ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Current Data Management Tasks", + "title": "Schema of Column File", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -8026,16 +8066,15 @@ }, "yaxes": [ { - "decimals": 0, "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -8047,14 +8086,29 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Memory trace", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 8 + }, + "id": 25, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Errors of DeltaIndex", + "description": "The total count of different kinds of commands received", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8064,21 +8118,21 @@ "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 62 + "x": 0, + "y": 9 }, "hiddenSeries": false, - "id": 237, + "id": 41, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": false, - "max": true, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -8092,27 +8146,38 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/delete_range|ingest/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_DTDeltaIndexError{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "sum(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "DeltaIndexError-{{instance}}", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "write block", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "DeltaIndexError", + "title": "Write Command OPS", "tooltip": { "shared": true, "sort": 0, @@ -8129,7 +8194,7 @@ "yaxes": [ { "decimals": null, - "format": "cps", + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -8142,7 +8207,7 @@ "logBase": 1, "max": null, "min": "0", - "show": false + "show": true } ], "yaxis": { @@ -8153,10 +8218,10 @@ { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The information of data sharing cache hit ratio. Data sharing cache is purpose-built for OLAP workload that can reduce repeated data reads of concurrent table scanning.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8166,26 +8231,26 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 70 + "x": 12, + "y": 9 }, "hiddenSeries": false, - "id": 132, + "id": 38, "legend": { "alignAsTable": true, "avg": false, - "current": false, + "current": true, "max": false, "min": false, "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -8196,7 +8261,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/cache_hit_ratio/", + "alias": "/fs|write/", "yaxis": 2 } ], @@ -8206,43 +8271,76 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"add_cache_total_bytes_limit\"}[1m])) by (type)", + "expr": "sum by (instance) (\ntiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} +\ntiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}\n)\n/\nsum by (instance) (\ntiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}\n)", "format": "time_series", "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{type}}", + "intervalFactor": 1, + "legendFormat": "amp-total-{{instance}}", "refId": "A" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_column_cache_packs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"data_sharing_hit\"}[1m]))/sum(rate(tiflash_storage_column_cache_packs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"data_sharing_hit|data_sharing_miss\"}[1m]))", + "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m])\n)\n/\nsum by (instance) (\nrate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[5m])\n)", + "format": "time_series", "hide": false, - "instant": false, "interval": "", - "legendFormat": "data_sharing_cache_hit_ratio", + "intervalFactor": 1, + "legendFormat": "amp-5min-{{instance}}", "refId": "B" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_column_cache_packs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"extra_column_hit\"}[1m]))/sum(rate(tiflash_storage_column_cache_packs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"extra_column_hit|extra_column_miss\"}[1m]))", + "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[10m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[10m])\n)\n/\nsum by (instance) (\nrate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[10m])\n)", + "format": "time_series", "hide": true, - "instant": false, "interval": "", - "legendFormat": "extra_column_cache_hit_ratio", + "intervalFactor": 1, + "legendFormat": "amp-10min-{{instance}}", "refId": "C" + }, + { + "exemplar": true, + "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30m])\n)\n/\nsum by (instance) (\nrate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[30m])\n)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "amp-30min-{{instance}}", + "refId": "D" + }, + { + "exemplar": true, + "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m])\n)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "fs-5min-{{instance}}", + "refId": "E" + }, + { + "exemplar": true, + "expr": "sum by (instance) (\nrate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[5m])\n)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "write-5min-{{instance}}", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Data Sharing", + "title": "Write Amplification", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, + "transformations": [], "type": "graph", "xaxis": { "buckets": null, @@ -8254,19 +8352,19 @@ "yaxes": [ { "decimals": null, - "format": "ops", + "format": "short", "label": null, "logBase": 1, - "max": null, + "max": "20", "min": "0", "show": true }, { - "format": "percentunit", + "format": "binBps", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -8281,7 +8379,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The information of read thread scheduling.", + "description": "Total number of storage engine read tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8290,22 +8388,22 @@ "fillGradient": 0, "gridPos": { "h": 8, - "w": 12, - "x": 12, - "y": 70 + "w": 24, + "x": 0, + "y": 17 }, "hiddenSeries": false, - "id": 269, + "id": 40, "legend": { "alignAsTable": true, - "avg": false, - "current": false, - "max": false, + "avg": true, + "current": true, + "max": true, "min": false, "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -8319,24 +8417,17 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/push_block/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ru_exhausted|sche_active_segment_limit|sche_from_cache|sche_new_task|sche_no_pool|sche_no_ru|sche_no_segment|sche_no_slot|push_block_bytes\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_storage_read_tasks_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", - "hide": false, - "interval": "", + "instant": false, "intervalFactor": 2, - "legendFormat": "{{type}}", + "legendFormat": "{{instance}}", "refId": "A" } ], @@ -8344,7 +8435,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Thread Scheduling", + "title": "Read Tasks OPS", "tooltip": { "shared": true, "sort": 0, @@ -8362,18 +8453,18 @@ { "decimals": null, "format": "ops", - "label": null, + "label": "", "logBase": 1, "max": null, "min": "0", "show": true }, { - "format": "binBps", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -8388,118 +8479,1670 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "cache misses or cache hits of mark_cache.\nBased on this infactor, we can check whether mark_cache is large enough", + "decimals": 1, + "description": "The throughput of (maybe foreground) tasks of storage in bytes", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 78 + "y": 25 }, + "height": "", "hiddenSeries": false, - "id": 169, + "id": 258, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, "min": false, + "rightSide": true, "show": true, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "max(tiflash_system_profile_event_MarkCacheMisses{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "interval": "", - "legendFormat": "mark cache misses", - "queryType": "randomWalk", - "refId": "A" - }, - { - "exemplar": true, - "expr": "max(tiflash_system_profile_event_MarkCacheHits{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_storage_subtask_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", "hide": false, "interval": "", - "legendFormat": "mark cache hits", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Effectiveness of Mark Cache", + "title": "SubTasks Write Throughput (bytes)", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, - "transformations": [ - { - "id": "calculateField", - "options": { - "alias": "mark cache count total", - "binary": { - "left": "mark cache misses", - "operator": "+", - "reducer": "sum", - "right": "mark cache hits" - }, - "mode": "binary", - "reduce": { - "reducer": "sum" - } - } + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The throughput of (maybe foreground) tasks of storage in rows", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 25 + }, + "height": "", + "hiddenSeries": false, + "id": 259, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 250, + "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": 5, + "points": false, + "renderer": "flot", + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_subtask_throughput_rows{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "SubTasks Write Throughput (rows)", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Total number of storage's internal sub tasks", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 5, + "w": 12, + "x": 0, + "y": 33 + }, + "hiddenSeries": false, + "id": 39, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "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": true, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Small Internal Tasks OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 1, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "opm", + "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": "Duration of storage's internal sub tasks", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 5, + "w": 12, + "x": 12, + "y": 33 + }, + "hiddenSeries": false, + "id": 42, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": null, + "sortDesc": null, + "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": false, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval]))) by (le,type) / 1000000000)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "max-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Small Internal Tasks Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 1, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "s", + "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": "Total number of storage's internal sub tasks", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 5, + "w": 12, + "x": 0, + "y": 38 + }, + "hiddenSeries": false, + "id": 130, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "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": true, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Large Internal Tasks OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 1, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "opm", + "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": "Duration of storage's internal sub tasks", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 5, + "w": 12, + "x": 12, + "y": 38 + }, + "hiddenSeries": false, + "id": 131, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": null, + "sortDesc": null, + "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": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval]))) by (le,type) / 1000000000)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "max-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Large Internal Tasks Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 1, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "s", + "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": "The number of currently opened file descriptors.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 43 + }, + "hiddenSeries": false, + "id": 50, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 250, + "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": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "tiflash_proxy_process_open_fds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tiflash\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_OpenFileForWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "W-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_OpenFileForRead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "R-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_OpenFileForReadWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "RW-{{instance}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Opened File Count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of open file descriptors action.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 43 + }, + "hiddenSeries": false, + "id": 22, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 250, + "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": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_FileOpen{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Open-{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_FileOpenFailed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "OpenFail-{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "File Open OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "OPS and duration of fsync operations.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 43 + }, + "hiddenSeries": false, + "id": 52, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 250, + "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": [ + { + "alias": "/max-fsync/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_FileFSync{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "ops-fsync-{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_system_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"fsync\"}[$__rate_interval]))) by (le, instance) / 1000000000)", + "hide": false, + "interval": "", + "legendFormat": "max-fsync-{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "FSync Status", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "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": "The number of different kinds of read operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 50 + }, + "hiddenSeries": false, + "id": 46, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": 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": [ + { + "expr": "sum(rate(tiflash_system_profile_event_PSMWriteIOCalls{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "Page", + "refId": "A" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_PSMWritePages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "PageFile", + "refId": "C" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "File Descriptor", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Disk Write OPS", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "none", + "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": "The number of different kinds of read operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 50 + }, + "hiddenSeries": false, + "id": 47, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": 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": [ + { + "expr": "sum(rate(tiflash_system_profile_event_PSMReadIOCalls{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "Page", + "refId": "A" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_PSMReadPages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "PageFile", + "refId": "C" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorRead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "File Descriptor", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Disk Read OPS", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "none", + "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}", + "decimals": 1, + "description": "The flow of different kinds of write operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 57 + }, + "height": "", + "hiddenSeries": false, + "id": 60, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatedByRow": true, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "File Descriptor", + "refId": "A", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Page", + "refId": "B" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "PageBackGround", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write flow", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The flow of different kinds of read operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 57 + }, + "height": "", + "hiddenSeries": false, + "id": 59, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatedByRow": true, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "File Descriptor", + "refId": "A", + "step": 10 }, { - "id": "calculateField", - "options": { - "alias": "mark cache effectiveness", - "binary": { - "left": "mark cache hits", - "operator": "/", - "reducer": "sum", - "right": "mark cache count total" - }, - "mode": "binary", - "reduce": { - "reducer": "sum" - } - } + "expr": "sum(rate(tiflash_system_profile_event_PSMReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Page", + "refId": "B" }, { - "id": "filterFieldsByName", - "options": { - "include": { - "names": [ - "Time", - "mark cache effectiveness" - ] - } - } + "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "PageBackGround", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Read flow", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The current processing number of segments' background management", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 65 + }, + "hiddenSeries": false, + "id": 67, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": 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": [ + { + "expr": "avg(tiflash_system_current_metric_DT_DeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "delta_merge-{{instance}}", + "refId": "A" + }, + { + "expr": "avg(tiflash_system_current_metric_DT_SegmentSplit{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "seg_split-{{instance}}", + "refId": "B" + }, + { + "expr": "avg(tiflash_system_current_metric_DT_SegmentMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "seg_merge-{{instance}}", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Current Data Management Tasks", + "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": "0", + "show": true + }, + { + "format": "none", + "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": "Errors of DeltaIndex", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 65 + }, + "hiddenSeries": false, + "id": 237, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": true, + "min": false, + "rightSide": true, + "show": 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_system_profile_event_DTDeltaIndexError{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "DeltaIndexError-{{instance}}", + "refId": "A" } ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "DeltaIndexError", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, "type": "graph", "xaxis": { "buckets": null, @@ -8510,19 +10153,20 @@ }, "yaxes": [ { - "format": "percentunit", + "decimals": null, + "format": "cps", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "percent", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": false } ], @@ -8537,47 +10181,47 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", + "description": "The information of data sharing cache hit ratio. Data sharing cache is purpose-built for OLAP workload that can reduce repeated data reads of concurrent table scanning.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 78 + "x": 0, + "y": 73 }, "hiddenSeries": false, - "id": 88, + "id": 132, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/max_snapshot_lifetime/", + "alias": "/cache_hit_ratio/", "yaxis": 2 } ], @@ -8586,114 +10230,39 @@ "steppedLine": false, "targets": [ { - "expr": "tiflash_system_current_metric_DT_SegmentReadTasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "read_tasks-{{instance}}", - "refId": "I" - }, - { - "expr": "tiflash_system_current_metric_PSMVCCSnapshotsList{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"add_cache_total_bytes_limit\"}[1m])) by (type)", "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "snapshot_list-{{instance}}", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" }, { - "expr": "tiflash_system_current_metric_PSMVCCNumSnapshots{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "heatmap", - "hide": true, - "intervalFactor": 1, - "legendFormat": "num_snapshot-{{instance}}", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_column_cache_packs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"data_sharing_hit\"}[1m]))/sum(rate(tiflash_storage_column_cache_packs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"data_sharing_hit|data_sharing_miss\"}[1m]))", + "hide": false, + "instant": false, + "interval": "", + "legendFormat": "data_sharing_cache_hit_ratio", "refId": "B" }, { - "expr": "tiflash_system_current_metric_DT_SnapshotOfRead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_column_cache_packs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"extra_column_hit\"}[1m]))/sum(rate(tiflash_storage_column_cache_packs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"extra_column_hit|extra_column_miss\"}[1m]))", "hide": true, - "intervalFactor": 1, - "legendFormat": "read-{{instance}}", + "instant": false, + "interval": "", + "legendFormat": "extra_column_cache_hit_ratio", "refId": "C" - }, - { - "expr": "tiflash_system_current_metric_DT_SnapshotOfReadRaw{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "read_raw-{{instance}}", - "refId": "D" - }, - { - "expr": "tiflash_system_current_metric_DT_SnapshotOfDeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "delta_merge-{{instance}}", - "refId": "E" - }, - { - "expr": "tiflash_system_current_metric_DT_SnapshotOfDeltaCompact{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "delta_compact-{{instance}}", - "refId": "J" - }, - { - "expr": "tiflash_system_current_metric_DT_SnapshotOfSegmentMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "seg_merge-{{instance}}", - "refId": "F" - }, - { - "expr": "tiflash_system_current_metric_DT_SnapshotOfSegmentSplit{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "seg_split-{{instance}}", - "refId": "G" - }, - { - "expr": "tiflash_system_current_metric_DT_SnapshotOfPlaceIndex{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "place_index-{{instance}}", - "refId": "H" - }, - { - "expr": "tiflash_system_asynchronous_metric_MaxDTDeltaOldestSnapshotLifetime{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "max_snapshot_lifetime-{{instance}}", - "refId": "K" - }, - { - "expr": "tiflash_system_asynchronous_metric_MaxDTStableOldestSnapshotLifetime{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "max_snapshot_lifetime_stable-{{instance}}", - "refId": "L" - }, - { - "expr": "tiflash_system_asynchronous_metric_MaxDTMetaOldestSnapshotLifetime{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "max_snapshot_lifetime_meta-{{instance}}", - "refId": "M" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Snapshots", + "title": "Data Sharing", "tooltip": { "shared": true, "sort": 0, @@ -8709,7 +10278,8 @@ }, "yaxes": [ { - "format": "short", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -8717,7 +10287,7 @@ "show": true }, { - "format": "s", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -8736,7 +10306,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The memory usage of mark cache and minmax index cache", + "description": "The information of read thread scheduling.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8746,29 +10316,26 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 86 + "x": 12, + "y": 73 }, "hiddenSeries": false, - "id": 238, + "id": 269, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -8779,11 +10346,8 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/limit/", - "color": "#F2495C", - "hideTooltip": true, - "legend": false, - "linewidth": 2 + "alias": "/push_block/", + "yaxis": 2 } ], "spaceLength": 10, @@ -8792,34 +10356,20 @@ "targets": [ { "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_MarkCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "hide": false, - "interval": "", - "legendFormat": "mark_cache_{{instance}}", - "refId": "L" - }, - { - "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_MinMaxIndexFiles{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ru_exhausted|sche_active_segment_limit|sche_from_cache|sche_new_task|sche_no_pool|sche_no_ru|sche_no_segment|sche_no_slot|push_block_bytes\"}[1m])) by (type)", + "format": "time_series", "hide": false, "interval": "", - "legendFormat": "minmax_index_cache_{{instance}}", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_RNMVCCIndexCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "hide": false, - "interval": "", - "legendFormat": "rn_mvcc_index_cache_{{instance}}", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Mark Cache and Minmax Index Cache Memory Usage", + "title": "Read Thread Scheduling", "tooltip": { "shared": true, "sort": 0, @@ -8835,7 +10385,8 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -8843,12 +10394,12 @@ "show": true }, { - "format": "short", + "format": "binBps", "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true } ], "yaxis": { @@ -8862,7 +10413,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8870,32 +10420,34 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 86 + "x": 0, + "y": 81 }, "hiddenSeries": false, - "id": 289, + "id": 292, "legend": { - "avg": false, + "alignAsTable": true, + "avg": true, "current": false, "max": false, "min": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -8905,32 +10457,49 @@ "targets": [ { "exemplar": true, - "expr": "tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"alloc_.*\"}", + "expr": "histogram_quantile(0.999, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "999-{{type}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" }, { "exemplar": true, - "expr": "-tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"dealloc_.*\"}", + "expr": "histogram_quantile(0.80, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", - "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "refId": "A" + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory by thread", + "title": "Read Thread Internal Duration", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -8943,11 +10512,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "µs", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -8956,7 +10525,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -8970,7 +10539,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Information about schema of column file, to learn the memory usage of schema", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8981,21 +10550,24 @@ "h": 8, "w": 12, "x": 12, - "y": 93 + "y": 81 }, "hiddenSeries": false, - "id": 168, + "id": 88, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "alertThreshold": true @@ -9005,49 +10577,129 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/max_snapshot_lifetime/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"current_size\"}) by (instance)", + "expr": "tiflash_system_current_metric_DT_SegmentReadTasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", "interval": "", - "legendFormat": "current_size", - "queryType": "randomWalk", - "refId": "A" + "intervalFactor": 1, + "legendFormat": "read_tasks-{{instance}}", + "refId": "I" }, { "exemplar": true, - "expr": "sum(rate(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"hit_count\"}[1m])) by (instance)", - "hide": false, + "expr": "tiflash_system_current_metric_PSMVCCSnapshotsList{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, "interval": "", - "legendFormat": "hit_count_ops", + "intervalFactor": 1, + "legendFormat": "snapshot_list-{{instance}}", + "refId": "A" + }, + { + "expr": "tiflash_system_current_metric_PSMVCCNumSnapshots{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "heatmap", + "hide": true, + "intervalFactor": 1, + "legendFormat": "num_snapshot-{{instance}}", "refId": "B" }, { - "exemplar": true, - "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"still_used_when_evict\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "still_used_when_evict", + "expr": "tiflash_system_current_metric_DT_SnapshotOfRead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "read-{{instance}}", "refId": "C" }, { - "exemplar": true, - "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"miss_count\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "miss_count", + "expr": "tiflash_system_current_metric_DT_SnapshotOfReadRaw{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "read_raw-{{instance}}", "refId": "D" + }, + { + "expr": "tiflash_system_current_metric_DT_SnapshotOfDeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "delta_merge-{{instance}}", + "refId": "E" + }, + { + "expr": "tiflash_system_current_metric_DT_SnapshotOfDeltaCompact{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "delta_compact-{{instance}}", + "refId": "J" + }, + { + "expr": "tiflash_system_current_metric_DT_SnapshotOfSegmentMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "seg_merge-{{instance}}", + "refId": "F" + }, + { + "expr": "tiflash_system_current_metric_DT_SnapshotOfSegmentSplit{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "seg_split-{{instance}}", + "refId": "G" + }, + { + "expr": "tiflash_system_current_metric_DT_SnapshotOfPlaceIndex{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "place_index-{{instance}}", + "refId": "H" + }, + { + "expr": "tiflash_system_asynchronous_metric_MaxDTDeltaOldestSnapshotLifetime{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "max_snapshot_lifetime-{{instance}}", + "refId": "K" + }, + { + "expr": "tiflash_system_asynchronous_metric_MaxDTStableOldestSnapshotLifetime{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "max_snapshot_lifetime_stable-{{instance}}", + "refId": "L" + }, + { + "expr": "tiflash_system_asynchronous_metric_MaxDTMetaOldestSnapshotLifetime{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "max_snapshot_lifetime_meta-{{instance}}", + "refId": "M" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema of Column File", + "title": "Read Snapshots", "tooltip": { "shared": true, "sort": 0, @@ -9067,15 +10719,15 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -9098,10 +10750,10 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 94 + "y": 89 }, "hiddenSeries": false, "id": 291, @@ -9188,7 +10840,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The compression ratio of different compression algorithm", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9198,24 +10850,23 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 101 + "x": 12, + "y": 89 }, "hiddenSeries": false, - "id": 294, + "id": 289, "legend": { - "alignAsTable": true, - "avg": true, - "current": true, + "avg": false, + "current": false, "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "alertThreshold": true @@ -9232,29 +10883,33 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_pack_compression_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"lz4_uncompressed_bytes\"}[1m]))/sum(rate(tiflash_storage_pack_compression_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"lz4_compressed_bytes\"}[1m]))", + "expr": "tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"alloc_.*\"}", + "format": "time_series", + "hide": false, "interval": "", - "legendFormat": "lz4", - "queryType": "randomWalk", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "B" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_pack_compression_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"lightweight_uncompressed_bytes\"}[1m]))/sum(rate(tiflash_storage_pack_compression_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"lightweight_compressed_bytes\"}[1m]))", + "expr": "-tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"dealloc_.*\"}", + "format": "time_series", "hide": false, "interval": "", - "legendFormat": "lightweight", - "refId": "B" + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compression Ratio", + "title": "Memory by thread", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -9267,7 +10922,7 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -9294,6 +10949,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The compression ratio of different compression algorithm", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9301,17 +10957,17 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 101 + "x": 0, + "y": 97 }, "hiddenSeries": false, - "id": 292, + "id": 294, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": true, - "current": false, + "current": true, "max": false, "min": false, "rightSide": true, @@ -9321,14 +10977,13 @@ }, "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": [], @@ -9338,46 +10993,26 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.999, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", + "expr": "sum(rate(tiflash_storage_pack_compression_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"lz4_uncompressed_bytes\"}[1m]))/sum(rate(tiflash_storage_pack_compression_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"lz4_compressed_bytes\"}[1m]))", "interval": "", - "intervalFactor": 1, - "legendFormat": "999-{{type}}", + "legendFormat": "lz4", + "queryType": "randomWalk", "refId": "A" }, { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", + "expr": "sum(rate(tiflash_storage_pack_compression_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"lightweight_uncompressed_bytes\"}[1m]))/sum(rate(tiflash_storage_pack_compression_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"lightweight_compressed_bytes\"}[1m]))", + "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "99-{{type}}", + "legendFormat": "lightweight", "refId": "B" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "95-{{type}}", - "refId": "C" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.80, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Thread Internal Duration", + "title": "Compression Ratio", "tooltip": { "shared": true, "sort": 0, @@ -9393,11 +11028,11 @@ }, "yaxes": [ { - "format": "µs", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -9406,7 +11041,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -9428,10 +11063,10 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 0, - "y": 109 + "x": 12, + "y": 97 }, "hiddenSeries": false, "id": 293, @@ -9529,12 +11164,12 @@ "h": 8, "w": 12, "x": 0, - "y": 117 + "y": 104 }, "hiddenSeries": false, "id": 301, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": true, "current": false, "max": false, @@ -9608,7 +11243,7 @@ "title": "VersionChain", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -9654,7 +11289,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 9 }, "id": 303, "panels": [ @@ -9675,7 +11310,7 @@ "h": 8, "w": 12, "x": 0, - "y": 7 + "y": 10 }, "hiddenSeries": false, "id": 84, @@ -9777,7 +11412,7 @@ "h": 8, "w": 12, "x": 12, - "y": 7 + "y": 10 }, "hiddenSeries": false, "id": 305, @@ -9881,7 +11516,7 @@ "h": 8, "w": 12, "x": 0, - "y": 15 + "y": 18 }, "hiddenSeries": false, "id": 266, @@ -9988,7 +11623,7 @@ "h": 8, "w": 12, "x": 12, - "y": 15 + "y": 18 }, "hiddenSeries": false, "id": 86, @@ -10136,7 +11771,7 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 10 }, "id": 64, "panels": [ @@ -10157,7 +11792,7 @@ "h": 8, "w": 24, "x": 0, - "y": 103 + "y": 10 }, "hiddenSeries": false, "id": 62, @@ -10276,7 +11911,7 @@ "h": 8, "w": 12, "x": 0, - "y": 111 + "y": 18 }, "height": "", "hiddenSeries": false, @@ -10395,7 +12030,7 @@ "h": 8, "w": 12, "x": 12, - "y": 111 + "y": 18 }, "height": "", "hiddenSeries": false, @@ -10512,7 +12147,7 @@ "h": 9, "w": 24, "x": 0, - "y": 119 + "y": 26 }, "height": "", "hiddenSeries": false, @@ -10634,7 +12269,7 @@ "h": 9, "w": 24, "x": 0, - "y": 128 + "y": 35 }, "hiddenSeries": false, "id": 90, @@ -10741,7 +12376,7 @@ "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 11 }, "id": 119, "panels": [ @@ -10766,7 +12401,7 @@ "h": 8, "w": 12, "x": 0, - "y": 8 + "y": 11 }, "hiddenSeries": false, "id": 128, @@ -10909,7 +12544,7 @@ "h": 8, "w": 12, "x": 12, - "y": 8 + "y": 11 }, "hiddenSeries": false, "id": 129, @@ -11026,7 +12661,7 @@ "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 19 }, "heatmap": {}, "hideZeroBuckets": true, @@ -11088,7 +12723,7 @@ "h": 8, "w": 12, "x": 12, - "y": 16 + "y": 19 }, "hiddenSeries": false, "id": 158, @@ -11224,7 +12859,7 @@ "h": 8, "w": 12, "x": 0, - "y": 24 + "y": 27 }, "hiddenSeries": false, "id": 163, @@ -11329,7 +12964,7 @@ "h": 8, "w": 12, "x": 12, - "y": 24 + "y": 27 }, "hiddenSeries": false, "id": 162, @@ -11449,7 +13084,7 @@ "h": 8, "w": 12, "x": 0, - "y": 32 + "y": 35 }, "hiddenSeries": false, "id": 164, @@ -11554,36 +13189,33 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The number of tables running under different mode in DeltaTree", - "editable": true, - "error": false, + "description": "The num of pending writers in PageStorage", "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 32 + "y": 35 }, "hiddenSeries": false, - "id": 123, + "id": 231, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideZero": true, - "max": false, + "hideEmpty": false, + "hideZero": false, + "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", + "sideWidth": 250, + "sort": "max", "sortDesc": true, "total": false, "values": true @@ -11591,7 +13223,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -11607,46 +13239,21 @@ "targets": [ { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolV2Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(tiflash_system_current_metric_PSPendingWriterNum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-V2", - "refId": "A", - "step": 10 - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolV3Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-V3", - "refId": "B" - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolMixMode{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-MixMode", - "refId": "C" - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolUniPS{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-UniPS", - "refId": "D" + "intervalFactor": 1, + "legendFormat": "size-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Number of Tables", + "title": "PageStorage Pending Writers Num", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -11661,7 +13268,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -11674,7 +13281,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -11700,7 +13307,7 @@ "h": 8, "w": 12, "x": 0, - "y": 40 + "y": 43 }, "height": "", "hiddenSeries": false, @@ -11798,33 +13405,36 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The num of pending writers in PageStorage", + "decimals": 1, + "description": "The number of tables running under different mode in DeltaTree", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 40 + "y": 43 }, "hiddenSeries": false, - "id": 231, + "id": 123, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, - "max": true, + "hideZero": true, + "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", + "sideWidth": null, + "sort": "current", "sortDesc": true, "total": false, "values": true @@ -11832,7 +13442,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -11848,23 +13458,49 @@ "targets": [ { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_PSPendingWriterNum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(tiflash_system_current_metric_StoragePoolV2Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 2, + "legendFormat": "V2-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_StoragePoolV3Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "size-{{instance}}", - "refId": "A" + "legendFormat": "V3-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_StoragePoolMixMode{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": true, + "interval": "", + "legendFormat": "Mix-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_StoragePoolUniPS{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "UniPS-{{instance}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PageStorage Pending Writers Num", + "title": "Number of Tables", "tooltip": { + "msResolution": false, "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -11877,11 +13513,11 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -11890,7 +13526,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -11915,7 +13551,7 @@ "h": 9, "w": 24, "x": 0, - "y": 48 + "y": 51 }, "hiddenSeries": false, "id": 232, @@ -12015,7 +13651,7 @@ "h": 1, "w": 24, "x": 0, - "y": 9 + "y": 12 }, "id": 34, "panels": [ @@ -12035,7 +13671,7 @@ "h": 7, "w": 12, "x": 0, - "y": 9 + "y": 12 }, "hiddenSeries": false, "id": 167, @@ -12135,7 +13771,7 @@ "h": 7, "w": 12, "x": 12, - "y": 9 + "y": 12 }, "hiddenSeries": false, "id": 35, @@ -12233,7 +13869,7 @@ "h": 7, "w": 12, "x": 0, - "y": 16 + "y": 19 }, "hiddenSeries": false, "id": 270, @@ -12333,7 +13969,7 @@ "h": 7, "w": 12, "x": 12, - "y": 16 + "y": 19 }, "hiddenSeries": false, "id": 271, @@ -12433,7 +14069,7 @@ "h": 7, "w": 12, "x": 0, - "y": 23 + "y": 26 }, "hiddenSeries": false, "id": 37, @@ -12567,7 +14203,7 @@ "h": 7, "w": 12, "x": 12, - "y": 23 + "y": 26 }, "hiddenSeries": false, "id": 36, @@ -12689,7 +14325,7 @@ "h": 7, "w": 24, "x": 0, - "y": 30 + "y": 33 }, "hiddenSeries": false, "id": 82, @@ -12844,7 +14480,7 @@ "h": 7, "w": 24, "x": 0, - "y": 37 + "y": 40 }, "hiddenSeries": false, "id": 242, @@ -12955,7 +14591,7 @@ "h": 7, "w": 12, "x": 0, - "y": 44 + "y": 47 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13025,7 +14661,7 @@ "h": 7, "w": 12, "x": 12, - "y": 44 + "y": 47 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13095,7 +14731,7 @@ "h": 7, "w": 12, "x": 0, - "y": 51 + "y": 54 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13167,7 +14803,7 @@ "h": 7, "w": 12, "x": 12, - "y": 51 + "y": 54 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13229,7 +14865,7 @@ "h": 7, "w": 12, "x": 0, - "y": 58 + "y": 61 }, "hiddenSeries": false, "id": 235, @@ -13329,7 +14965,7 @@ "h": 7, "w": 12, "x": 12, - "y": 58 + "y": 61 }, "hiddenSeries": false, "id": 241, @@ -13437,7 +15073,7 @@ "h": 7, "w": 12, "x": 0, - "y": 65 + "y": 68 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13518,7 +15154,7 @@ "h": 7, "w": 12, "x": 12, - "y": 65 + "y": 68 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13591,7 +15227,7 @@ "h": 7, "w": 12, "x": 0, - "y": 72 + "y": 75 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13667,7 +15303,7 @@ "h": 7, "w": 12, "x": 12, - "y": 72 + "y": 75 }, "hiddenSeries": false, "id": 249, @@ -13773,7 +15409,7 @@ "h": 7, "w": 12, "x": 0, - "y": 79 + "y": 82 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13846,7 +15482,7 @@ "h": 7, "w": 12, "x": 12, - "y": 79 + "y": 82 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13919,7 +15555,7 @@ "h": 7, "w": 12, "x": 0, - "y": 86 + "y": 89 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13992,7 +15628,7 @@ "h": 7, "w": 12, "x": 12, - "y": 86 + "y": 89 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14057,7 +15693,7 @@ "h": 7, "w": 12, "x": 0, - "y": 93 + "y": 96 }, "hiddenSeries": false, "id": 240, @@ -14161,7 +15797,7 @@ "h": 7, "w": 12, "x": 12, - "y": 93 + "y": 96 }, "hiddenSeries": false, "id": 239, @@ -14298,7 +15934,7 @@ "h": 7, "w": 24, "x": 0, - "y": 100 + "y": 103 }, "hiddenSeries": false, "id": 75, @@ -14422,7 +16058,7 @@ "h": 7, "w": 12, "x": 0, - "y": 107 + "y": 110 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14492,7 +16128,7 @@ "h": 7, "w": 12, "x": 12, - "y": 107 + "y": 110 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14555,7 +16191,7 @@ "h": 7, "w": 12, "x": 0, - "y": 114 + "y": 117 }, "hiddenSeries": false, "id": 263, @@ -14653,13 +16289,15 @@ "h": 7, "w": 12, "x": 12, - "y": 114 + "y": 117 }, "hiddenSeries": false, "id": 272, "legend": { "avg": false, "current": false, + "hideEmpty": true, + "hideZero": true, "max": false, "min": false, "show": true, @@ -14710,7 +16348,7 @@ "title": "Memory by thread", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -14768,7 +16406,7 @@ "h": 7, "w": 12, "x": 0, - "y": 121 + "y": 124 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14840,7 +16478,7 @@ "h": 7, "w": 12, "x": 12, - "y": 121 + "y": 124 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14906,7 +16544,7 @@ "h": 7, "w": 24, "x": 0, - "y": 128 + "y": 131 }, "height": "", "hiddenSeries": false, @@ -15016,7 +16654,7 @@ "h": 7, "w": 24, "x": 0, - "y": 135 + "y": 138 }, "height": "", "hiddenSeries": false, @@ -15132,7 +16770,7 @@ "h": 7, "w": 12, "x": 0, - "y": 142 + "y": 145 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15201,7 +16839,7 @@ "h": 7, "w": 12, "x": 12, - "y": 142 + "y": 145 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15271,7 +16909,7 @@ "h": 7, "w": 12, "x": 0, - "y": 149 + "y": 152 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15337,7 +16975,7 @@ "h": 7, "w": 12, "x": 12, - "y": 149 + "y": 152 }, "hiddenSeries": false, "id": 91, @@ -15465,7 +17103,7 @@ "h": 7, "w": 12, "x": 0, - "y": 156 + "y": 159 }, "hiddenSeries": false, "id": 296, @@ -15561,7 +17199,7 @@ "h": 1, "w": 24, "x": 0, - "y": 10 + "y": 13 }, "id": 95, "panels": [ @@ -15581,7 +17219,7 @@ "h": 8, "w": 12, "x": 0, - "y": 10 + "y": 13 }, "hiddenSeries": false, "id": 99, @@ -15734,7 +17372,7 @@ "h": 8, "w": 12, "x": 12, - "y": 10 + "y": 13 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15791,7 +17429,7 @@ "h": 1, "w": 24, "x": 0, - "y": 11 + "y": 14 }, "id": 274, "panels": [ @@ -15826,7 +17464,7 @@ "h": 8, "w": 12, "x": 0, - "y": 11 + "y": 14 }, "hiddenSeries": false, "id": 286, @@ -15947,7 +17585,7 @@ "h": 8, "w": 12, "x": 12, - "y": 11 + "y": 14 }, "hiddenSeries": false, "id": 288, @@ -16076,7 +17714,7 @@ "h": 8, "w": 12, "x": 0, - "y": 19 + "y": 22 }, "hiddenSeries": false, "id": 282, @@ -16203,7 +17841,7 @@ "h": 8, "w": 12, "x": 12, - "y": 19 + "y": 22 }, "hiddenSeries": false, "id": 284, @@ -16305,7 +17943,7 @@ "h": 1, "w": 24, "x": 0, - "y": 12 + "y": 15 }, "id": 171, "panels": [ @@ -16325,7 +17963,7 @@ "h": 8, "w": 24, "x": 0, - "y": 11 + "y": 15 }, "hiddenSeries": false, "id": 173, @@ -16426,7 +18064,7 @@ "h": 8, "w": 12, "x": 0, - "y": 19 + "y": 23 }, "hiddenSeries": false, "id": 187, @@ -16554,7 +18192,7 @@ "h": 8, "w": 12, "x": 12, - "y": 19 + "y": 23 }, "height": "", "hiddenSeries": false, @@ -16673,7 +18311,7 @@ "h": 8, "w": 12, "x": 0, - "y": 27 + "y": 31 }, "height": "", "hiddenSeries": false, @@ -16783,7 +18421,7 @@ "h": 8, "w": 12, "x": 12, - "y": 27 + "y": 31 }, "height": "", "hiddenSeries": false, @@ -16896,7 +18534,7 @@ "h": 8, "w": 12, "x": 0, - "y": 35 + "y": 39 }, "hiddenSeries": false, "id": 176, @@ -17004,7 +18642,7 @@ "h": 8, "w": 12, "x": 12, - "y": 35 + "y": 39 }, "hiddenSeries": false, "id": 175, @@ -17131,7 +18769,7 @@ "h": 8, "w": 12, "x": 0, - "y": 43 + "y": 47 }, "hiddenSeries": false, "id": 189, @@ -17233,7 +18871,7 @@ "h": 8, "w": 12, "x": 12, - "y": 43 + "y": 47 }, "hiddenSeries": false, "id": 191, @@ -17333,7 +18971,7 @@ "h": 8, "w": 12, "x": 0, - "y": 51 + "y": 55 }, "hiddenSeries": false, "id": 193, @@ -17459,7 +19097,7 @@ "h": 8, "w": 12, "x": 12, - "y": 51 + "y": 55 }, "hiddenSeries": false, "id": 195, @@ -17570,7 +19208,7 @@ "h": 8, "w": 12, "x": 0, - "y": 59 + "y": 63 }, "hiddenSeries": false, "id": 201, @@ -17706,7 +19344,7 @@ "h": 8, "w": 12, "x": 12, - "y": 59 + "y": 63 }, "hiddenSeries": false, "id": 233, @@ -17830,7 +19468,7 @@ "h": 8, "w": 12, "x": 0, - "y": 67 + "y": 71 }, "hiddenSeries": false, "id": 236, @@ -17945,7 +19583,7 @@ "h": 8, "w": 12, "x": 12, - "y": 67 + "y": 71 }, "hiddenSeries": false, "id": 252, @@ -18052,7 +19690,7 @@ "h": 8, "w": 12, "x": 0, - "y": 75 + "y": 79 }, "hiddenSeries": false, "id": 251, @@ -18159,7 +19797,7 @@ "h": 8, "w": 12, "x": 12, - "y": 75 + "y": 79 }, "hiddenSeries": false, "id": 253, @@ -18266,7 +19904,7 @@ "h": 8, "w": 12, "x": 0, - "y": 83 + "y": 87 }, "hiddenSeries": false, "id": 254, @@ -18367,7 +20005,7 @@ "h": 1, "w": 24, "x": 0, - "y": 13 + "y": 16 }, "id": 172, "panels": [ @@ -18388,7 +20026,7 @@ "h": 8, "w": 12, "x": 0, - "y": 12 + "y": 16 }, "hiddenSeries": false, "id": 178, @@ -18506,7 +20144,7 @@ "h": 8, "w": 12, "x": 12, - "y": 12 + "y": 16 }, "hiddenSeries": false, "id": 179, @@ -18680,7 +20318,7 @@ "h": 8, "w": 12, "x": 0, - "y": 20 + "y": 24 }, "hiddenSeries": false, "id": 182, @@ -18798,7 +20436,7 @@ "h": 8, "w": 12, "x": 12, - "y": 20 + "y": 24 }, "hiddenSeries": false, "id": 180, @@ -18925,7 +20563,7 @@ "h": 8, "w": 12, "x": 0, - "y": 28 + "y": 32 }, "hiddenSeries": false, "id": 185, @@ -19052,7 +20690,7 @@ "h": 8, "w": 12, "x": 12, - "y": 28 + "y": 32 }, "hiddenSeries": false, "id": 186, @@ -19154,7 +20792,7 @@ "h": 8, "w": 12, "x": 0, - "y": 36 + "y": 40 }, "hiddenSeries": false, "id": 188, @@ -19273,7 +20911,7 @@ "h": 1, "w": 24, "x": 0, - "y": 14 + "y": 17 }, "id": 202, "panels": [ @@ -19294,7 +20932,7 @@ "h": 8, "w": 12, "x": 0, - "y": 13 + "y": 17 }, "hiddenSeries": false, "id": 224, @@ -19394,7 +21032,7 @@ "h": 8, "w": 12, "x": 12, - "y": 13 + "y": 17 }, "hiddenSeries": false, "id": 226, @@ -19502,7 +21140,7 @@ "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 25 }, "hiddenSeries": false, "id": 228, @@ -19605,7 +21243,7 @@ "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 25 }, "hiddenSeries": false, "id": 222, @@ -19778,7 +21416,7 @@ "h": 8, "w": 12, "x": 0, - "y": 29 + "y": 33 }, "hiddenSeries": false, "id": 230, @@ -19933,7 +21571,7 @@ "h": 8, "w": 12, "x": 12, - "y": 29 + "y": 33 }, "hiddenSeries": false, "id": 218, @@ -20063,7 +21701,7 @@ "h": 8, "w": 12, "x": 0, - "y": 37 + "y": 41 }, "hiddenSeries": false, "id": 220, @@ -20193,7 +21831,7 @@ "h": 8, "w": 12, "x": 12, - "y": 37 + "y": 41 }, "hiddenSeries": false, "id": 216, @@ -20319,7 +21957,7 @@ "h": 8, "w": 12, "x": 0, - "y": 109 + "y": 49 }, "hiddenSeries": false, "id": 300, @@ -20421,7 +22059,7 @@ "h": 1, "w": 24, "x": 0, - "y": 15 + "y": 18 }, "id": 248, "panels": [ @@ -20442,7 +22080,7 @@ "h": 8, "w": 12, "x": 0, - "y": 15 + "y": 18 }, "hiddenSeries": false, "id": 246, From 8de8ae187023554403e4642321da3c40c05c7c59 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Mon, 7 Jul 2025 22:46:39 +0800 Subject: [PATCH 10/10] Output rs_operator to scan_details Signed-off-by: JaySon-Huang --- dbms/src/Storages/DeltaMerge/Filter/In.h | 14 ++++++++++ dbms/src/Storages/DeltaMerge/Filter/IsNull.h | 8 ++++++ .../DeltaMerge/Filter/PushDownExecutor.cpp | 3 +-- .../DeltaMerge/Filter/PushDownExecutor.h | 6 ++++- .../Storages/DeltaMerge/Filter/RSOperator.h | 26 +++++++++++++++++++ .../Storages/DeltaMerge/Filter/Unsupported.h | 8 ++++++ dbms/src/Storages/StorageDeltaMerge.cpp | 11 +++++--- 7 files changed, 69 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/Filter/In.h b/dbms/src/Storages/DeltaMerge/Filter/In.h index 26824f96cf4..797623175a0 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/In.h +++ b/dbms/src/Storages/DeltaMerge/Filter/In.h @@ -49,6 +49,20 @@ class In : public RSOperator return buf.toString(); } + Poco::JSON::Object::Ptr toJSONObject() override + { + Poco::JSON::Object::Ptr obj = new Poco::JSON::Object(); + obj->set("op", name()); + obj->set("col", attr.col_name); + Poco::JSON::Array arr; + for (const auto & v : values) + { + arr.add(applyVisitor(FieldVisitorToDebugString(), v)); + } + obj->set("value", arr); + return obj; + } + RSResults roughCheck(size_t start_pack, size_t pack_count, const RSCheckParam & param) override { // If values is empty (for example where a in ()), all packs will not match. diff --git a/dbms/src/Storages/DeltaMerge/Filter/IsNull.h b/dbms/src/Storages/DeltaMerge/Filter/IsNull.h index 0c42f3560ef..4a97da5bcbc 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/IsNull.h +++ b/dbms/src/Storages/DeltaMerge/Filter/IsNull.h @@ -34,6 +34,14 @@ class IsNull : public RSOperator String toDebugString() override { return fmt::format(R"({{"op":"{}","col":"{}"}})", name(), attr.col_name); } + Poco::JSON::Object::Ptr toJSONObject() override + { + Poco::JSON::Object::Ptr obj = new Poco::JSON::Object(); + obj->set("op", name()); + obj->set("col", attr.col_name); + return obj; + } + RSResults roughCheck(size_t start_pack, size_t pack_count, const RSCheckParam & param) override { auto rs_index = getRSIndex(param, attr); diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp index 84daec58c59..f0ae394824e 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -236,7 +235,7 @@ Poco::JSON::Object::Ptr PushDownExecutor::toJSONObject() const Poco::JSON::Object::Ptr json = new Poco::JSON::Object(); if (rs_operator) { - json->set("rs_operator", rs_operator->toDebugString()); + json->set("rs_operator", rs_operator->toJSONObject()); } if (ann_query_info) { diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h index 9d631cf7927..5beeb0ab0f2 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h @@ -14,9 +14,13 @@ #pragma once +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + #include #include -#include #include #include #include diff --git a/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h b/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h index 636b43eb665..b1a98837836 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h +++ b/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h @@ -14,6 +14,11 @@ #pragma once +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + #include #include #include @@ -49,6 +54,7 @@ class RSOperator virtual String name() = 0; virtual String toDebugString() = 0; + virtual Poco::JSON::Object::Ptr toJSONObject() = 0; virtual RSResults roughCheck(size_t start_pack, size_t pack_count, const RSCheckParam & param) = 0; @@ -87,6 +93,14 @@ class ColCmpVal : public RSOperator attr.col_name, applyVisitor(FieldVisitorToDebugString(), value)); } + Poco::JSON::Object::Ptr toJSONObject() override + { + Poco::JSON::Object::Ptr obj = new Poco::JSON::Object(); + obj->set("op", name()); + obj->set("col", attr.col_name); + obj->set("value", applyVisitor(FieldVisitorToDebugString(), value)); + return obj; + } }; @@ -123,6 +137,18 @@ class LogicalOp : public RSOperator buf.append("]}"); return buf.toString(); } + Poco::JSON::Object::Ptr toJSONObject() override + { + Poco::JSON::Object::Ptr obj = new Poco::JSON::Object(); + obj->set("op", name()); + Poco::JSON::Array arr; + for (const auto & child : children) + { + arr.add(child->toJSONObject()); + } + obj->set("children", arr); + return obj; + } }; inline std::optional getRSIndex(const RSCheckParam & param, const Attr & attr) diff --git a/dbms/src/Storages/DeltaMerge/Filter/Unsupported.h b/dbms/src/Storages/DeltaMerge/Filter/Unsupported.h index 70534ca3862..2e9f647fb46 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/Unsupported.h +++ b/dbms/src/Storages/DeltaMerge/Filter/Unsupported.h @@ -34,6 +34,14 @@ class Unsupported : public RSOperator String toDebugString() override { return fmt::format(R"({{"op":"{}","reason":"{}"}})", name(), reason); } + Poco::JSON::Object::Ptr toJSONObject() override + { + Poco::JSON::Object::Ptr obj = new Poco::JSON::Object(); + obj->set("op", name()); + obj->set("reason", reason); + return obj; + } + RSResults roughCheck(size_t /*start_pack*/, size_t pack_count, const RSCheckParam & /*param*/) override { return RSResults(pack_count, RSResult::Some); diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 94b70fc5057..c3b05f7522c 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -48,6 +48,7 @@ #include #include #include +#include #include #include #include @@ -863,7 +864,7 @@ BlockInputStreams StorageDeltaMerge::read( query_info.req_id, tracing_logger); - auto filter = PushDownExecutor::build( + auto pushdown_executor = PushDownExecutor::build( query_info, columns_to_read, store->getTableColumns(), @@ -875,6 +876,7 @@ BlockInputStreams StorageDeltaMerge::read( auto runtime_filter_list = parseRuntimeFilterList(query_info, store->getTableColumns(), context, tracing_logger); const auto & scan_context = mvcc_query_info.scan_context; + scan_context->pushdown_executor = pushdown_executor; auto streams = store->read( context, @@ -883,7 +885,7 @@ BlockInputStreams StorageDeltaMerge::read( ranges, num_streams, /*start_ts=*/mvcc_query_info.start_ts, - filter, + pushdown_executor, runtime_filter_list, query_info.dag_query ? query_info.dag_query->rf_max_wait_time_ms : 0, query_info.req_id, @@ -953,7 +955,7 @@ void StorageDeltaMerge::read( query_info.req_id, tracing_logger); - auto filter = PushDownExecutor::build( + auto pushdown_executor = PushDownExecutor::build( query_info, columns_to_read, store->getTableColumns(), @@ -965,6 +967,7 @@ void StorageDeltaMerge::read( auto runtime_filter_list = parseRuntimeFilterList(query_info, store->getTableColumns(), context, tracing_logger); const auto & scan_context = mvcc_query_info.scan_context; + scan_context->pushdown_executor = pushdown_executor; store->read( exec_context_, @@ -975,7 +978,7 @@ void StorageDeltaMerge::read( ranges, num_streams, /*start_ts=*/mvcc_query_info.start_ts, - filter, + pushdown_executor, runtime_filter_list, query_info.dag_query ? query_info.dag_query->rf_max_wait_time_ms : 0, query_info.req_id,