From 399cf61711ae202d80b5df57e7b1688918a8c54f Mon Sep 17 00:00:00 2001 From: JaySon Date: Tue, 8 Jul 2025 16:09:24 +0800 Subject: [PATCH 1/6] This is an automated cherry-pick of #10275 Signed-off-by: ti-chi-bot --- dbms/src/Common/CurrentMetrics.cpp | 7 + dbms/src/Interpreters/AsynchronousMetrics.cpp | 5 + .../DeltaMerge/ColumnFile/ColumnFile.h | 22 +- .../ColumnFile/ColumnFileInMemory.cpp | 20 +- .../ColumnFile/ColumnFileInMemory.h | 25 +- .../DeltaMerge/Delta/DeltaValueSpace.cpp | 7 + .../DeltaMerge/Delta/DeltaValueSpace.h | 3 +- .../Storages/DeltaMerge/Delta/MemTableSet.cpp | 145 +- .../Storages/DeltaMerge/Delta/MemTableSet.h | 92 +- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 99 +- .../DeltaMerge/DeltaMergeStore_Statistics.cpp | 45 +- .../DeltaMerge/DeltaMergeStore_Statistics.h | 123 + dbms/src/Storages/DeltaMerge/Filter/In.h | 14 + dbms/src/Storages/DeltaMerge/Filter/IsNull.h | 8 + .../DeltaMerge/Filter/PushDownFilter.cpp | 15 + .../DeltaMerge/Filter/PushDownFilter.h | 17 +- .../Storages/DeltaMerge/Filter/RSOperator.h | 26 + .../Storages/DeltaMerge/Filter/Unsupported.h | 8 + .../Storages/DeltaMerge/Remote/Serializer.cpp | 2 +- dbms/src/Storages/DeltaMerge/ScanContext.cpp | 6 + dbms/src/Storages/DeltaMerge/ScanContext.h | 44 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 4 +- dbms/src/Storages/DeltaMerge/Segment.h | 4 + .../DeltaMerge/StoragePool/StoragePool.cpp | 4 +- .../DeltaMerge/tests/gtest_segment.cpp | 1 + dbms/src/Storages/IStorage.cpp | 12 + dbms/src/Storages/IStorage.h | 6 + dbms/src/Storages/ITableDeclaration.h | 2 +- dbms/src/Storages/StorageDeltaMerge.cpp | 37 +- dbms/src/Storages/StorageDeltaMerge.h | 18 + .../System/StorageSystemDTSegments.cpp | 3 + .../Storages/System/StorageSystemDTTables.cpp | 4 + dbms/src/TestUtils/gtests_dbms_main.cpp | 7 +- dbms/src/TiDB/Schema/SchemaSyncService.cpp | 8 + libs/libcommon/include/common/logger_util.h | 36 + libs/libdaemon/src/BaseDaemon.cpp | 18 +- metrics/grafana/tiflash_summary.json | 6631 ++++++++++++----- 37 files changed, 5559 insertions(+), 1969 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h create mode 100644 libs/libcommon/include/common/logger_util.h diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index d2786a934c8..3a188efb888 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -64,6 +64,13 @@ M(DT_SnapshotOfPlaceIndex) \ 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) \ diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index dd8df736a72..223e53691d1 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 diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h index 69ee51e20c8..8c4f5ebd5e4 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h @@ -74,19 +74,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: @@ -95,6 +82,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; @@ -138,7 +126,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 74bd79b986e..63e3e82799c 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.cpp @@ -42,6 +42,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); @@ -65,7 +66,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, @@ -73,28 +80,31 @@ 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) { 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; - 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 a719bc7b155..9a8bd2f93e5 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h @@ -31,6 +31,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; @@ -66,6 +80,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; } @@ -81,9 +96,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/DeltaValueSpace.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp index 3c306c32888..ba571e386df 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp @@ -275,8 +275,15 @@ size_t DeltaValueSpace::getTotalCacheBytes() const return mem_table_set->getBytes() + persisted_file_set->getTotalCacheBytes(); } +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() + persisted_file_set->getValidCacheRows(); } diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h index 3432b747701..13bbec9c5af 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -84,7 +84,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; @@ -219,6 +219,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 0d1f4a5f578..ddeb0bbec2f 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp @@ -22,10 +22,85 @@ #include #include +<<<<<<< HEAD namespace DB { namespace DM { +======= +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) + , column_files(in_memory_files) + , log(Logger::get()) +{ + 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) + { + new_rows += file->getRows(); + new_bytes += file->getBytes(); + new_alloc_bytes += file->getAllocateBytes(); + new_deletes += file->getDeletes(); + } + stat.resetTo(column_files.size(), new_rows, new_bytes, new_alloc_bytes, new_deletes); +} + +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) void MemTableSet::appendColumnFileInner(const ColumnFilePtr & column_file) { if (!column_files.empty()) @@ -38,11 +113,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(); - 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( @@ -182,31 +258,37 @@ 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; + 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) { - 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, // 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; + + stat.append( // + limit, + append_bytes, + append_res.new_alloc_bytes, + /*deletes_added*/ 0, + /*files_added*/ 0); } void MemTableSet::appendDeleteRange(const RowKeyRange & delete_range) @@ -273,13 +355,22 @@ 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()); + stat.deletes.load()); +<<<<<<< HEAD return snap; +======= + return std::make_shared( + data_provider, + std::move(column_files_snap), + stat.rows, + stat.bytes, + stat.deletes); +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( @@ -312,7 +403,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, @@ -320,9 +411,15 @@ ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( "Files: {}", flush_task->getFlushRows(), flush_task->getFlushDeletes(), +<<<<<<< HEAD rows.load(), deletes.load(), columnFilesToString(column_files)); +======= + stat.rows.load(), + stat.deletes.load(), + ColumnFile::filesToString(column_files)); +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) throw Exception("Rows and deletes check failed.", ErrorCodes::LOGICAL_ERROR); } @@ -345,6 +442,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) { @@ -352,13 +450,16 @@ 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; - 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 2b91f5855c9..bef950dd628 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; @@ -34,19 +32,60 @@ 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 public: #endif + // Keep track of the number of mem-table in memory. + CurrentMetrics::Increment holder_counter; + // Note that we must update `column_files_count` for outer thread-safe after `column_files` changed ColumnFiles column_files; - // 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 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; @@ -54,18 +93,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. @@ -75,20 +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 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 /** @@ -153,5 +174,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 a7b065d0584..8b6b221ac2f 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -77,104 +78,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 cf816358006..4f2d9492bc1 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() @@ -33,11 +32,13 @@ 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); stat.segment_count = segments.size(); + stat.column_count = original_table_columns.size(); for (const auto & [handle, segment] : segments) { @@ -66,6 +67,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(); } @@ -89,6 +91,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; @@ -155,8 +158,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(); @@ -166,16 +167,25 @@ 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(); + { + // 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_index_size = delta->getDeltaIndexBytes(); stat.stable_page_id = stable->getId(); @@ -294,5 +304,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..9dbbf7d702f --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h @@ -0,0 +1,123 @@ +// 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 column_count = 0; + 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; + UInt64 delta_cache_alloc_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/Filter/In.h b/dbms/src/Storages/DeltaMerge/Filter/In.h index 01443a51078..b78af878f8a 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 39c814aa1ae..5ebf2ff1be5 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/PushDownFilter.cpp b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.cpp index b2f6fc3ca7c..1727deb57ff 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.cpp @@ -198,4 +198,19 @@ PushDownFilterPtr PushDownFilter::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->toJSONObject()); + } + 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/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h index eb680b6c45e..69fc9db27e8 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.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 @@ -35,14 +40,20 @@ class PushDownFilter public: PushDownFilter( const RSOperatorPtr & rs_operator_, +<<<<<<< HEAD:dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h const ExpressionActionsPtr & beofre_where_, +======= + const ANNQueryInfoPtr & ann_query_info_, + const FTSQueryInfoPtr & fts_query_info_, + const ExpressionActionsPtr & before_where_, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)):dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h const ExpressionActionsPtr & project_after_where_, const ColumnDefinesPtr & filter_columns_, const String filter_column_name_, const ExpressionActionsPtr & extra_cast_, const ColumnDefinesPtr & columns_after_cast_) : 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_) @@ -54,6 +65,8 @@ class PushDownFilter : rs_operator(rs_operator_) {} + Poco::JSON::Object::Ptr toJSONObject() const; + // Use by StorageDisaggregated. static PushDownFilterPtr build( const DM::RSOperatorPtr & rs_operator, @@ -78,7 +91,7 @@ class PushDownFilter 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/Filter/RSOperator.h b/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h index 82832c0a9da..8498523b481 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 @@ -50,6 +55,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; @@ -85,6 +91,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; + } }; @@ -121,6 +135,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 6eb68e5703b..ae46aa2f1fa 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/DeltaMerge/Remote/Serializer.cpp b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp index f260c54f012..97edab433bb 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp @@ -333,7 +333,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/ScanContext.cpp b/dbms/src/Storages/DeltaMerge/ScanContext.cpp index 64191ac89f9..a87a9cdc7ad 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 @@ -183,6 +184,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 5084296d34a..01ece99966a 100644 --- a/dbms/src/Storages/DeltaMerge/ScanContext.h +++ b/dbms/src/Storages/DeltaMerge/ScanContext.h @@ -28,6 +28,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`. @@ -67,7 +69,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 @@ -98,7 +99,48 @@ class ScanContext std::atomic total_vector_idx_read_vec_time_ms{0}; std::atomic total_vector_idx_read_others_time_ms{0}; +<<<<<<< HEAD +======= + std::atomic inverted_idx_load_from_s3{0}; + std::atomic inverted_idx_load_from_disk{0}; + std::atomic inverted_idx_load_from_cache{0}; + std::atomic inverted_idx_load_time_ms{0}; + std::atomic inverted_idx_search_time_ms{0}; + std::atomic inverted_idx_search_skipped_packs{0}; + std::atomic inverted_idx_indexed_rows{0}; + std::atomic inverted_idx_search_selected_rows{0}; + + std::atomic fts_n_from_inmemory_noindex{0}; + std::atomic fts_n_from_tiny_index{0}; + std::atomic fts_n_from_tiny_noindex{0}; + std::atomic fts_n_from_dmf_index{0}; + std::atomic fts_n_from_dmf_noindex{0}; + std::atomic fts_rows_from_inmemory_noindex{0}; + std::atomic fts_rows_from_tiny_index{0}; + std::atomic fts_rows_from_tiny_noindex{0}; + std::atomic fts_rows_from_dmf_index{0}; + std::atomic fts_rows_from_dmf_noindex{0}; + std::atomic fts_idx_load_total_ms{0}; + std::atomic fts_idx_load_from_cache{0}; + std::atomic fts_idx_load_from_column_file{0}; + std::atomic fts_idx_load_from_stable_s3{0}; + std::atomic fts_idx_load_from_stable_disk{0}; + std::atomic fts_idx_search_n{0}; + std::atomic fts_idx_search_total_ms{0}; + std::atomic fts_idx_dm_search_rows{0}; + std::atomic fts_idx_dm_total_read_fts_ms{0}; + std::atomic fts_idx_dm_total_read_others_ms{0}; + std::atomic fts_idx_tiny_search_rows{0}; + std::atomic fts_idx_tiny_total_read_fts_ms{0}; + std::atomic fts_idx_tiny_total_read_others_ms{0}; + std::atomic fts_brute_total_read_ms{0}; + std::atomic fts_brute_total_search_ms{0}; + + const KeyspaceID keyspace_id; + ReadMode read_mode = ReadMode::Normal; // note: share struct padding with keyspace_id +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) const String resource_group_name; + PushDownExecutorPtr pushdown_executor; explicit ScanContext(const String & name = "") : resource_group_name(name) diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 5d5c281555c..5491c897852 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -116,6 +116,7 @@ extern const Metric DT_SnapshotOfDeltaMerge; extern const Metric DT_SnapshotOfPlaceIndex; extern const Metric DT_SnapshotOfSegmentIngest; extern const Metric DT_SnapshotOfBitmapFilter; +extern const Metric DT_NumSegment; } // namespace CurrentMetrics namespace DB @@ -283,7 +284,8 @@ Segment::Segment( // PageIdU64 next_segment_id_, const DeltaValueSpacePtr & delta_, const StableValueSpacePtr & stable_) - : epoch(epoch_) + : holder_counter(CurrentMetrics::DT_NumSegment) + , 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 bfe0419a132..ee802a99b17 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -800,6 +800,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; diff --git a/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp b/dbms/src/Storages/DeltaMerge/StoragePool/StoragePool.cpp index 83f1b115932..16e8d925ef8 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 @@ -893,7 +893,7 @@ PageIdU64 StoragePool::newDataPageIdForDTFile(StableDiskDelegator & delegator, c // 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={}", + "The DTFile is already exists, continue to acquire another ID. call={} path={} file_id={}", who, existed_path, dtfile_id); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp index b66f3f2a3aa..639f6f4badc 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp @@ -608,6 +608,7 @@ try } CATCH + class SegmentEnableLogicalSplitTest : public SegmentOperationTest { protected: 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 bc75af28843..c1954ee5b14 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -65,6 +65,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; @@ -349,6 +353,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 2b27fea4b02..8fc482c0bbd 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -47,6 +47,7 @@ #include #include #include +#include #include #include #include @@ -61,6 +62,10 @@ #include #include +namespace CurrentMetrics +{ +extern const Metric DT_NumStorageDeltaMerge; +} // namespace CurrentMetrics namespace DB { @@ -87,7 +92,11 @@ StorageDeltaMerge::StorageDeltaMerge( Timestamp tombstone, Context & global_context_) : IManageableStorage{columns_, tombstone} +<<<<<<< HEAD , data_path_contains_database_name(db_engine != "TiFlash") +======= + , holder_counter(CurrentMetrics::DT_NumStorageDeltaMerge, 1) +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) , store_inited(false) , max_column_id_used(0) , global_context(global_context_.getGlobalContext()) @@ -835,11 +844,23 @@ BlockInputStreams StorageDeltaMerge::read( query_info.req_id, tracing_logger); +<<<<<<< HEAD auto filter = PushDownFilter::build(query_info, columns_to_read, store->getTableColumns(), context, tracing_logger); +======= + auto pushdown_executor = PushDownExecutor::build( + query_info, + columns_to_read, + store->getTableColumns(), + query_info.dag_query ? query_info.dag_query->used_indexes + : google::protobuf::RepeatedPtrField{}, + context, + tracing_logger); +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) 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, @@ -848,7 +869,7 @@ BlockInputStreams StorageDeltaMerge::read( ranges, num_streams, /*start_ts=*/mvcc_query_info.start_ts, - filter, + pushdown_executor, runtime_filter_list, query_info.dag_query == nullptr ? 0 : query_info.dag_query->rf_max_wait_time_ms, query_info.req_id, @@ -918,11 +939,23 @@ void StorageDeltaMerge::read( query_info.req_id, tracing_logger); +<<<<<<< HEAD auto filter = PushDownFilter::build(query_info, columns_to_read, store->getTableColumns(), context, tracing_logger); +======= + auto pushdown_executor = PushDownExecutor::build( + query_info, + columns_to_read, + store->getTableColumns(), + query_info.dag_query ? query_info.dag_query->used_indexes + : google::protobuf::RepeatedPtrField{}, + context, + tracing_logger); +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) 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_, @@ -933,7 +966,7 @@ void StorageDeltaMerge::read( ranges, num_streams, /*start_ts=*/mvcc_query_info.start_ts, - filter, + pushdown_executor, runtime_filter_list, query_info.dag_query == nullptr ? 0 : query_info.dag_query->rf_max_wait_time_ms, query_info.req_id, diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 889274032a7..f483525fb3b 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -275,6 +275,9 @@ class StorageDeltaMerge }; const bool data_path_contains_database_name = false; + // 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. @@ -308,9 +311,24 @@ class StorageDeltaMerge // Used to allocate new column-id when this table is NOT synced from TiDB ColumnID max_column_id_used; +<<<<<<< HEAD std::atomic shutdown_called{false}; std::atomic next_version = 1; //TODO: remove this!!! +======= + // 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; + + // `decoding_schema_changed` and `decoding_schema_epoch` must be used under the protection of table structure lock + bool decoding_schema_changed = false; + + const bool data_path_contains_database_name = false; +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) Context & global_context; diff --git a/dbms/src/Storages/System/StorageSystemDTSegments.cpp b/dbms/src/Storages/System/StorageSystemDTSegments.cpp index 0502501a891..c5d859c01e9 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 @@ -60,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()}, @@ -152,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 33c8813128d..f84fe6dc197 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()}, @@ -55,6 +56,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()}, @@ -167,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); @@ -178,6 +181,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); diff --git a/dbms/src/TestUtils/gtests_dbms_main.cpp b/dbms/src/TestUtils/gtests_dbms_main.cpp index fc70d15fb77..f25552c9712 100644 --- a/dbms/src/TestUtils/gtests_dbms_main.cpp +++ b/dbms/src/TestUtils/gtests_dbms_main.cpp @@ -23,7 +23,11 @@ #include #include #include +<<<<<<< HEAD #include +======= +#include +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) #include #include @@ -66,7 +70,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("trace", 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/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; 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 6d60bd3ad8a..77bb5ef6838 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -77,6 +77,7 @@ #include #include #include +#include #include #include #include @@ -668,21 +669,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. @@ -757,7 +743,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()) { diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index b042ca665b2..aafe0a2daf3 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, @@ -4793,66 +4840,50 @@ "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", + "description": "The MPP query count in TiFlash", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 4 + "y": 46 }, "hiddenSeries": false, - "id": 107, + "id": 157, "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, @@ -4861,10 +4892,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": "max(tiflash_mpp_task_manager{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance, type)", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4872,7 +4904,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Min TSO", + "title": "MPP Query count", "tooltip": { "shared": true, "sort": 0, @@ -4888,13 +4920,12 @@ }, "yaxes": [ { - "decimals": null, "format": "none", - "label": "TSO", + "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true }, { "format": "short", @@ -4916,29 +4947,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": 46 }, "hiddenSeries": false, - "id": 109, + "id": 103, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -4947,13 +4974,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": [], @@ -4963,54 +4991,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": "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": "", - "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=\"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": "Max Threads", "tooltip": { "shared": true, "sort": 0, @@ -5026,12 +5019,11 @@ }, "yaxes": [ { - "decimals": null, "format": "none", - "label": "Threads", - "logBase": 10, + "label": null, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5054,29 +5046,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": 53 }, "hiddenSeries": false, - "id": 111, + "id": 199, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5085,13 +5073,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": [], @@ -5101,27 +5090,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": "tiflash_mpp_task_monitor{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}", "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": "Time of the Longest Live MPP Task", "tooltip": { "shared": true, "sort": 0, @@ -5137,12 +5118,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "Queries", + "format": "s", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5165,29 +5145,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the count of active/ waiting tasks", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 12 + "y": 53 }, "hiddenSeries": false, - "id": 113, + "id": 166, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5196,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": [], @@ -5212,27 +5189,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": "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=\"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": "Data size in send and receive queue", "tooltip": { "shared": true, "sort": 0, @@ -5248,12 +5217,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "Tasks", + "format": "bytes", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5269,6 +5237,8 @@ "align": false, "alignLevel": null } +<<<<<<< HEAD +======= }, { "aliasColors": {}, @@ -5276,29 +5246,25 @@ "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": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 20 + "y": 60 }, "hiddenSeries": false, - "id": 117, + "id": 297, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5307,13 +5273,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": [], @@ -5323,19 +5290,19 @@ "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": "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}}-{{resource_group}}", - "queryType": "randomWalk", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Hard Limit Exceeded Count", + "title": "Network Transmission", "tooltip": { "shared": true, "sort": 0, @@ -5351,12 +5318,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "", + "format": "bytes", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5379,8 +5345,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the time of waiting for schedule", + "description": "The establish calldata details", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5388,18 +5353,18 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 20 + "y": 60 }, "hiddenSeries": false, - "id": 115, + "id": 299, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5408,13 +5373,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": [], @@ -5424,37 +5390,19 @@ "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_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}}-{{resource_group}}-80", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", "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, - "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": "Task Waiting Duration", + "title": "Establish calldata details", "tooltip": { "shared": true, "sort": 0, @@ -5470,12 +5418,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "s", - "label": "Time", + "format": "none", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5491,10 +5438,11 @@ "align": false, "alignLevel": null } +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "repeat": null, - "title": "Task Scheduler", + "title": "Coprocessor", "type": "row" }, { @@ -5506,7 +5454,7 @@ "x": 0, "y": 4 }, - "id": 16, + "id": 105, "panels": [ { "aliasColors": {}, @@ -5514,43 +5462,43 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Executed DDL jobs per minute", + "decimals": null, + "description": "the min_tso of each instance", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, "y": 5 }, "hiddenSeries": false, - "id": 19, + "id": 107, "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, + "lines": false, "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { - "alertThreshold": true + "alertThreshold": false }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, - "points": false, + "pointradius": 1, + "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -5558,41 +5506,19 @@ "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=\"min_tso\"}) by (instance, resource_group)", + "interval": "", + "legendFormat": "{{instance}}-{{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", - "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": "Min TSO", "tooltip": { "shared": true, "sort": 0, @@ -5609,15 +5535,15 @@ "yaxes": [ { "decimals": null, - "format": "opm", - "label": null, + "format": "none", + "label": "TSO", "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5636,42 +5562,44 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Executed DDL apply jobs per minute", + "decimals": null, + "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, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, "y": 5 }, "hiddenSeries": false, - "id": 18, + "id": 109, "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": [], @@ -5680,18 +5608,55 @@ "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=\"thread_soft_limit\"}) 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=\"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": "Schema Apply OPM", + "title": "Estimated Thread Usage and Limit", "tooltip": { "shared": true, "sort": 0, @@ -5708,15 +5673,15 @@ "yaxes": [ { "decimals": null, - "format": "opm", - "label": null, - "logBase": 1, + "format": "none", + "label": "Threads", + "logBase": 10, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5735,97 +5700,74 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": null, + "description": "the count of active/ waiting queries", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 12 + "y": 13 }, "hiddenSeries": false, - "id": 20, + "id": 111, "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}}", + "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": "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=\"active_queries_count\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{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": "Active and Waiting Queries Count", "tooltip": { "shared": true, "sort": 0, @@ -5841,19 +5783,20 @@ }, "yaxes": [ { - "format": "s", - "label": null, + "decimals": null, + "format": "none", + "label": "Queries", "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 } ], @@ -5861,32 +5804,19 @@ "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 count of active/ waiting tasks", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, @@ -5894,58 +5824,53 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 6 + "x": 12, + "y": 13 }, "hiddenSeries": false, - "id": 41, + "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": [ - { - "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": "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" }, { - "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", + "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, - "intervalFactor": 1, - "legendFormat": "write block", + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "B" } ], @@ -5953,7 +5878,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Command OPS", + "title": "Active and Waiting Tasks Count", "tooltip": { "shared": true, "sort": 0, @@ -5970,19 +5895,19 @@ "yaxes": [ { "decimals": null, - "format": "ops", - "label": null, + "format": "none", + "label": "Tasks", "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 } ], @@ -5994,12 +5919,15 @@ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "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": 0, @@ -6007,13 +5935,13 @@ "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 6 + "x": 0, + "y": 21 }, "hiddenSeries": false, - "id": 38, + "id": 117, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": false, @@ -6025,98 +5953,40 @@ }, "lines": true, "linewidth": 1, - "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "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)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "amp-total-{{instance}}", - "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)", - "format": "time_series", + "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": "", - "intervalFactor": 1, - "legendFormat": "amp-5min-{{instance}}", + "legendFormat": "{{instance}}-{{resource_group}}", + "queryType": "randomWalk", "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)", - "format": "time_series", - "hide": true, - "interval": "", - "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": "Write Amplification", + "title": "Hard Limit Exceeded Count", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, - "transformations": [], "type": "graph", "xaxis": { "buckets": null, @@ -6128,15 +5998,15 @@ "yaxes": [ { "decimals": null, - "format": "short", - "label": null, + "format": "none", + "label": "", "logBase": 1, - "max": "20", - "min": "0", + "max": null, + "min": null, "show": true }, { - "format": "binBps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -6155,7 +6025,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage engine read tasks", + "decimals": null, + "description": "the time of waiting for schedule", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6164,17 +6035,17 @@ "fillGradient": 0, "gridPos": { "h": 8, - "w": 24, - "x": 0, - "y": 14 + "w": 12, + "x": 12, + "y": 21 }, "hiddenSeries": false, - "id": 40, + "id": 115, "legend": { - "alignAsTable": true, - "avg": true, + "alignAsTable": false, + "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -6183,14 +6054,13 @@ }, "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": [], @@ -6199,19 +6069,38 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_read_tasks_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}", + "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" + }, + { + "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": "Read Tasks OPS", + "title": "Task Waiting Duration", "tooltip": { "shared": true, "sort": 0, @@ -6228,15 +6117,15 @@ "yaxes": [ { "decimals": null, - "format": "ops", - "label": "", + "format": "s", + "label": "Time", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -6248,15 +6137,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, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of (maybe foreground) tasks of storage in bytes", + "description": "Executed DDL jobs per minute", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6264,34 +6168,28 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 22 + "y": 6 }, - "height": "", "hiddenSeries": false, - "id": 258, + "id": 19, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": false, - "max": true, + "current": false, + "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 }, @@ -6300,37 +6198,50 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, - "seriesOverrides": [ - { - "alias": "/total/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "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": "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": "{{type}}", - "refId": "A", - "step": 10 + "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", + "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": "SubTasks Write Throughput (bytes)", + "title": "Schema Internal DDL OPM", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6343,7 +6254,8 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": null, + "format": "opm", "label": null, "logBase": 1, "max": null, @@ -6351,12 +6263,12 @@ "show": true }, { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -6370,8 +6282,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of (maybe foreground) tasks of storage in rows", + "description": "Executed DDL apply jobs per minute", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6379,34 +6290,28 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 22 + "y": 6 }, - "height": "", "hiddenSeries": false, - "id": 259, + "id": 18, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": false, - "max": true, + "current": false, + "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 }, @@ -6415,37 +6320,27 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, - "seriesOverrides": [ - { - "alias": "/total/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "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)", + "expr": "avg(increase(tiflash_schema_trigger_count{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 + "legendFormat": "triggle-by-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "SubTasks Write Throughput (rows)", + "title": "Schema Apply OPM", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6458,7 +6353,8 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": null, + "format": "opm", "label": null, "logBase": 1, "max": null, @@ -6466,12 +6362,12 @@ "show": true }, { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -6485,30 +6381,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 7, "w": 12, "x": 0, - "y": 30 + "y": 13 }, "hiddenSeries": false, - "id": 39, + "id": 20, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": false, + "hideZero": false, "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, - "sideWidth": null, "total": false, "values": false }, @@ -6524,27 +6419,59 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/^applying/", + "yaxis": 2 + } + ], "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": "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": 2, - "legendFormat": "{{type}}", + "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}}", + "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": "Small Internal Tasks OPS", + "title": "Schema Apply Duration", "tooltip": { "shared": true, "sort": 0, @@ -6560,8 +6487,7 @@ }, "yaxes": [ { - "decimals": 1, - "format": "ops", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -6569,40 +6495,59 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, - "max": null, + "max": "2", "min": "0", - "show": false + "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}", - "description": "Duration of storage's internal sub tasks", + "description": "TiFlash CPU usage calculated with process CPU running seconds.", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { - "h": 5, + "h": 8, "w": 12, - "x": 12, - "y": 30 + "x": 0, + "y": 7 }, "hiddenSeries": false, - "id": 42, + "id": 310, "legend": { "alignAsTable": true, "avg": false, @@ -6611,15 +6556,16 @@ "min": false, "rightSide": true, "show": true, - "sort": null, - "sortDesc": null, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -6628,28 +6574,46 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2 + } + ], "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)", + "exemplar": true, + "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": 2, - "legendFormat": "max-{{type}}", - "refId": "A" + "legendFormat": "{{instance}}", + "refId": "A", + "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": "Small Internal Tasks Duration", + "title": "CPU Usage (irate)", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -6665,7 +6629,7 @@ "yaxes": [ { "decimals": 1, - "format": "s", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -6673,11 +6637,11 @@ "show": true }, { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": false } ], @@ -6692,37 +6656,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": 35 + "x": 12, + "y": 7 }, "hiddenSeries": false, - "id": 130, + "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 }, @@ -6731,28 +6701,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": "Large Internal Tasks OPS", + "title": "Segment Reader", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -6768,7 +6759,7 @@ "yaxes": [ { "decimals": 1, - "format": "ops", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -6776,11 +6767,11 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": false } ], @@ -6795,7 +6786,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6803,13 +6793,13 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 8, "w": 12, - "x": 12, - "y": 35 + "x": 0, + "y": 15 }, "hiddenSeries": false, - "id": 131, + "id": 316, "legend": { "alignAsTable": true, "avg": false, @@ -6818,8 +6808,8 @@ "min": false, "rightSide": true, "show": true, - "sort": null, - "sortDesc": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -6842,12 +6832,11 @@ "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_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" } ], @@ -6855,10 +6844,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Large Internal Tasks Duration", + "title": "Request QPS by instance", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -6871,8 +6860,8 @@ }, "yaxes": [ { - "decimals": 1, - "format": "s", + "decimals": null, + "format": "none", "label": null, "logBase": 1, "max": null, @@ -6880,12 +6869,12 @@ "show": true }, { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": false + "min": null, + "show": true } ], "yaxis": { @@ -6899,33 +6888,35 @@ "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 flow of different kinds of read operations", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 40 + "h": 8, + "w": 12, + "x": 12, + "y": 15 }, + "height": "", "hiddenSeries": false, - "id": 50, + "id": 314, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, - "hideZero": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", + "sideWidth": null, + "sort": "current", "sortDesc": true, "total": false, "values": true @@ -6942,58 +6933,50 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "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)", + "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": 1, - "legendFormat": "W-{{instance}}", - "refId": "B" + "intervalFactor": 2, + "legendFormat": "File Descriptor-{{instance}}", + "refId": "A", + "step": 10 }, { "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_system_profile_event_PSMReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", - "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "R-{{instance}}", - "refId": "C" + "legendFormat": "Page-{{instance}}", + "refId": "B" }, { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_OpenFileForReadWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", - "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "RW-{{instance}}", - "refId": "D" + "legendFormat": "PageBackGround-{{instance}}", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Opened File Count", + "title": "Read Throughput by instance", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -7006,7 +6989,7 @@ }, "yaxes": [ { - "format": "none", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -7018,8 +7001,8 @@ "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true } ], "yaxis": { @@ -7033,7 +7016,7 @@ "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": "The total count of different kinds of commands received", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7041,33 +7024,28 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 40 + "h": 8, + "w": 12, + "x": 0, + "y": 23 }, "hiddenSeries": false, - "id": 22, + "id": 318, "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", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -7076,35 +7054,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_FileOpen{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "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", - "interval": "", + "hide": false, "intervalFactor": 1, - "legendFormat": "Open-{{instance}}", - "refId": "A" + "legendFormat": "write block-{{instance}}", + "refId": "C" }, { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_FileOpenFailed{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 (instance, type)", "format": "time_series", - "interval": "", + "hide": false, "intervalFactor": 1, - "legendFormat": "OpenFail-{{instance}}", - "refId": "B" + "legendFormat": "{{type}}-{{instance}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "File Open OPS", + "title": "Write Command OPS By Instance", "tooltip": { "shared": true, "sort": 0, @@ -7120,6 +7101,7 @@ }, "yaxes": [ { + "decimals": null, "format": "ops", "label": null, "logBase": 1, @@ -7128,12 +7110,12 @@ "show": true }, { - "format": "short", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true } ], "yaxis": { @@ -7147,7 +7129,8 @@ "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)", + "decimals": 1, + "description": "The throughput of write by instance", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7155,17 +7138,20 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 40 + "h": 8, + "w": 12, + "x": 12, + "y": 23 }, + "height": "", "hiddenSeries": false, - "id": 52, + "id": 312, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": false, "max": true, "min": false, "rightSide": true, @@ -7179,7 +7165,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -7188,9 +7174,10 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/max-fsync/", + "alias": "/total/", "yaxis": 2 } ], @@ -7200,19 +7187,21 @@ "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": "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": "ops-fsync-{{instance}}", - "refId": "A" + "legendFormat": "write-{{instance}}", + "refId": "A", + "step": 10 }, { "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)", + "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": "", - "legendFormat": "max-fsync-{{instance}}", + "legendFormat": "ingest-{{instance}}", "refId": "B" } ], @@ -7220,7 +7209,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "FSync Status", + "title": "Write Throughput By Instance", "tooltip": { "shared": true, "sort": 0, @@ -7236,8 +7225,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -7245,46 +7233,59 @@ "show": true }, { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { "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 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": 8 }, "hiddenSeries": false, - "id": 46, + "id": 325, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -7295,13 +7296,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": [], @@ -7310,33 +7311,20 @@ "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]))", + "exemplar": true, + "expr": "tiflash_system_current_metric_NumKeyspace{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "File Descriptor", - "refId": "D" + "legendFormat": "keyspace-{{instance}}", + "refId": "I" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Disk Write OPS", + "title": "Number of Keyspaces", "tooltip": { "shared": true, "sort": 2, @@ -7352,21 +7340,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": { @@ -7380,27 +7367,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": 8 }, "hiddenSeries": false, - "id": 47, + "id": 324, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -7411,13 +7396,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": [], @@ -7426,33 +7411,30 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_PSMReadIOCalls{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "exemplar": true, + "expr": "tiflash_system_current_metric_DT_NumStorageDeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "Page", - "refId": "A" + "interval": "", + "intervalFactor": 1, + "legendFormat": "tables-{{instance}}", + "refId": "I" }, { - "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_NumIStorage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "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", + "interval": "", "intervalFactor": 1, - "legendFormat": "File Descriptor", - "refId": "D" + "legendFormat": "tables-all-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Disk Read OPS", + "title": "Number of Physical Tables", "tooltip": { "shared": true, "sort": 2, @@ -7468,21 +7450,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": { @@ -7496,8 +7477,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of different kinds of write operations", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7508,24 +7488,18 @@ "h": 8, "w": 12, "x": 0, - "y": 54 + "y": 16 }, - "height": "", "hiddenSeries": false, - "id": 60, + "id": 319, "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 }, @@ -7538,44 +7512,39 @@ }, "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_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]))", + "exemplar": true, + "expr": "tiflash_system_current_metric_DT_NumSegment{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "Page", - "refId": "B" + "legendFormat": "segments-{{instance}}", + "refId": "I" }, { - "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundWriteBytes{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": "PageBackGround", - "refId": "C" + "legendFormat": "mem_table-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write flow", + "title": "Number of Segments", "tooltip": { "shared": true, "sort": 2, @@ -7591,20 +7560,20 @@ }, "yaxes": [ { - "format": "binBps", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { @@ -7618,8 +7587,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of different kinds of read operations", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7630,24 +7598,18 @@ "h": 8, "w": 12, "x": 12, - "y": 54 + "y": 16 }, - "height": "", "hiddenSeries": false, - "id": 59, + "id": 323, "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 }, @@ -7660,44 +7622,39 @@ }, "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 - }, - { - "expr": "sum(rate(tiflash_system_profile_event_PSMReadBytes{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", + "interval": "", "intervalFactor": 1, - "legendFormat": "Page", - "refId": "B" + "legendFormat": "bytes-{{instance}}", + "refId": "I" }, { - "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundReadBytes{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": "PageBackGround", - "refId": "C" + "legendFormat": "bytes-allocated-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read flow", + "title": "Bytes of MemTables", "tooltip": { "shared": true, "sort": 2, @@ -7713,20 +7670,20 @@ }, "yaxes": [ { - "format": "binBps", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { @@ -7737,11 +7694,10 @@ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The current processing number of segments' background management", + "description": "The memory usage of mark cache and minmax index cache", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7752,25 +7708,28 @@ "h": 8, "w": 12, "x": 0, - "y": 62 + "y": 24 }, "hiddenSeries": false, - "id": 67, + "id": 238, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": 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 as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -7779,39 +7738,49 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2 + } + ], "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", + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_MarkCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "hide": false, - "intervalFactor": 1, - "legendFormat": "delta_merge-{{instance}}", - "refId": "A" + "interval": "", + "legendFormat": "mark_cache_{{instance}}", + "refId": "L" }, { - "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" + "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": "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": "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": "Current Data Management Tasks", + "title": "Mark Cache and Minmax Index Cache Memory Usage", "tooltip": { "shared": true, "sort": 0, @@ -7827,8 +7796,7 @@ }, "yaxes": [ { - "decimals": 0, - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -7836,12 +7804,12 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7855,42 +7823,39 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Errors of DeltaIndex", + "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": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 62 + "y": 24 }, "hiddenSeries": false, - "id": 237, + "id": 169, "legend": { - "alignAsTable": true, "avg": false, "current": false, - "max": true, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "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": [], @@ -7900,49 +7865,99 @@ "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, + "expr": "max(tiflash_system_profile_event_MarkCacheMisses{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "interval": "", - "intervalFactor": 1, - "legendFormat": "DeltaIndexError-{{instance}}", + "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)", + "hide": false, + "interval": "", + "legendFormat": "mark cache hits", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "DeltaIndexError", + "title": "Effectiveness of Mark Cache", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": null, - "format": "cps", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, + "transformations": [ { - "format": "opm", - "label": null, - "logBase": 1, + "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, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, "max": null, - "min": "0", + "min": null, + "show": true + }, + { + "format": "percent", + "label": null, + "logBase": 1, + "max": null, + "min": null, "show": false } ], @@ -7957,7 +7972,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The storage I/O limiter metrics.", + "description": "Information about schema of column file, to learn the memory usage of schema", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7968,25 +7983,26 @@ "h": 8, "w": 12, "x": 0, - "y": 70 + "y": 32 }, "hiddenSeries": false, - "id": 84, + "id": 168, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -8001,22 +8017,46 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_io_limiter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{type}}", + "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" + }, + { + "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" + }, + { + "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": "I/O Limiter Throughput", + "title": "Schema of Column File", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -8029,12 +8069,11 @@ }, "yaxes": [ { - "decimals": 0, - "format": "binBps", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -8050,38 +8089,53 @@ "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": "The storage I/O limiter metrics.", + "description": "The total count of different kinds of commands received", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 70 + "x": 0, + "y": 9 }, "hiddenSeries": false, - "id": 266, + "id": 41, "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, @@ -8092,12 +8146,12 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/-/", + "alias": "/delete_range|ingest/", "yaxis": 2 } ], @@ -8106,21 +8160,27 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_io_limiter_pending_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "instant": false, - "interval": "", + "hide": false, "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": "I/O Limiter Pending Rate and Duration", + "title": "Write Command OPS", "tooltip": { "shared": true, "sort": 0, @@ -8136,7 +8196,7 @@ }, "yaxes": [ { - "decimals": 0, + "decimals": null, "format": "ops", "label": null, "logBase": 1, @@ -8145,11 +8205,11 @@ "show": true }, { - "format": "s", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -8161,10 +8221,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": [] @@ -8174,26 +8234,26 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 78 + "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 }, @@ -8204,7 +8264,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "cache_hit_ratio", + "alias": "/fs|write/", "yaxis": 2 } ], @@ -8214,50 +8274,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_stale|add_cache_succ|add_cache_total_bytes_limit|add_cache_reach_count_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_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_hit|get_cache_copy\"}[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, "interval": "", - "legendFormat": "get_cache_hit", - "refId": "C" + "intervalFactor": 1, + "legendFormat": "amp-5min-{{instance}}", + "refId": "B" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_miss|get_cache_hit|get_cache_part|get_cache_copy\"}[1m]))", - "hide": false, + "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, "interval": "", - "legendFormat": "get_cache_total", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "amp-10min-{{instance}}", + "refId": "C" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_hit|get_cache_copy\"}[1m]))/sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_miss|get_cache_hit|get_cache_part|get_cache_copy\"}[1m]))", - "hide": false, - "instant": false, + "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": "", - "legendFormat": "cache_hit_ratio", + "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, @@ -8269,19 +8355,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 } ], @@ -8296,7 +8382,2830 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "I/O Limiter current pending count.", + "description": "Total number of storage engine read tasks", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 24, + "x": 0, + "y": 17 + }, + "hiddenSeries": false, + "id": 40, + "legend": { + "alignAsTable": true, + "avg": true, + "current": 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_storage_read_tasks_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Read Tasks OPS", + "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": "", + "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 throughput of (maybe foreground) tasks of storage in bytes", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 25 + }, + "height": "", + "hiddenSeries": false, + "id": 258, + "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_bytes{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 (bytes)", + "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}", + "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 + }, + { + "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" + }, + { + "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, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "cps", + "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": "The storage I/O limiter metrics.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 70 + }, + "hiddenSeries": false, + "id": 84, + "legend": { + "alignAsTable": true, + "avg": false, + "current": 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": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tiflash_storage_io_limiter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "I/O Limiter Throughput", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 0, + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The storage I/O limiter metrics.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 70 + }, + "hiddenSeries": false, + "id": 266, + "legend": { + "alignAsTable": true, + "avg": false, + "current": 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": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/-/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_io_limiter_pending_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "I/O Limiter Pending Rate and Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 0, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "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": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 78 +======= + "y": 73 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 132, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "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": [ + { + "alias": "cache_hit_ratio", + "yaxis": 2 + } + ], + "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=~\"add_cache_stale|add_cache_succ|add_cache_total_bytes_limit|add_cache_reach_count_limit\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_hit|get_cache_copy\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "get_cache_hit", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_miss|get_cache_hit|get_cache_part|get_cache_copy\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "get_cache_total", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_hit|get_cache_copy\"}[1m]))/sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_miss|get_cache_hit|get_cache_part|get_cache_copy\"}[1m]))", + "hide": false, + "instant": false, + "interval": "", + "legendFormat": "cache_hit_ratio", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Data Sharing", + "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": "percentunit", + "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}", + "description": "I/O Limiter current pending count.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 78 + }, + "hiddenSeries": false, + "id": 86, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/pending/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "avg(tiflash_system_current_metric_RateLimiterPendingWriteRequest{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "other-current-{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "bgwrite-current-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "fgwrite-current-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "bgread-current-{{instance}}", + "refId": "D" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "fgread-current-{{instance}}", + "refId": "E" + }, + { + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "hide": false, + "interval": "", + "legendFormat": "{{type}}-pending-max", + "refId": "F" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": false, + "interval": "", + "legendFormat": "{{type}}-pending-P99", + "refId": "G" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "I/O Limiter Current Pending Count", + "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": "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": "cache misses or cache hits of mark_cache.\nBased on this infactor, we can check whether mark_cache is large enough", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 86 + }, + "hiddenSeries": false, + "id": 169, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "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)", + "hide": false, + "interval": "", + "legendFormat": "mark cache hits", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Effectiveness of Mark Cache", + "tooltip": { + "shared": true, + "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, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "percent", + "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 information of read thread scheduling.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 86 +======= + "y": 73 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 269, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "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": [ + { + "alias": "/push_block/", + "yaxis": 2 + } + ], + "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)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Read Thread Scheduling", + "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": "binBps", + "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}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, +<<<<<<< HEAD + "x": 12, + "y": 117 +======= + "x": 0, + "y": 81 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 292, + "legend": { +<<<<<<< HEAD + "alignAsTable": false, +======= + "alignAsTable": true, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "avg": true, + "current": false, + "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": [ + { + "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", + "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": 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", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "µs", +<<<<<<< HEAD + "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 memory usage of mark cache and minmax index cache", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 94 + }, + "hiddenSeries": false, + "id": 238, + "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", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "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\"}", + "hide": false, + "interval": "", + "legendFormat": "minmax_index_cache_{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_RNDeltaIndexCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "hide": false, + "interval": "", + "legendFormat": "rn_delta_index_cache_{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Mark Cache and Minmax Index Cache Memory Usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, +======= + "label": null, + "logBase": 1, + "max": null, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "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": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8307,20 +11216,24 @@ "h": 8, "w": 12, "x": 12, - "y": 78 +<<<<<<< HEAD + "y": 94 +======= + "y": 81 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "hiddenSeries": false, - "id": 86, + "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, @@ -8336,7 +11249,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/pending/", + "alias": "/max_snapshot_lifetime/", "yaxis": 2 } ], @@ -8345,68 +11258,217 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_system_current_metric_RateLimiterPendingWriteRequest{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "exemplar": true, + "expr": "tiflash_system_current_metric_DT_SegmentReadTasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", - "hide": true, "interval": "", "intervalFactor": 1, - "legendFormat": "other-current-{{instance}}", - "refId": "A" + "legendFormat": "read_tasks-{{instance}}", + "refId": "I" }, { "exemplar": true, - "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) 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": "bgwrite-current-{{instance}}", + "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": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "fgwrite-current-{{instance}}", + "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": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "bgread-current-{{instance}}", + "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" }, { - "exemplar": true, - "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "fgread-current-{{instance}}", + "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" }, { - "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", - "hide": false, - "interval": "", - "legendFormat": "{{type}}-pending-max", + "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", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "s", + "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}", +<<<<<<< HEAD + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 102 + }, + "hiddenSeries": false, + "id": 291, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": false, + "expr": "tiflash_memory_usage_by_class{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", "interval": "", - "legendFormat": "{{type}}-pending-P99", - "refId": "G" + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Current Pending Count", + "title": "Memory by class", "tooltip": { "shared": true, "sort": 0, @@ -8422,16 +11484,15 @@ }, "yaxes": [ { - "decimals": 0, - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -8450,7 +11511,9 @@ "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", +======= +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8458,13 +11521,14 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, +<<<<<<< HEAD + "h": 7, "w": 12, - "x": 0, - "y": 86 + "x": 12, + "y": 102 }, "hiddenSeries": false, - "id": 169, + "id": 289, "legend": { "avg": false, "current": false, @@ -8476,6 +11540,7 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "alertThreshold": true @@ -8492,73 +11557,34 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_system_profile_event_MarkCacheMisses{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"alloc_.*\"}", + "format": "time_series", "interval": "", - "legendFormat": "mark cache misses", - "queryType": "randomWalk", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "B" }, { "exemplar": true, - "expr": "max(tiflash_system_profile_event_MarkCacheHits{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "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": "mark cache hits", - "refId": "B" + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Effectiveness of Mark Cache", + "title": "Memory by thread", "tooltip": { "shared": true, "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, @@ -8569,7 +11595,7 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -8577,12 +11603,12 @@ "show": true }, { - "format": "percent", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -8596,70 +11622,73 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The information of read thread scheduling.", + "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": 12, - "y": 86 + "y": 109 }, "hiddenSeries": false, - "id": 269, + "id": 168, +======= + "h": 8, + "w": 12, + "x": 0, + "y": 89 + }, + "hiddenSeries": false, + "id": 291, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "legend": { - "alignAsTable": true, "avg": false, "current": false, "max": false, "min": false, - "rightSide": true, "show": true, "total": false, "values": false }, "lines": true, "linewidth": 1, +<<<<<<< HEAD +======= "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": "/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": "tiflash_memory_usage_by_class{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", - "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A" + "legendFormat": "{{instance}}-{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Thread Scheduling", + "title": "Memory by class", "tooltip": { "shared": true, "sort": 0, @@ -8675,20 +11704,19 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "binBps", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -8703,6 +11731,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8713,31 +11742,29 @@ "h": 8, "w": 12, "x": 12, - "y": 117 + "y": 89 }, "hiddenSeries": false, - "id": 292, + "id": 289, "legend": { - "alignAsTable": false, - "avg": 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 as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -8747,49 +11774,33 @@ "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", - "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))", + "expr": "tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"alloc_.*\"}", "format": "time_series", + "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "99-{{type}}", + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{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": 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))", + "expr": "-tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"dealloc_.*\"}", "format": "time_series", + "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Thread Internal Duration", + "title": "Memory by thread", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -8802,11 +11813,11 @@ }, "yaxes": [ { - "format": "µs", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -8815,7 +11826,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -8829,90 +11840,105 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The memory usage of mark cache and minmax index cache", + "description": "The compression ratio of different compression algorithm", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 94 + "y": 97 }, "hiddenSeries": false, - "id": 238, + "id": 294, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/limit/", - "color": "#F2495C", - "hideTooltip": true, - "legend": false, - "linewidth": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_MarkCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "hide": false, +<<<<<<< HEAD + "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"current_size\"}) by (instance)", "interval": "", - "legendFormat": "mark_cache_{{instance}}", - "refId": "L" + "legendFormat": "current_size", +======= + "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": "", + "legendFormat": "lz4", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "queryType": "randomWalk", + "refId": "A" }, { "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_MinMaxIndexFiles{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", +<<<<<<< HEAD + "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": "minmax_index_cache_{{instance}}", - "refId": "A" + "legendFormat": "hit_count_ops", + "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", + "refId": "C" }, { "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_RNDeltaIndexCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "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": "rn_delta_index_cache_{{instance}}", + "legendFormat": "miss_count", + "refId": "D" +======= + "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": "", + "legendFormat": "lightweight", "refId": "B" +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Mark Cache and Minmax Index Cache Memory Usage", +<<<<<<< HEAD + "title": "Schema of Column File", +======= + "title": "Compression Ratio", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "tooltip": { "shared": true, "sort": 0, @@ -8928,11 +11954,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -8941,7 +11967,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -8955,7 +11981,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", + "description": "The count of the compression algorithm used by each data part", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8963,27 +11989,31 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, +<<<<<<< HEAD + "x": 0, + "y": 117 +======= "x": 12, - "y": 94 + "y": 97 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "hiddenSeries": false, - "id": 88, + "id": 293, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, - "total": false, + "total": true, "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { "alertThreshold": true @@ -8993,125 +12023,26 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/max_snapshot_lifetime/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "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\"}", - "format": "time_series", - "hide": true, - "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" - }, - { - "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" - }, - { - "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", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_pack_compression_algorithm_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "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" + "interval": "", + "legendFormat": "{{type}}", + "queryType": "randomWalk", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Snapshots", + "title": "Compression Algorithm Count", "tooltip": { "shared": true, "sort": 0, @@ -9131,15 +12062,15 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -9154,7 +12085,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", + "description": "The compression ratio of different compression algorithm", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9162,25 +12093,30 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 102 +<<<<<<< HEAD + "y": 109 +======= + "y": 104 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "hiddenSeries": false, - "id": 291, + "id": 294, "legend": { - "avg": false, - "current": false, + "alignAsTable": true, + "avg": true, + "current": true, "max": false, "min": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { "alertThreshold": true @@ -9197,11 +12133,18 @@ "targets": [ { "exemplar": true, - "expr": "tiflash_memory_usage_by_class{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "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": 2, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "lz4", + "queryType": "randomWalk", + "refId": "A" + }, + { + "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]))", + "hide": false, + "interval": "", + "legendFormat": "lightweight", "refId": "B" } ], @@ -9209,10 +12152,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory by class", + "title": "Compression Ratio", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -9225,7 +12168,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -9245,14 +12188,32 @@ "align": false, "alignLevel": null } - }, + } + ], + "repeat": null, + "title": "Storage", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 9 + }, +<<<<<<< HEAD +======= + "id": 303, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", + "description": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9260,26 +12221,30 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 102 + "x": 0, + "y": 10 }, "hiddenSeries": false, - "id": 289, + "id": 84, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -9294,22 +12259,11 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"alloc_.*\"}", - "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "refId": "B" - }, - { - "exemplar": true, - "expr": "-tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"dealloc_.*\"}", + "expr": "sum(rate(tiflash_storage_io_limiter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "hide": false, - "interval": "", + "instant": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{type}}", "refId": "A" } ], @@ -9317,10 +12271,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory by thread", + "title": "I/O Limiter Throughput", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -9333,11 +12287,12 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": 0, + "format": "binBps", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -9360,7 +12315,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Information about schema of column file, to learn the memory usage of schema", + "description": "Current limit bytes per second of Storage I/O limiter", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9371,22 +12326,27 @@ "h": 8, "w": 12, "x": 12, - "y": 109 + "y": 10 }, "hiddenSeries": false, - "id": 168, + "id": 305, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": true, "show": true, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -9397,47 +12357,25 @@ "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": false, + "stack": true, "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": "sum(tiflash_storage_io_limiter_curr{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", + "format": "time_series", + "instant": false, "interval": "", - "legendFormat": "current_size", - "queryType": "randomWalk", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" - }, - { - "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", - "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", - "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", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema of Column File", + "title": "I/O Limiter Threshold", "tooltip": { "shared": true, "sort": 0, @@ -9453,11 +12391,12 @@ }, "yaxes": [ { - "format": "short", + "decimals": 0, + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -9480,7 +12419,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The count of the compression algorithm used by each data part", + "description": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9491,24 +12430,25 @@ "h": 8, "w": 12, "x": 0, - "y": 117 + "y": 18 }, "hiddenSeries": false, - "id": 293, + "id": 266, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, - "total": true, + "total": false, "values": true }, "lines": true, "linewidth": 1, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -9517,18 +12457,24 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/-/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_pack_compression_algorithm_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, + "expr": "sum(rate(tiflash_storage_io_limiter_pending_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "instant": false, "interval": "", + "intervalFactor": 2, "legendFormat": "{{type}}", - "queryType": "randomWalk", "refId": "A" } ], @@ -9536,7 +12482,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compression Algorithm Count", + "title": "I/O Limiter Pending Rate and Duration", "tooltip": { "shared": true, "sort": 0, @@ -9552,15 +12498,16 @@ }, "yaxes": [ { - "format": "short", + "decimals": 0, + "format": "ops", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -9579,7 +12526,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The compression ratio of different compression algorithm", + "description": "I/O Limiter current pending count.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9589,24 +12536,25 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 109 + "x": 12, + "y": 18 }, "hiddenSeries": false, - "id": 294, + "id": 86, "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 @@ -9616,33 +12564,79 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/pending/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "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": "avg(tiflash_system_current_metric_RateLimiterPendingWriteRequest{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": true, "interval": "", - "legendFormat": "lz4", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "other-current-{{instance}}", "refId": "A" }, { "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": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "lightweight", + "legendFormat": "bgwrite-current-{{instance}}", "refId": "B" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "fgwrite-current-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "bgread-current-{{instance}}", + "refId": "D" + }, + { + "exemplar": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "fgread-current-{{instance}}", + "refId": "E" + }, + { + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "hide": false, + "interval": "", + "legendFormat": "{{type}}-pending-max", + "refId": "F" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": false, + "interval": "", + "legendFormat": "{{type}}-pending-P99", + "refId": "G" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compression Ratio", + "title": "I/O Limiter Current Pending Count", "tooltip": { "shared": true, "sort": 0, @@ -9658,15 +12652,16 @@ }, "yaxes": [ { + "decimals": 0, "format": "short", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -9680,8 +12675,7 @@ } } ], - "repeat": null, - "title": "Storage", + "title": "Rate Limiter", "type": "row" }, { @@ -9691,8 +12685,9 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 10 }, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "id": 64, "panels": [ { @@ -9712,7 +12707,7 @@ "h": 8, "w": 24, "x": 0, - "y": 103 + "y": 10 }, "hiddenSeries": false, "id": 62, @@ -9831,7 +12826,7 @@ "h": 8, "w": 12, "x": 0, - "y": 111 + "y": 18 }, "height": "", "hiddenSeries": false, @@ -9950,7 +12945,7 @@ "h": 8, "w": 12, "x": 12, - "y": 111 + "y": 18 }, "height": "", "hiddenSeries": false, @@ -10067,7 +13062,7 @@ "h": 9, "w": 24, "x": 0, - "y": 119 + "y": 26 }, "height": "", "hiddenSeries": false, @@ -10189,7 +13184,7 @@ "h": 9, "w": 24, "x": 0, - "y": 128 + "y": 35 }, "hiddenSeries": false, "id": 90, @@ -10296,7 +13291,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 7 +======= + "y": 11 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 119, "panels": [ @@ -10321,7 +13320,7 @@ "h": 8, "w": 12, "x": 0, - "y": 8 + "y": 11 }, "hiddenSeries": false, "id": 128, @@ -10464,7 +13463,7 @@ "h": 8, "w": 12, "x": 12, - "y": 8 + "y": 11 }, "hiddenSeries": false, "id": 129, @@ -10581,7 +13580,7 @@ "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 19 }, "heatmap": {}, "hideZeroBuckets": true, @@ -10643,7 +13642,7 @@ "h": 8, "w": 12, "x": 12, - "y": 16 + "y": 19 }, "hiddenSeries": false, "id": 158, @@ -10779,7 +13778,7 @@ "h": 8, "w": 12, "x": 0, - "y": 24 + "y": 27 }, "hiddenSeries": false, "id": 163, @@ -10884,7 +13883,7 @@ "h": 8, "w": 12, "x": 12, - "y": 24 + "y": 27 }, "hiddenSeries": false, "id": 162, @@ -11004,7 +14003,7 @@ "h": 8, "w": 12, "x": 0, - "y": 32 + "y": 35 }, "hiddenSeries": false, "id": 164, @@ -11109,36 +14108,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 @@ -11146,7 +14142,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -11162,46 +14158,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" @@ -11216,7 +14187,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -11229,7 +14200,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -11255,7 +14226,7 @@ "h": 8, "w": 12, "x": 0, - "y": 40 + "y": 43 }, "height": "", "hiddenSeries": false, @@ -11353,33 +14324,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 @@ -11387,7 +14361,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -11403,23 +14377,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", @@ -11432,11 +14432,11 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -11445,7 +14445,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -11470,7 +14470,7 @@ "h": 9, "w": 24, "x": 0, - "y": 48 + "y": 51 }, "hiddenSeries": false, "id": 232, @@ -11570,7 +14570,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 8 +======= + "y": 12 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 34, "panels": [ @@ -11590,7 +14594,7 @@ "h": 7, "w": 12, "x": 0, - "y": 9 + "y": 12 }, "hiddenSeries": false, "id": 167, @@ -11690,7 +14694,7 @@ "h": 7, "w": 12, "x": 12, - "y": 9 + "y": 12 }, "hiddenSeries": false, "id": 35, @@ -11788,7 +14792,7 @@ "h": 7, "w": 12, "x": 0, - "y": 16 + "y": 19 }, "hiddenSeries": false, "id": 270, @@ -11888,7 +14892,7 @@ "h": 7, "w": 12, "x": 12, - "y": 16 + "y": 19 }, "hiddenSeries": false, "id": 271, @@ -11988,7 +14992,7 @@ "h": 7, "w": 12, "x": 0, - "y": 23 + "y": 26 }, "hiddenSeries": false, "id": 37, @@ -12122,7 +15126,7 @@ "h": 7, "w": 12, "x": 12, - "y": 23 + "y": 26 }, "hiddenSeries": false, "id": 36, @@ -12244,7 +15248,7 @@ "h": 7, "w": 24, "x": 0, - "y": 30 + "y": 33 }, "hiddenSeries": false, "id": 82, @@ -12399,7 +15403,7 @@ "h": 7, "w": 24, "x": 0, - "y": 37 + "y": 40 }, "hiddenSeries": false, "id": 242, @@ -12510,7 +15514,7 @@ "h": 7, "w": 12, "x": 0, - "y": 44 + "y": 47 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12580,7 +15584,7 @@ "h": 7, "w": 12, "x": 12, - "y": 44 + "y": 47 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12650,7 +15654,7 @@ "h": 7, "w": 12, "x": 0, - "y": 51 + "y": 54 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12722,7 +15726,7 @@ "h": 7, "w": 12, "x": 12, - "y": 51 + "y": 54 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12784,7 +15788,7 @@ "h": 7, "w": 12, "x": 0, - "y": 58 + "y": 61 }, "hiddenSeries": false, "id": 235, @@ -12884,7 +15888,7 @@ "h": 7, "w": 12, "x": 12, - "y": 58 + "y": 61 }, "hiddenSeries": false, "id": 241, @@ -12992,7 +15996,7 @@ "h": 7, "w": 12, "x": 0, - "y": 65 + "y": 68 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13073,7 +16077,7 @@ "h": 7, "w": 12, "x": 12, - "y": 65 + "y": 68 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13146,7 +16150,7 @@ "h": 7, "w": 12, "x": 0, - "y": 72 + "y": 75 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13222,7 +16226,7 @@ "h": 7, "w": 12, "x": 12, - "y": 72 + "y": 75 }, "hiddenSeries": false, "id": 249, @@ -13328,7 +16332,7 @@ "h": 7, "w": 12, "x": 0, - "y": 79 + "y": 82 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13401,7 +16405,7 @@ "h": 7, "w": 12, "x": 12, - "y": 79 + "y": 82 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13474,7 +16478,7 @@ "h": 7, "w": 12, "x": 0, - "y": 86 + "y": 89 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13547,7 +16551,7 @@ "h": 7, "w": 12, "x": 12, - "y": 86 + "y": 89 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13612,7 +16616,7 @@ "h": 7, "w": 12, "x": 0, - "y": 93 + "y": 96 }, "hiddenSeries": false, "id": 240, @@ -13716,7 +16720,7 @@ "h": 7, "w": 12, "x": 12, - "y": 93 + "y": 96 }, "hiddenSeries": false, "id": 239, @@ -13853,7 +16857,7 @@ "h": 7, "w": 24, "x": 0, - "y": 100 + "y": 103 }, "hiddenSeries": false, "id": 75, @@ -13977,7 +16981,7 @@ "h": 7, "w": 12, "x": 0, - "y": 107 + "y": 110 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14047,7 +17051,7 @@ "h": 7, "w": 12, "x": 12, - "y": 107 + "y": 110 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14110,7 +17114,7 @@ "h": 7, "w": 12, "x": 0, - "y": 114 + "y": 117 }, "hiddenSeries": false, "id": 263, @@ -14208,13 +17212,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, @@ -14265,7 +17271,7 @@ "title": "Memory by thread", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -14323,7 +17329,7 @@ "h": 7, "w": 12, "x": 0, - "y": 121 + "y": 124 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14395,7 +17401,7 @@ "h": 7, "w": 12, "x": 12, - "y": 121 + "y": 124 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14461,7 +17467,7 @@ "h": 7, "w": 24, "x": 0, - "y": 128 + "y": 131 }, "height": "", "hiddenSeries": false, @@ -14571,7 +17577,7 @@ "h": 7, "w": 24, "x": 0, - "y": 135 + "y": 138 }, "height": "", "hiddenSeries": false, @@ -14687,7 +17693,7 @@ "h": 7, "w": 12, "x": 0, - "y": 142 + "y": 145 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14756,7 +17762,7 @@ "h": 7, "w": 12, "x": 12, - "y": 142 + "y": 145 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14826,7 +17832,7 @@ "h": 7, "w": 12, "x": 0, - "y": 149 + "y": 152 }, "heatmap": {}, "hideZeroBuckets": true, @@ -14892,7 +17898,7 @@ "h": 7, "w": 12, "x": 12, - "y": 149 + "y": 152 }, "hiddenSeries": false, "id": 91, @@ -15020,7 +18026,7 @@ "h": 7, "w": 12, "x": 0, - "y": 156 + "y": 159 }, "hiddenSeries": false, "id": 296, @@ -15116,7 +18122,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 9 +======= + "y": 13 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 95, "panels": [ @@ -15136,7 +18146,7 @@ "h": 8, "w": 12, "x": 0, - "y": 10 + "y": 13 }, "hiddenSeries": false, "id": 99, @@ -15289,7 +18299,7 @@ "h": 8, "w": 12, "x": 12, - "y": 10 + "y": 13 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15346,7 +18356,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 10 +======= + "y": 14 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 274, "panels": [ @@ -15381,7 +18395,7 @@ "h": 8, "w": 12, "x": 0, - "y": 11 + "y": 14 }, "hiddenSeries": false, "id": 286, @@ -15502,7 +18516,7 @@ "h": 8, "w": 12, "x": 12, - "y": 11 + "y": 14 }, "hiddenSeries": false, "id": 288, @@ -15631,7 +18645,7 @@ "h": 8, "w": 12, "x": 0, - "y": 19 + "y": 22 }, "hiddenSeries": false, "id": 282, @@ -15758,7 +18772,7 @@ "h": 8, "w": 12, "x": 12, - "y": 19 + "y": 22 }, "hiddenSeries": false, "id": 284, @@ -15860,7 +18874,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 11 +======= + "y": 15 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 171, "panels": [ @@ -15880,7 +18898,7 @@ "h": 8, "w": 24, "x": 0, - "y": 11 + "y": 15 }, "hiddenSeries": false, "id": 173, @@ -15981,7 +18999,7 @@ "h": 8, "w": 12, "x": 0, - "y": 19 + "y": 23 }, "hiddenSeries": false, "id": 187, @@ -16109,7 +19127,7 @@ "h": 8, "w": 12, "x": 12, - "y": 19 + "y": 23 }, "height": "", "hiddenSeries": false, @@ -16228,7 +19246,7 @@ "h": 8, "w": 12, "x": 0, - "y": 27 + "y": 31 }, "height": "", "hiddenSeries": false, @@ -16338,7 +19356,7 @@ "h": 8, "w": 12, "x": 12, - "y": 27 + "y": 31 }, "height": "", "hiddenSeries": false, @@ -16451,7 +19469,7 @@ "h": 8, "w": 12, "x": 0, - "y": 35 + "y": 39 }, "hiddenSeries": false, "id": 176, @@ -16559,7 +19577,7 @@ "h": 8, "w": 12, "x": 12, - "y": 35 + "y": 39 }, "hiddenSeries": false, "id": 175, @@ -16686,7 +19704,7 @@ "h": 8, "w": 12, "x": 0, - "y": 43 + "y": 47 }, "hiddenSeries": false, "id": 189, @@ -16788,7 +19806,7 @@ "h": 8, "w": 12, "x": 12, - "y": 43 + "y": 47 }, "hiddenSeries": false, "id": 191, @@ -16888,7 +19906,7 @@ "h": 8, "w": 12, "x": 0, - "y": 51 + "y": 55 }, "hiddenSeries": false, "id": 193, @@ -17014,7 +20032,7 @@ "h": 8, "w": 12, "x": 12, - "y": 51 + "y": 55 }, "hiddenSeries": false, "id": 195, @@ -17125,7 +20143,7 @@ "h": 8, "w": 12, "x": 0, - "y": 59 + "y": 63 }, "hiddenSeries": false, "id": 201, @@ -17261,7 +20279,7 @@ "h": 8, "w": 12, "x": 12, - "y": 59 + "y": 63 }, "hiddenSeries": false, "id": 233, @@ -17385,7 +20403,7 @@ "h": 8, "w": 12, "x": 0, - "y": 67 + "y": 71 }, "hiddenSeries": false, "id": 236, @@ -17500,7 +20518,7 @@ "h": 8, "w": 12, "x": 12, - "y": 67 + "y": 71 }, "hiddenSeries": false, "id": 252, @@ -17607,7 +20625,7 @@ "h": 8, "w": 12, "x": 0, - "y": 75 + "y": 79 }, "hiddenSeries": false, "id": 251, @@ -17714,7 +20732,7 @@ "h": 8, "w": 12, "x": 12, - "y": 75 + "y": 79 }, "hiddenSeries": false, "id": 253, @@ -17821,7 +20839,7 @@ "h": 8, "w": 12, "x": 0, - "y": 83 + "y": 87 }, "hiddenSeries": false, "id": 254, @@ -17922,7 +20940,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 12 +======= + "y": 16 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 172, "panels": [ @@ -17943,7 +20965,7 @@ "h": 8, "w": 12, "x": 0, - "y": 12 + "y": 16 }, "hiddenSeries": false, "id": 178, @@ -18061,7 +21083,7 @@ "h": 8, "w": 12, "x": 12, - "y": 12 + "y": 16 }, "hiddenSeries": false, "id": 179, @@ -18235,7 +21257,7 @@ "h": 8, "w": 12, "x": 0, - "y": 20 + "y": 24 }, "hiddenSeries": false, "id": 182, @@ -18353,7 +21375,7 @@ "h": 8, "w": 12, "x": 12, - "y": 20 + "y": 24 }, "hiddenSeries": false, "id": 180, @@ -18480,7 +21502,7 @@ "h": 8, "w": 12, "x": 0, - "y": 28 + "y": 32 }, "hiddenSeries": false, "id": 185, @@ -18607,7 +21629,7 @@ "h": 8, "w": 12, "x": 12, - "y": 28 + "y": 32 }, "hiddenSeries": false, "id": 186, @@ -18709,7 +21731,7 @@ "h": 8, "w": 12, "x": 0, - "y": 36 + "y": 40 }, "hiddenSeries": false, "id": 188, @@ -18828,7 +21850,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 13 +======= + "y": 17 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 202, "panels": [ @@ -18849,7 +21875,7 @@ "h": 8, "w": 12, "x": 0, - "y": 13 + "y": 17 }, "hiddenSeries": false, "id": 224, @@ -18949,7 +21975,7 @@ "h": 8, "w": 12, "x": 12, - "y": 13 + "y": 17 }, "hiddenSeries": false, "id": 226, @@ -19057,7 +22083,7 @@ "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 25 }, "hiddenSeries": false, "id": 228, @@ -19160,7 +22186,7 @@ "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 25 }, "hiddenSeries": false, "id": 222, @@ -19333,7 +22359,7 @@ "h": 8, "w": 12, "x": 0, - "y": 29 + "y": 33 }, "hiddenSeries": false, "id": 230, @@ -19488,7 +22514,7 @@ "h": 8, "w": 12, "x": 12, - "y": 29 + "y": 33 }, "hiddenSeries": false, "id": 218, @@ -19618,7 +22644,7 @@ "h": 8, "w": 12, "x": 0, - "y": 37 + "y": 41 }, "hiddenSeries": false, "id": 220, @@ -19748,7 +22774,7 @@ "h": 8, "w": 12, "x": 12, - "y": 37 + "y": 41 }, "hiddenSeries": false, "id": 216, @@ -19856,6 +22882,117 @@ "align": false, "alignLevel": null } +<<<<<<< HEAD +======= + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "wait notify task details", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 49 + }, + "hiddenSeries": false, + "id": 300, + "legend": { + "alignAsTable": true, + "avg": false, + "current": 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": [ + { + "exemplar": true, + "expr": "max(tiflash_pipeline_wait_on_notify_tasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance, type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(tiflash_pipeline_wait_on_notify_tasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", + "hide": false, + "interval": "", + "legendFormat": "sum({{type}})", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Wait notify task details", + "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": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "title": "Pipeline Model", @@ -19868,7 +23005,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 14 +======= + "y": 18 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 248, "panels": [ @@ -19889,7 +23030,7 @@ "h": 8, "w": 12, "x": 0, - "y": 15 + "y": 18 }, "hiddenSeries": false, "id": 246, From a9025afab279c86a2c2c7f996ee0c7a609304a2f Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Tue, 8 Jul 2025 20:14:56 +0800 Subject: [PATCH 2/6] Resolve conflict Signed-off-by: JaySon-Huang --- .../DeltaMerge/ColumnFile/ColumnFile.h | 13 ++ .../ColumnFile/ColumnFileInMemory.h | 14 -- .../Storages/DeltaMerge/Delta/MemTableSet.cpp | 33 +---- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 102 ++++++++++++++- .../DeltaMerge/DeltaMergeStore_Statistics.cpp | 1 - .../DeltaMerge/DeltaMergeStore_Statistics.h | 123 ------------------ dbms/src/Storages/DeltaMerge/Filter/In.h | 2 +- .../DeltaMerge/Filter/PushDownFilter.cpp | 6 +- .../DeltaMerge/Filter/PushDownFilter.h | 6 - .../DeltaMerge/Filter/WithANNQueryInfo.h | 9 ++ .../Storages/DeltaMerge/Remote/Serializer.cpp | 2 +- dbms/src/Storages/DeltaMerge/ScanContext.cpp | 2 +- dbms/src/Storages/DeltaMerge/ScanContext.h | 47 +------ .../DeltaMerge/tests/gtest_segment.cpp | 1 - dbms/src/Storages/StorageDeltaMerge.cpp | 33 +---- dbms/src/Storages/StorageDeltaMerge.h | 15 --- .../System/StorageSystemDTSegments.cpp | 1 - dbms/src/TestUtils/gtests_dbms_main.cpp | 3 - 18 files changed, 140 insertions(+), 273 deletions(-) delete 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 8c4f5ebd5e4..5dc2989dd4d 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h @@ -74,6 +74,19 @@ 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: diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h index 9a8bd2f93e5..8e627df1d5f 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h @@ -31,20 +31,6 @@ 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; diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp index ddeb0bbec2f..952e94b4081 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp @@ -22,12 +22,6 @@ #include #include -<<<<<<< HEAD -namespace DB -{ -namespace DM -{ -======= namespace CurrentMetrics { extern const Metric DT_NumMemTable; @@ -100,7 +94,6 @@ MemTableSet::MemTableSet(const ColumnFiles & in_memory_files) stat.resetTo(column_files.size(), new_rows, new_bytes, new_alloc_bytes, new_deletes); } ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) void MemTableSet::appendColumnFileInner(const ColumnFilePtr & column_file) { if (!column_files.empty()) @@ -326,9 +319,9 @@ ColumnFileSetSnapshotPtr MemTableSet::createSnapshot( column_files.back()->disableAppend(); auto snap = std::make_shared(data_provider); - snap->rows = rows; - snap->bytes = bytes; - snap->deletes = deletes; + snap->rows = stat.rows; + snap->bytes = stat.bytes; + snap->deletes = stat.deletes; snap->column_files.reserve(column_files.size()); size_t total_rows = 0; @@ -361,16 +354,7 @@ ColumnFileSetSnapshotPtr MemTableSet::createSnapshot( total_deletes, stat.deletes.load()); -<<<<<<< HEAD return snap; -======= - return std::make_shared( - data_provider, - std::move(column_files_snap), - stat.rows, - stat.bytes, - stat.deletes); ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( @@ -411,15 +395,9 @@ ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( "Files: {}", flush_task->getFlushRows(), flush_task->getFlushDeletes(), -<<<<<<< HEAD - rows.load(), - deletes.load(), - columnFilesToString(column_files)); -======= stat.rows.load(), stat.deletes.load(), - ColumnFile::filesToString(column_files)); ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + columnFilesToString(column_files)); throw Exception("Rows and deletes check failed.", ErrorCodes::LOGICAL_ERROR); } @@ -463,5 +441,4 @@ void MemTableSet::removeColumnFilesInFlushTask(const ColumnFileFlushTask & flush } -} // namespace DM -} // namespace DB +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 8b6b221ac2f..3aebdbd1c23 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -27,7 +27,6 @@ #include #include #include -#include #include #include #include @@ -78,6 +77,107 @@ 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_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 column_count = 0; + 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; + UInt64 delta_cache_alloc_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 4f2d9492bc1..81ed528c7c2 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -13,7 +13,6 @@ // limitations under the License. #include -#include #include #include #include diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h deleted file mode 100644 index 9dbbf7d702f..00000000000 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.h +++ /dev/null @@ -1,123 +0,0 @@ -// 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 column_count = 0; - 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; - UInt64 delta_cache_alloc_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/Filter/In.h b/dbms/src/Storages/DeltaMerge/Filter/In.h index b78af878f8a..bfe87418a57 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/In.h +++ b/dbms/src/Storages/DeltaMerge/Filter/In.h @@ -47,7 +47,7 @@ class In : public RSOperator ","); buf.append("]}"); return buf.toString(); - }; + } Poco::JSON::Object::Ptr toJSONObject() override { diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.cpp b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.cpp index 1727deb57ff..6e3ab68663a 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.cpp @@ -199,17 +199,13 @@ PushDownFilterPtr PushDownFilter::build( tracing_logger); } -Poco::JSON::Object::Ptr PushDownExecutor::toJSONObject() const +Poco::JSON::Object::Ptr PushDownFilter::toJSONObject() const { Poco::JSON::Object::Ptr json = new Poco::JSON::Object(); if (rs_operator) { json->set("rs_operator", rs_operator->toJSONObject()); } - if (ann_query_info) - { - json->set("ann_query_info", ann_query_info->ShortDebugString()); - } return json; } diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h index 69fc9db27e8..538531afe0a 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -40,13 +40,7 @@ class PushDownFilter public: PushDownFilter( const RSOperatorPtr & rs_operator_, -<<<<<<< HEAD:dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h - const ExpressionActionsPtr & beofre_where_, -======= - const ANNQueryInfoPtr & ann_query_info_, - const FTSQueryInfoPtr & fts_query_info_, const ExpressionActionsPtr & before_where_, ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)):dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h const ExpressionActionsPtr & project_after_where_, const ColumnDefinesPtr & filter_columns_, const String filter_column_name_, diff --git a/dbms/src/Storages/DeltaMerge/Filter/WithANNQueryInfo.h b/dbms/src/Storages/DeltaMerge/Filter/WithANNQueryInfo.h index df721a93edd..0cfd4b6f75c 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/WithANNQueryInfo.h +++ b/dbms/src/Storages/DeltaMerge/Filter/WithANNQueryInfo.h @@ -45,6 +45,15 @@ class WithANNQueryInfo : public RSOperator return ""; } + Poco::JSON::Object::Ptr toJSONObject() override + { + Poco::JSON::Object::Ptr obj = new Poco::JSON::Object(); + if (child) + obj->set("child", child->toJSONObject()); + obj->set("ann_query_info", ann_query_info->ShortDebugString()); + return obj; + } + ColIds getColumnIDs() override { if (child) diff --git a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp index 97edab433bb..f260c54f012 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp @@ -333,7 +333,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/ScanContext.cpp b/dbms/src/Storages/DeltaMerge/ScanContext.cpp index a87a9cdc7ad..18d65170dd8 100644 --- a/dbms/src/Storages/DeltaMerge/ScanContext.cpp +++ b/dbms/src/Storages/DeltaMerge/ScanContext.cpp @@ -18,7 +18,7 @@ #include #pragma GCC diagnostic pop #include -#include +#include #include #include diff --git a/dbms/src/Storages/DeltaMerge/ScanContext.h b/dbms/src/Storages/DeltaMerge/ScanContext.h index 01ece99966a..e73c056c4c2 100644 --- a/dbms/src/Storages/DeltaMerge/ScanContext.h +++ b/dbms/src/Storages/DeltaMerge/ScanContext.h @@ -28,8 +28,8 @@ namespace DB::DM { -class PushDownExecutor; -using PushDownExecutorPtr = std::shared_ptr; +class PushDownFilter; +using PushDownFilterPtr = 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,6 +69,7 @@ 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 @@ -99,48 +100,8 @@ class ScanContext std::atomic total_vector_idx_read_vec_time_ms{0}; std::atomic total_vector_idx_read_others_time_ms{0}; -<<<<<<< HEAD -======= - std::atomic inverted_idx_load_from_s3{0}; - std::atomic inverted_idx_load_from_disk{0}; - std::atomic inverted_idx_load_from_cache{0}; - std::atomic inverted_idx_load_time_ms{0}; - std::atomic inverted_idx_search_time_ms{0}; - std::atomic inverted_idx_search_skipped_packs{0}; - std::atomic inverted_idx_indexed_rows{0}; - std::atomic inverted_idx_search_selected_rows{0}; - - std::atomic fts_n_from_inmemory_noindex{0}; - std::atomic fts_n_from_tiny_index{0}; - std::atomic fts_n_from_tiny_noindex{0}; - std::atomic fts_n_from_dmf_index{0}; - std::atomic fts_n_from_dmf_noindex{0}; - std::atomic fts_rows_from_inmemory_noindex{0}; - std::atomic fts_rows_from_tiny_index{0}; - std::atomic fts_rows_from_tiny_noindex{0}; - std::atomic fts_rows_from_dmf_index{0}; - std::atomic fts_rows_from_dmf_noindex{0}; - std::atomic fts_idx_load_total_ms{0}; - std::atomic fts_idx_load_from_cache{0}; - std::atomic fts_idx_load_from_column_file{0}; - std::atomic fts_idx_load_from_stable_s3{0}; - std::atomic fts_idx_load_from_stable_disk{0}; - std::atomic fts_idx_search_n{0}; - std::atomic fts_idx_search_total_ms{0}; - std::atomic fts_idx_dm_search_rows{0}; - std::atomic fts_idx_dm_total_read_fts_ms{0}; - std::atomic fts_idx_dm_total_read_others_ms{0}; - std::atomic fts_idx_tiny_search_rows{0}; - std::atomic fts_idx_tiny_total_read_fts_ms{0}; - std::atomic fts_idx_tiny_total_read_others_ms{0}; - std::atomic fts_brute_total_read_ms{0}; - std::atomic fts_brute_total_search_ms{0}; - - const KeyspaceID keyspace_id; - ReadMode read_mode = ReadMode::Normal; // note: share struct padding with keyspace_id ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) const String resource_group_name; - PushDownExecutorPtr pushdown_executor; + PushDownFilterPtr pushdown_executor; explicit ScanContext(const String & name = "") : resource_group_name(name) diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp index 639f6f4badc..b66f3f2a3aa 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp @@ -608,7 +608,6 @@ try } CATCH - class SegmentEnableLogicalSplitTest : public SegmentOperationTest { protected: diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 8fc482c0bbd..785cd85c72a 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -92,11 +92,8 @@ StorageDeltaMerge::StorageDeltaMerge( Timestamp tombstone, Context & global_context_) : IManageableStorage{columns_, tombstone} -<<<<<<< HEAD , data_path_contains_database_name(db_engine != "TiFlash") -======= , holder_counter(CurrentMetrics::DT_NumStorageDeltaMerge, 1) ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) , store_inited(false) , max_column_id_used(0) , global_context(global_context_.getGlobalContext()) @@ -844,23 +841,12 @@ BlockInputStreams StorageDeltaMerge::read( query_info.req_id, tracing_logger); -<<<<<<< HEAD auto filter = PushDownFilter::build(query_info, columns_to_read, store->getTableColumns(), context, tracing_logger); -======= - auto pushdown_executor = PushDownExecutor::build( - query_info, - columns_to_read, - store->getTableColumns(), - query_info.dag_query ? query_info.dag_query->used_indexes - : google::protobuf::RepeatedPtrField{}, - context, - tracing_logger); ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) 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; + scan_context->pushdown_executor = filter; auto streams = store->read( context, @@ -869,7 +855,7 @@ BlockInputStreams StorageDeltaMerge::read( ranges, num_streams, /*start_ts=*/mvcc_query_info.start_ts, - pushdown_executor, + filter, runtime_filter_list, query_info.dag_query == nullptr ? 0 : query_info.dag_query->rf_max_wait_time_ms, query_info.req_id, @@ -939,23 +925,12 @@ void StorageDeltaMerge::read( query_info.req_id, tracing_logger); -<<<<<<< HEAD auto filter = PushDownFilter::build(query_info, columns_to_read, store->getTableColumns(), context, tracing_logger); -======= - auto pushdown_executor = PushDownExecutor::build( - query_info, - columns_to_read, - store->getTableColumns(), - query_info.dag_query ? query_info.dag_query->used_indexes - : google::protobuf::RepeatedPtrField{}, - context, - tracing_logger); ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) 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; + scan_context->pushdown_executor = filter; store->read( exec_context_, @@ -966,7 +941,7 @@ void StorageDeltaMerge::read( ranges, num_streams, /*start_ts=*/mvcc_query_info.start_ts, - pushdown_executor, + filter, runtime_filter_list, query_info.dag_query == nullptr ? 0 : query_info.dag_query->rf_max_wait_time_ms, query_info.req_id, diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index f483525fb3b..8fd0bbe991b 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -311,24 +311,9 @@ class StorageDeltaMerge // Used to allocate new column-id when this table is NOT synced from TiDB ColumnID max_column_id_used; -<<<<<<< HEAD std::atomic shutdown_called{false}; std::atomic next_version = 1; //TODO: remove this!!! -======= - // 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; - - // `decoding_schema_changed` and `decoding_schema_epoch` must be used under the protection of table structure lock - bool decoding_schema_changed = false; - - const bool data_path_contains_database_name = false; ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) Context & global_context; diff --git a/dbms/src/Storages/System/StorageSystemDTSegments.cpp b/dbms/src/Storages/System/StorageSystemDTSegments.cpp index c5d859c01e9..aec140ddac0 100644 --- a/dbms/src/Storages/System/StorageSystemDTSegments.cpp +++ b/dbms/src/Storages/System/StorageSystemDTSegments.cpp @@ -21,7 +21,6 @@ #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 f25552c9712..4caeed23150 100644 --- a/dbms/src/TestUtils/gtests_dbms_main.cpp +++ b/dbms/src/TestUtils/gtests_dbms_main.cpp @@ -23,11 +23,8 @@ #include #include #include -<<<<<<< HEAD #include -======= #include ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) #include #include From a004120d6575f84a7a80605db1c8bd45466ad290 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Tue, 8 Jul 2025 00:30:49 +0800 Subject: [PATCH 3/6] Add RateLimter threshold metrics Signed-off-by: JaySon-Huang --- dbms/src/Common/TiFlashMetrics.h | 7 +++++++ dbms/src/IO/BaseFile/RateLimiter.cpp | 4 ++++ 2 files changed, 11 insertions(+) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 26c0273d29d..8e622cd41b2 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -353,6 +353,13 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_fg_write_alloc_bytes, {"type", "fg_write_alloc_bytes"}), \ F(type_bg_write_req_bytes, {"type", "bg_write_req_bytes"}), \ F(type_bg_write_alloc_bytes, {"type", "bg_write_alloc_bytes"})) \ + M(tiflash_storage_io_limiter_curr, \ + "Current limit bytes per second of Storage I/O limiter", \ + Gauge, \ + F(type_fg_read_bytes, {"type", "fg_read_bytes"}), \ + F(type_bg_read_bytes, {"type", "bg_read_bytes"}), \ + F(type_fg_write_bytes, {"type", "fg_write_bytes"}), \ + F(type_bg_write_bytes, {"type", "bg_write_bytes"})) \ M(tiflash_storage_rough_set_filter_rate, \ "Bucketed histogram of rough set filter rate", \ Histogram, \ diff --git a/dbms/src/IO/BaseFile/RateLimiter.cpp b/dbms/src/IO/BaseFile/RateLimiter.cpp index a4d857561ca..8afc0c3c140 100644 --- a/dbms/src/IO/BaseFile/RateLimiter.cpp +++ b/dbms/src/IO/BaseFile/RateLimiter.cpp @@ -526,6 +526,7 @@ void IORateLimiter::updateReadLimiter(Int64 bg_bytes, Int64 fg_bytes) { bg_read_limiter->updateMaxBytesPerSec(bg_bytes); } + GET_METRIC(tiflash_storage_io_limiter_curr, type_bg_read_bytes).Set(bg_bytes); if (fg_bytes == 0) { @@ -539,6 +540,7 @@ void IORateLimiter::updateReadLimiter(Int64 bg_bytes, Int64 fg_bytes) { fg_read_limiter->updateMaxBytesPerSec(fg_bytes); } + GET_METRIC(tiflash_storage_io_limiter_curr, type_fg_read_bytes).Set(fg_bytes); } void IORateLimiter::updateWriteLimiter(Int64 bg_bytes, Int64 fg_bytes) @@ -556,6 +558,7 @@ void IORateLimiter::updateWriteLimiter(Int64 bg_bytes, Int64 fg_bytes) { bg_write_limiter->updateMaxBytesPerSec(bg_bytes); } + GET_METRIC(tiflash_storage_io_limiter_curr, type_bg_write_bytes).Set(bg_bytes); if (fg_bytes == 0) { @@ -569,6 +572,7 @@ void IORateLimiter::updateWriteLimiter(Int64 bg_bytes, Int64 fg_bytes) { fg_write_limiter->updateMaxBytesPerSec(fg_bytes); } + GET_METRIC(tiflash_storage_io_limiter_curr, type_fg_write_bytes).Set(fg_bytes); } void IORateLimiter::setBackgroundThreadIds(std::vector thread_ids) From 6c6492369513df364b181623de76056299882d54 Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Tue, 8 Jul 2025 20:31:43 +0800 Subject: [PATCH 4/6] Update TiFlash-Summary Signed-off-by: JaySon-Huang --- metrics/grafana/tiflash_summary.json | 2154 +++++--------------------- 1 file changed, 383 insertions(+), 1771 deletions(-) diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index aafe0a2daf3..7b44dc580f9 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -2916,7 +2916,6 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -3154,7 +3153,7 @@ "x": 0, "y": 2 }, - "id": 327, + "id": 298, "panels": [ { "aliasColors": {}, @@ -3173,10 +3172,10 @@ "h": 7, "w": 12, "x": 0, - "y": 3 + "y": 2 }, "hiddenSeries": false, - "id": 329, + "id": 300, "legend": { "alignAsTable": true, "avg": false, @@ -3286,10 +3285,10 @@ "h": 7, "w": 12, "x": 12, - "y": 3 + "y": 2 }, "hiddenSeries": false, - "id": 331, + "id": 302, "legend": { "alignAsTable": true, "avg": false, @@ -3391,10 +3390,10 @@ "h": 7, "w": 12, "x": 0, - "y": 10 + "y": 9 }, "hiddenSeries": false, - "id": 333, + "id": 304, "legend": { "alignAsTable": true, "avg": false, @@ -3496,10 +3495,10 @@ "h": 7, "w": 12, "x": 12, - "y": 10 + "y": 9 }, "hiddenSeries": false, - "id": 335, + "id": 306, "legend": { "alignAsTable": true, "avg": false, @@ -3615,7 +3614,7 @@ "h": 7, "w": 12, "x": 0, - "y": 4 + "y": 3 }, "hiddenSeries": false, "id": 9, @@ -3713,7 +3712,7 @@ "h": 7, "w": 12, "x": 12, - "y": 4 + "y": 3 }, "hiddenSeries": false, "id": 2, @@ -3810,7 +3809,7 @@ "h": 7, "w": 12, "x": 0, - "y": 11 + "y": 10 }, "hiddenSeries": false, "id": 11, @@ -3933,7 +3932,7 @@ "h": 7, "w": 12, "x": 12, - "y": 11 + "y": 10 }, "hiddenSeries": false, "id": 12, @@ -4030,7 +4029,7 @@ "h": 7, "w": 12, "x": 0, - "y": 18 + "y": 17 }, "hiddenSeries": false, "id": 13, @@ -4148,7 +4147,7 @@ "h": 7, "w": 12, "x": 12, - "y": 18 + "y": 17 }, "hiddenSeries": false, "id": 14, @@ -4247,7 +4246,7 @@ "h": 7, "w": 12, "x": 0, - "y": 25 + "y": 24 }, "hiddenSeries": false, "id": 63, @@ -4363,7 +4362,7 @@ "h": 7, "w": 12, "x": 12, - "y": 25 + "y": 24 }, "hiddenSeries": false, "id": 165, @@ -4462,7 +4461,7 @@ "h": 7, "w": 12, "x": 0, - "y": 32 + "y": 31 }, "hiddenSeries": false, "id": 100, @@ -4561,7 +4560,7 @@ "h": 7, "w": 12, "x": 12, - "y": 32 + "y": 31 }, "hiddenSeries": false, "id": 77, @@ -4659,7 +4658,7 @@ "h": 7, "w": 12, "x": 0, - "y": 39 + "y": 38 }, "hiddenSeries": false, "id": 102, @@ -4758,7 +4757,7 @@ "h": 7, "w": 12, "x": 12, - "y": 39 + "y": 38 }, "hiddenSeries": false, "id": 101, @@ -4858,7 +4857,7 @@ "h": 7, "w": 12, "x": 0, - "y": 46 + "y": 45 }, "hiddenSeries": false, "id": 157, @@ -4957,7 +4956,7 @@ "h": 7, "w": 12, "x": 12, - "y": 46 + "y": 45 }, "hiddenSeries": false, "id": 103, @@ -5056,7 +5055,7 @@ "h": 7, "w": 12, "x": 0, - "y": 53 + "y": 52 }, "hiddenSeries": false, "id": 199, @@ -5155,7 +5154,7 @@ "h": 7, "w": 12, "x": 12, - "y": 53 + "y": 52 }, "hiddenSeries": false, "id": 166, @@ -5237,208 +5236,6 @@ "align": false, "alignLevel": null } -<<<<<<< HEAD -======= - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 0, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 60 - }, - "hiddenSeries": false, - "id": 297, - "legend": { - "alignAsTable": true, - "avg": false, - "current": 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": [ - { - "exemplar": true, - "expr": "sum(rate(tiflash_network_transmission_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Network Transmission", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The establish calldata details", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 0, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 60 - }, - "hiddenSeries": false, - "id": 299, - "legend": { - "alignAsTable": true, - "avg": false, - "current": 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": [ - { - "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)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Establish calldata details", - "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": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "repeat": null, @@ -5474,7 +5271,7 @@ "h": 8, "w": 12, "x": 0, - "y": 5 + "y": 4 }, "hiddenSeries": false, "id": 107, @@ -5576,7 +5373,7 @@ "h": 8, "w": 12, "x": 12, - "y": 5 + "y": 4 }, "hiddenSeries": false, "id": 109, @@ -5714,7 +5511,7 @@ "h": 8, "w": 12, "x": 0, - "y": 13 + "y": 12 }, "hiddenSeries": false, "id": 111, @@ -5825,7 +5622,7 @@ "h": 8, "w": 12, "x": 12, - "y": 13 + "y": 12 }, "hiddenSeries": false, "id": 113, @@ -5936,7 +5733,7 @@ "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 20 }, "hiddenSeries": false, "id": 117, @@ -6037,7 +5834,7 @@ "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 20 }, "hiddenSeries": false, "id": 115, @@ -6171,7 +5968,7 @@ "h": 7, "w": 12, "x": 0, - "y": 6 + "y": 5 }, "hiddenSeries": false, "id": 19, @@ -6293,7 +6090,7 @@ "h": 7, "w": 12, "x": 12, - "y": 6 + "y": 5 }, "hiddenSeries": false, "id": 18, @@ -6391,7 +6188,7 @@ "h": 7, "w": 12, "x": 0, - "y": 13 + "y": 12 }, "hiddenSeries": false, "id": 20, @@ -6522,7 +6319,7 @@ "x": 0, "y": 6 }, - "id": 307, + "id": 308, "panels": [ { "aliasColors": {}, @@ -6544,7 +6341,7 @@ "h": 8, "w": 12, "x": 0, - "y": 7 + "y": 1 }, "hiddenSeries": false, "id": 310, @@ -6671,10 +6468,10 @@ "h": 8, "w": 12, "x": 12, - "y": 7 + "y": 1 }, "hiddenSeries": false, - "id": 309, + "id": 312, "legend": { "alignAsTable": true, "avg": false, @@ -6796,10 +6593,10 @@ "h": 8, "w": 12, "x": 0, - "y": 15 + "y": 9 }, "hiddenSeries": false, - "id": 316, + "id": 314, "legend": { "alignAsTable": true, "avg": false, @@ -6900,11 +6697,11 @@ "h": 8, "w": 12, "x": 12, - "y": 15 + "y": 9 }, "height": "", "hiddenSeries": false, - "id": 314, + "id": 316, "legend": { "alignAsTable": true, "avg": false, @@ -7027,7 +6824,7 @@ "h": 8, "w": 12, "x": 0, - "y": 23 + "y": 17 }, "hiddenSeries": false, "id": 318, @@ -7141,11 +6938,11 @@ "h": 8, "w": 12, "x": 12, - "y": 23 + "y": 17 }, "height": "", "hiddenSeries": false, - "id": 312, + "id": 320, "legend": { "alignAsTable": true, "avg": false, @@ -7259,7 +7056,7 @@ "x": 0, "y": 7 }, - "id": 321, + "id": 322, "panels": [ { "aliasColors": {}, @@ -7281,7 +7078,7 @@ "y": 8 }, "hiddenSeries": false, - "id": 325, + "id": 324, "legend": { "alignAsTable": true, "avg": false, @@ -7381,7 +7178,7 @@ "y": 8 }, "hiddenSeries": false, - "id": 324, + "id": 326, "legend": { "alignAsTable": true, "avg": false, @@ -7491,7 +7288,7 @@ "y": 16 }, "hiddenSeries": false, - "id": 319, + "id": 328, "legend": { "alignAsTable": true, "avg": false, @@ -7601,7 +7398,7 @@ "y": 16 }, "hiddenSeries": false, - "id": 323, + "id": 330, "legend": { "alignAsTable": true, "avg": false, @@ -7711,7 +7508,7 @@ "y": 24 }, "hiddenSeries": false, - "id": 238, + "id": 332, "legend": { "alignAsTable": true, "avg": false, @@ -7837,7 +7634,7 @@ "y": 24 }, "hiddenSeries": false, - "id": 169, + "id": 334, "legend": { "avg": false, "current": false, @@ -7986,7 +7783,7 @@ "y": 32 }, "hiddenSeries": false, - "id": 168, + "id": 336, "legend": { "alignAsTable": true, "avg": false, @@ -10184,31 +9981,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The storage I/O limiter metrics.", + "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": 0, - "y": 70 + "y": 73 }, "hiddenSeries": false, - "id": 84, + "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, @@ -10219,28 +10016,60 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "cache_hit_ratio", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_io_limiter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"add_cache_stale|add_cache_succ|add_cache_total_bytes_limit|add_cache_reach_count_limit\"}[1m])) by (type)", "format": "time_series", - "instant": false, + "hide": false, + "interval": "", "intervalFactor": 2, "legendFormat": "{{type}}", "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_hit|get_cache_copy\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "get_cache_hit", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_miss|get_cache_hit|get_cache_part|get_cache_copy\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "get_cache_total", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_hit|get_cache_copy\"}[1m]))/sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_miss|get_cache_hit|get_cache_part|get_cache_copy\"}[1m]))", + "hide": false, + "instant": false, + "interval": "", + "legendFormat": "cache_hit_ratio", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Throughput", + "title": "Data Sharing", "tooltip": { "shared": true, "sort": 0, @@ -10256,8 +10085,8 @@ }, "yaxes": [ { - "decimals": 0, - "format": "binBps", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -10265,11 +10094,11 @@ "show": true }, { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -10284,31 +10113,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The storage I/O limiter metrics.", + "description": "The information of read thread scheduling.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 70 + "y": 73 }, "hiddenSeries": false, - "id": 266, + "id": 269, "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, @@ -10319,12 +10148,12 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/-/", + "alias": "/push_block/", "yaxis": 2 } ], @@ -10334,9 +10163,9 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_io_limiter_pending_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "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", - "instant": false, + "hide": false, "interval": "", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -10347,7 +10176,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Pending Rate and Duration", + "title": "Read Thread Scheduling", "tooltip": { "shared": true, "sort": 0, @@ -10363,7 +10192,7 @@ }, "yaxes": [ { - "decimals": 0, + "decimals": null, "format": "ops", "label": null, "logBase": 1, @@ -10372,11 +10201,11 @@ "show": true }, { - "format": "s", + "format": "binBps", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -10391,35 +10220,30 @@ "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": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, -<<<<<<< HEAD - "y": 78 -======= - "y": 73 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "y": 81 }, "hiddenSeries": false, - "id": 132, + "id": 292, "legend": { - "alignAsTable": true, - "avg": false, + "alignAsTable": false, + "avg": true, "current": false, "max": false, "min": false, "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -10433,49 +10257,45 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "cache_hit_ratio", - "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=~\"add_cache_stale|add_cache_succ|add_cache_total_bytes_limit|add_cache_reach_count_limit\"}[1m])) by (type)", + "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", - "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{type}}", + "intervalFactor": 1, + "legendFormat": "999-{{type}}", "refId": "A" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_hit|get_cache_copy\"}[1m]))", - "hide": false, + "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": "", - "legendFormat": "get_cache_hit", - "refId": "C" + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_miss|get_cache_hit|get_cache_part|get_cache_copy\"}[1m]))", - "hide": false, + "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": "", - "legendFormat": "get_cache_total", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_hit|get_cache_copy\"}[1m]))/sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"get_cache_miss|get_cache_hit|get_cache_part|get_cache_copy\"}[1m]))", - "hide": false, - "instant": false, + "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": "", - "legendFormat": "cache_hit_ratio", + "intervalFactor": 1, + "legendFormat": "80-{{type}}", "refId": "D" } ], @@ -10483,7 +10303,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Data Sharing", + "title": "Read Thread Internal Duration", "tooltip": { "shared": true, "sort": 0, @@ -10499,8 +10319,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "µs", "label": null, "logBase": 1, "max": null, @@ -10508,12 +10327,12 @@ "show": true }, { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { @@ -10527,7 +10346,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "I/O Limiter current pending count.", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -10538,20 +10357,20 @@ "h": 8, "w": 12, "x": 12, - "y": 78 + "y": 81 }, "hiddenSeries": false, - "id": 86, + "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, @@ -10567,7 +10386,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/pending/", + "alias": "/max_snapshot_lifetime/", "yaxis": 2 } ], @@ -10576,760 +10395,74 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_system_current_metric_RateLimiterPendingWriteRequest{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "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\"}", "format": "time_series", "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "other-current-{{instance}}", + "legendFormat": "snapshot_list-{{instance}}", "refId": "A" }, { - "exemplar": true, - "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "bgwrite-current-{{instance}}", + "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": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "fgwrite-current-{{instance}}", + "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": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "bgread-current-{{instance}}", + "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" }, { - "exemplar": true, - "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "fgread-current-{{instance}}", + "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" }, { - "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", - "hide": false, - "interval": "", - "legendFormat": "{{type}}-pending-max", + "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" }, { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": false, - "interval": "", - "legendFormat": "{{type}}-pending-P99", - "refId": "G" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "I/O Limiter Current Pending Count", - "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": "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": "cache misses or cache hits of mark_cache.\nBased on this infactor, we can check whether mark_cache is large enough", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 86 - }, - "hiddenSeries": false, - "id": 169, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "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)", - "hide": false, - "interval": "", - "legendFormat": "mark cache hits", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Effectiveness of Mark Cache", - "tooltip": { - "shared": true, - "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, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "percentunit", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "percent", - "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 information of read thread scheduling.", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 0, - "fillGradient": 0, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, -<<<<<<< HEAD - "y": 86 -======= - "y": 73 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) - }, - "hiddenSeries": false, - "id": 269, - "legend": { - "alignAsTable": true, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "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": [ - { - "alias": "/push_block/", - "yaxis": 2 - } - ], - "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)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Read Thread Scheduling", - "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": "binBps", - "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}", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 8, - "w": 12, -<<<<<<< HEAD - "x": 12, - "y": 117 -======= - "x": 0, - "y": 81 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) - }, - "hiddenSeries": false, - "id": 292, - "legend": { -<<<<<<< HEAD - "alignAsTable": false, -======= - "alignAsTable": true, ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) - "avg": true, - "current": false, - "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": [ - { - "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", - "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": 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", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "µs", -<<<<<<< HEAD - "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 memory usage of mark cache and minmax index cache", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 0, - "fillGradient": 0, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 94 - }, - "hiddenSeries": false, - "id": 238, - "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", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/limit/", - "color": "#F2495C", - "hideTooltip": true, - "legend": false, - "linewidth": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "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\"}", - "hide": false, - "interval": "", - "legendFormat": "minmax_index_cache_{{instance}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_RNDeltaIndexCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "hide": false, - "interval": "", - "legendFormat": "rn_delta_index_cache_{{instance}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Mark Cache and Minmax Index Cache Memory Usage", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, -======= - "label": null, - "logBase": 1, - "max": null, ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) - "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": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, -<<<<<<< HEAD - "y": 94 -======= - "y": 81 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) - }, - "hiddenSeries": false, - "id": 88, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/max_snapshot_lifetime/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "tiflash_system_current_metric_DT_SegmentReadTasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "read_tasks-{{instance}}", - "refId": "I" - }, - { - "exemplar": true, - "expr": "tiflash_system_current_metric_PSMVCCSnapshotsList{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "hide": true, - "interval": "", - "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" - }, - { - "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" - }, - { - "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}}", + "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" }, { @@ -11351,235 +10484,25 @@ { "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", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "s", - "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}", -<<<<<<< HEAD - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 102 - }, - "hiddenSeries": false, - "id": 291, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "tiflash_memory_usage_by_class{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Memory by class", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", -======= ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "gridPos": { -<<<<<<< HEAD - "h": 7, - "w": 12, - "x": 12, - "y": 102 - }, - "hiddenSeries": false, - "id": 289, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"alloc_.*\"}", - "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "refId": "B" + "hide": true, + "intervalFactor": 1, + "legendFormat": "max_snapshot_lifetime_stable-{{instance}}", + "refId": "L" }, { - "exemplar": true, - "expr": "-tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"dealloc_.*\"}", + "expr": "tiflash_system_asynchronous_metric_MaxDTMetaOldestSnapshotLifetime{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "refId": "A" + "hide": true, + "intervalFactor": 1, + "legendFormat": "max_snapshot_lifetime_meta-{{instance}}", + "refId": "M" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory by thread", + "title": "Read Snapshots", "tooltip": { "shared": true, "sort": 0, @@ -11595,19 +10518,19 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "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 } ], @@ -11622,7 +10545,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": [] @@ -11630,22 +10553,13 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 109 - }, - "hiddenSeries": false, - "id": 168, -======= - "h": 8, + "h": 7, "w": 12, "x": 0, "y": 89 }, "hiddenSeries": false, "id": 291, ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "legend": { "avg": false, "current": false, @@ -11657,8 +10571,6 @@ }, "lines": true, "linewidth": 1, -<<<<<<< HEAD -======= "links": [], "nullPointMode": "null", "options": { @@ -11739,7 +10651,7 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, "y": 89 @@ -11776,7 +10688,6 @@ "exemplar": true, "expr": "tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"alloc_.*\"}", "format": "time_series", - "hide": false, "interval": "", "intervalFactor": 2, "legendFormat": "{{instance}}-{{type}}", @@ -11800,7 +10711,7 @@ "title": "Memory by thread", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -11848,10 +10759,10 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 97 + "y": 96 }, "hiddenSeries": false, "id": 294, @@ -11868,7 +10779,6 @@ }, "lines": true, "linewidth": 1, ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "nullPointMode": "null", "options": { "alertThreshold": true @@ -11885,60 +10795,26 @@ "targets": [ { "exemplar": true, -<<<<<<< HEAD - "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", -======= "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": "", "legendFormat": "lz4", ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "queryType": "randomWalk", "refId": "A" }, { "exemplar": true, -<<<<<<< HEAD - "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", - "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", - "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", - "refId": "D" -======= "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": "", "legendFormat": "lightweight", "refId": "B" ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, -<<<<<<< HEAD - "title": "Schema of Column File", -======= "title": "Compression Ratio", ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "tooltip": { "shared": true, "sort": 0, @@ -11989,15 +10865,10 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, -<<<<<<< HEAD - "x": 0, - "y": 117 -======= "x": 12, - "y": 97 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "y": 96 }, "hiddenSeries": false, "id": 293, @@ -12078,116 +10949,6 @@ "align": false, "alignLevel": null } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The compression ratio of different compression algorithm", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, -<<<<<<< HEAD - "y": 109 -======= - "y": 104 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) - }, - "hiddenSeries": false, - "id": 294, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "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": "", - "legendFormat": "lz4", - "queryType": "randomWalk", - "refId": "A" - }, - { - "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]))", - "hide": false, - "interval": "", - "legendFormat": "lightweight", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Compression Ratio", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } } ], "repeat": null, @@ -12203,9 +10964,7 @@ "x": 0, "y": 9 }, -<<<<<<< HEAD -======= - "id": 303, + "id": 338, "panels": [ { "aliasColors": {}, @@ -12227,7 +10986,7 @@ "y": 10 }, "hiddenSeries": false, - "id": 84, + "id": 340, "legend": { "alignAsTable": true, "avg": false, @@ -12329,7 +11088,7 @@ "y": 10 }, "hiddenSeries": false, - "id": 305, + "id": 342, "legend": { "alignAsTable": true, "avg": false, @@ -12433,7 +11192,7 @@ "y": 18 }, "hiddenSeries": false, - "id": 266, + "id": 344, "legend": { "alignAsTable": true, "avg": false, @@ -12540,7 +11299,7 @@ "y": 18 }, "hiddenSeries": false, - "id": 86, + "id": 346, "legend": { "alignAsTable": true, "avg": false, @@ -12687,7 +11446,6 @@ "x": 0, "y": 10 }, ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "id": 64, "panels": [ { @@ -12707,7 +11465,7 @@ "h": 8, "w": 24, "x": 0, - "y": 10 + "y": 103 }, "hiddenSeries": false, "id": 62, @@ -12826,7 +11584,7 @@ "h": 8, "w": 12, "x": 0, - "y": 18 + "y": 111 }, "height": "", "hiddenSeries": false, @@ -12945,7 +11703,7 @@ "h": 8, "w": 12, "x": 12, - "y": 18 + "y": 111 }, "height": "", "hiddenSeries": false, @@ -13062,7 +11820,7 @@ "h": 9, "w": 24, "x": 0, - "y": 26 + "y": 119 }, "height": "", "hiddenSeries": false, @@ -13184,7 +11942,7 @@ "h": 9, "w": 24, "x": 0, - "y": 35 + "y": 128 }, "hiddenSeries": false, "id": 90, @@ -13291,11 +12049,7 @@ "h": 1, "w": 24, "x": 0, -<<<<<<< HEAD - "y": 7 -======= "y": 11 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 119, "panels": [ @@ -13320,7 +12074,7 @@ "h": 8, "w": 12, "x": 0, - "y": 11 + "y": 8 }, "hiddenSeries": false, "id": 128, @@ -13463,7 +12217,7 @@ "h": 8, "w": 12, "x": 12, - "y": 11 + "y": 8 }, "hiddenSeries": false, "id": 129, @@ -13580,7 +12334,7 @@ "h": 8, "w": 12, "x": 0, - "y": 19 + "y": 16 }, "heatmap": {}, "hideZeroBuckets": true, @@ -13642,7 +12396,7 @@ "h": 8, "w": 12, "x": 12, - "y": 19 + "y": 16 }, "hiddenSeries": false, "id": 158, @@ -13778,7 +12532,7 @@ "h": 8, "w": 12, "x": 0, - "y": 27 + "y": 24 }, "hiddenSeries": false, "id": 163, @@ -13883,7 +12637,7 @@ "h": 8, "w": 12, "x": 12, - "y": 27 + "y": 24 }, "hiddenSeries": false, "id": 162, @@ -14003,7 +12757,7 @@ "h": 8, "w": 12, "x": 0, - "y": 35 + "y": 32 }, "hiddenSeries": false, "id": 164, @@ -14108,33 +12862,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": 35 + "y": 32 }, "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 @@ -14142,7 +12899,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -14158,21 +12915,46 @@ "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", + "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": "", - "intervalFactor": 1, - "legendFormat": "size-{{instance}}", - "refId": "A" + "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" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PageStorage Pending Writers Num", + "title": "Number of Tables", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -14187,7 +12969,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -14200,7 +12982,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -14226,7 +13008,7 @@ "h": 8, "w": 12, "x": 0, - "y": 43 + "y": 40 }, "height": "", "hiddenSeries": false, @@ -14322,38 +13104,35 @@ "aliasColors": {}, "bars": false, "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, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The num of pending writers in PageStorage", "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 43 + "y": 40 }, "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 @@ -14361,7 +13140,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -14377,49 +13156,23 @@ "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", - "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": "", - "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" + "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": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -14432,11 +13185,11 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -14445,7 +13198,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -14470,7 +13223,7 @@ "h": 9, "w": 24, "x": 0, - "y": 51 + "y": 48 }, "hiddenSeries": false, "id": 232, @@ -14570,11 +13323,7 @@ "h": 1, "w": 24, "x": 0, -<<<<<<< HEAD - "y": 8 -======= "y": 12 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 34, "panels": [ @@ -14594,7 +13343,7 @@ "h": 7, "w": 12, "x": 0, - "y": 12 + "y": 9 }, "hiddenSeries": false, "id": 167, @@ -14694,7 +13443,7 @@ "h": 7, "w": 12, "x": 12, - "y": 12 + "y": 9 }, "hiddenSeries": false, "id": 35, @@ -14792,7 +13541,7 @@ "h": 7, "w": 12, "x": 0, - "y": 19 + "y": 16 }, "hiddenSeries": false, "id": 270, @@ -14892,7 +13641,7 @@ "h": 7, "w": 12, "x": 12, - "y": 19 + "y": 16 }, "hiddenSeries": false, "id": 271, @@ -14992,7 +13741,7 @@ "h": 7, "w": 12, "x": 0, - "y": 26 + "y": 23 }, "hiddenSeries": false, "id": 37, @@ -15126,7 +13875,7 @@ "h": 7, "w": 12, "x": 12, - "y": 26 + "y": 23 }, "hiddenSeries": false, "id": 36, @@ -15248,7 +13997,7 @@ "h": 7, "w": 24, "x": 0, - "y": 33 + "y": 30 }, "hiddenSeries": false, "id": 82, @@ -15403,7 +14152,7 @@ "h": 7, "w": 24, "x": 0, - "y": 40 + "y": 37 }, "hiddenSeries": false, "id": 242, @@ -15514,7 +14263,7 @@ "h": 7, "w": 12, "x": 0, - "y": 47 + "y": 44 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15584,7 +14333,7 @@ "h": 7, "w": 12, "x": 12, - "y": 47 + "y": 44 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15654,7 +14403,7 @@ "h": 7, "w": 12, "x": 0, - "y": 54 + "y": 51 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15726,7 +14475,7 @@ "h": 7, "w": 12, "x": 12, - "y": 54 + "y": 51 }, "heatmap": {}, "hideZeroBuckets": true, @@ -15788,7 +14537,7 @@ "h": 7, "w": 12, "x": 0, - "y": 61 + "y": 58 }, "hiddenSeries": false, "id": 235, @@ -15888,7 +14637,7 @@ "h": 7, "w": 12, "x": 12, - "y": 61 + "y": 58 }, "hiddenSeries": false, "id": 241, @@ -15996,7 +14745,7 @@ "h": 7, "w": 12, "x": 0, - "y": 68 + "y": 65 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16077,7 +14826,7 @@ "h": 7, "w": 12, "x": 12, - "y": 68 + "y": 65 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16150,7 +14899,7 @@ "h": 7, "w": 12, "x": 0, - "y": 75 + "y": 72 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16226,7 +14975,7 @@ "h": 7, "w": 12, "x": 12, - "y": 75 + "y": 72 }, "hiddenSeries": false, "id": 249, @@ -16332,7 +15081,7 @@ "h": 7, "w": 12, "x": 0, - "y": 82 + "y": 79 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16405,7 +15154,7 @@ "h": 7, "w": 12, "x": 12, - "y": 82 + "y": 79 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16478,7 +15227,7 @@ "h": 7, "w": 12, "x": 0, - "y": 89 + "y": 86 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16551,7 +15300,7 @@ "h": 7, "w": 12, "x": 12, - "y": 89 + "y": 86 }, "heatmap": {}, "hideZeroBuckets": true, @@ -16616,7 +15365,7 @@ "h": 7, "w": 12, "x": 0, - "y": 96 + "y": 93 }, "hiddenSeries": false, "id": 240, @@ -16720,7 +15469,7 @@ "h": 7, "w": 12, "x": 12, - "y": 96 + "y": 93 }, "hiddenSeries": false, "id": 239, @@ -16857,7 +15606,7 @@ "h": 7, "w": 24, "x": 0, - "y": 103 + "y": 100 }, "hiddenSeries": false, "id": 75, @@ -16981,7 +15730,7 @@ "h": 7, "w": 12, "x": 0, - "y": 110 + "y": 107 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17051,7 +15800,7 @@ "h": 7, "w": 12, "x": 12, - "y": 110 + "y": 107 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17114,7 +15863,7 @@ "h": 7, "w": 12, "x": 0, - "y": 117 + "y": 114 }, "hiddenSeries": false, "id": 263, @@ -17212,15 +15961,13 @@ "h": 7, "w": 12, "x": 12, - "y": 117 + "y": 114 }, "hiddenSeries": false, "id": 272, "legend": { "avg": false, "current": false, - "hideEmpty": true, - "hideZero": true, "max": false, "min": false, "show": true, @@ -17271,7 +16018,7 @@ "title": "Memory by thread", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -17329,7 +16076,7 @@ "h": 7, "w": 12, "x": 0, - "y": 124 + "y": 121 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17401,7 +16148,7 @@ "h": 7, "w": 12, "x": 12, - "y": 124 + "y": 121 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17467,7 +16214,7 @@ "h": 7, "w": 24, "x": 0, - "y": 131 + "y": 128 }, "height": "", "hiddenSeries": false, @@ -17577,7 +16324,7 @@ "h": 7, "w": 24, "x": 0, - "y": 138 + "y": 135 }, "height": "", "hiddenSeries": false, @@ -17693,7 +16440,7 @@ "h": 7, "w": 12, "x": 0, - "y": 145 + "y": 142 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17762,7 +16509,7 @@ "h": 7, "w": 12, "x": 12, - "y": 145 + "y": 142 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17832,7 +16579,7 @@ "h": 7, "w": 12, "x": 0, - "y": 152 + "y": 149 }, "heatmap": {}, "hideZeroBuckets": true, @@ -17898,7 +16645,7 @@ "h": 7, "w": 12, "x": 12, - "y": 152 + "y": 149 }, "hiddenSeries": false, "id": 91, @@ -18026,7 +16773,7 @@ "h": 7, "w": 12, "x": 0, - "y": 159 + "y": 156 }, "hiddenSeries": false, "id": 296, @@ -18122,11 +16869,7 @@ "h": 1, "w": 24, "x": 0, -<<<<<<< HEAD - "y": 9 -======= "y": 13 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 95, "panels": [ @@ -18146,7 +16889,7 @@ "h": 8, "w": 12, "x": 0, - "y": 13 + "y": 10 }, "hiddenSeries": false, "id": 99, @@ -18299,7 +17042,7 @@ "h": 8, "w": 12, "x": 12, - "y": 13 + "y": 10 }, "heatmap": {}, "hideZeroBuckets": true, @@ -18356,11 +17099,7 @@ "h": 1, "w": 24, "x": 0, -<<<<<<< HEAD - "y": 10 -======= "y": 14 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 274, "panels": [ @@ -18395,7 +17134,7 @@ "h": 8, "w": 12, "x": 0, - "y": 14 + "y": 11 }, "hiddenSeries": false, "id": 286, @@ -18516,7 +17255,7 @@ "h": 8, "w": 12, "x": 12, - "y": 14 + "y": 11 }, "hiddenSeries": false, "id": 288, @@ -18645,7 +17384,7 @@ "h": 8, "w": 12, "x": 0, - "y": 22 + "y": 19 }, "hiddenSeries": false, "id": 282, @@ -18772,7 +17511,7 @@ "h": 8, "w": 12, "x": 12, - "y": 22 + "y": 19 }, "hiddenSeries": false, "id": 284, @@ -18874,11 +17613,7 @@ "h": 1, "w": 24, "x": 0, -<<<<<<< HEAD - "y": 11 -======= "y": 15 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 171, "panels": [ @@ -18898,7 +17633,7 @@ "h": 8, "w": 24, "x": 0, - "y": 15 + "y": 11 }, "hiddenSeries": false, "id": 173, @@ -18999,7 +17734,7 @@ "h": 8, "w": 12, "x": 0, - "y": 23 + "y": 19 }, "hiddenSeries": false, "id": 187, @@ -19127,7 +17862,7 @@ "h": 8, "w": 12, "x": 12, - "y": 23 + "y": 19 }, "height": "", "hiddenSeries": false, @@ -19246,7 +17981,7 @@ "h": 8, "w": 12, "x": 0, - "y": 31 + "y": 27 }, "height": "", "hiddenSeries": false, @@ -19356,7 +18091,7 @@ "h": 8, "w": 12, "x": 12, - "y": 31 + "y": 27 }, "height": "", "hiddenSeries": false, @@ -19469,7 +18204,7 @@ "h": 8, "w": 12, "x": 0, - "y": 39 + "y": 35 }, "hiddenSeries": false, "id": 176, @@ -19577,7 +18312,7 @@ "h": 8, "w": 12, "x": 12, - "y": 39 + "y": 35 }, "hiddenSeries": false, "id": 175, @@ -19704,7 +18439,7 @@ "h": 8, "w": 12, "x": 0, - "y": 47 + "y": 43 }, "hiddenSeries": false, "id": 189, @@ -19806,7 +18541,7 @@ "h": 8, "w": 12, "x": 12, - "y": 47 + "y": 43 }, "hiddenSeries": false, "id": 191, @@ -19906,7 +18641,7 @@ "h": 8, "w": 12, "x": 0, - "y": 55 + "y": 51 }, "hiddenSeries": false, "id": 193, @@ -20032,7 +18767,7 @@ "h": 8, "w": 12, "x": 12, - "y": 55 + "y": 51 }, "hiddenSeries": false, "id": 195, @@ -20143,7 +18878,7 @@ "h": 8, "w": 12, "x": 0, - "y": 63 + "y": 59 }, "hiddenSeries": false, "id": 201, @@ -20279,7 +19014,7 @@ "h": 8, "w": 12, "x": 12, - "y": 63 + "y": 59 }, "hiddenSeries": false, "id": 233, @@ -20403,7 +19138,7 @@ "h": 8, "w": 12, "x": 0, - "y": 71 + "y": 67 }, "hiddenSeries": false, "id": 236, @@ -20518,7 +19253,7 @@ "h": 8, "w": 12, "x": 12, - "y": 71 + "y": 67 }, "hiddenSeries": false, "id": 252, @@ -20625,7 +19360,7 @@ "h": 8, "w": 12, "x": 0, - "y": 79 + "y": 75 }, "hiddenSeries": false, "id": 251, @@ -20732,7 +19467,7 @@ "h": 8, "w": 12, "x": 12, - "y": 79 + "y": 75 }, "hiddenSeries": false, "id": 253, @@ -20839,7 +19574,7 @@ "h": 8, "w": 12, "x": 0, - "y": 87 + "y": 83 }, "hiddenSeries": false, "id": 254, @@ -20940,11 +19675,7 @@ "h": 1, "w": 24, "x": 0, -<<<<<<< HEAD - "y": 12 -======= "y": 16 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 172, "panels": [ @@ -20965,7 +19696,7 @@ "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 12 }, "hiddenSeries": false, "id": 178, @@ -21083,7 +19814,7 @@ "h": 8, "w": 12, "x": 12, - "y": 16 + "y": 12 }, "hiddenSeries": false, "id": 179, @@ -21257,7 +19988,7 @@ "h": 8, "w": 12, "x": 0, - "y": 24 + "y": 20 }, "hiddenSeries": false, "id": 182, @@ -21375,7 +20106,7 @@ "h": 8, "w": 12, "x": 12, - "y": 24 + "y": 20 }, "hiddenSeries": false, "id": 180, @@ -21502,7 +20233,7 @@ "h": 8, "w": 12, "x": 0, - "y": 32 + "y": 28 }, "hiddenSeries": false, "id": 185, @@ -21629,7 +20360,7 @@ "h": 8, "w": 12, "x": 12, - "y": 32 + "y": 28 }, "hiddenSeries": false, "id": 186, @@ -21731,7 +20462,7 @@ "h": 8, "w": 12, "x": 0, - "y": 40 + "y": 36 }, "hiddenSeries": false, "id": 188, @@ -21850,11 +20581,7 @@ "h": 1, "w": 24, "x": 0, -<<<<<<< HEAD - "y": 13 -======= "y": 17 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 202, "panels": [ @@ -21875,7 +20602,7 @@ "h": 8, "w": 12, "x": 0, - "y": 17 + "y": 13 }, "hiddenSeries": false, "id": 224, @@ -21975,7 +20702,7 @@ "h": 8, "w": 12, "x": 12, - "y": 17 + "y": 13 }, "hiddenSeries": false, "id": 226, @@ -22083,7 +20810,7 @@ "h": 8, "w": 12, "x": 0, - "y": 25 + "y": 21 }, "hiddenSeries": false, "id": 228, @@ -22186,7 +20913,7 @@ "h": 8, "w": 12, "x": 12, - "y": 25 + "y": 21 }, "hiddenSeries": false, "id": 222, @@ -22359,7 +21086,7 @@ "h": 8, "w": 12, "x": 0, - "y": 33 + "y": 29 }, "hiddenSeries": false, "id": 230, @@ -22514,7 +21241,7 @@ "h": 8, "w": 12, "x": 12, - "y": 33 + "y": 29 }, "hiddenSeries": false, "id": 218, @@ -22644,7 +21371,7 @@ "h": 8, "w": 12, "x": 0, - "y": 41 + "y": 37 }, "hiddenSeries": false, "id": 220, @@ -22774,7 +21501,7 @@ "h": 8, "w": 12, "x": 12, - "y": 41 + "y": 37 }, "hiddenSeries": false, "id": 216, @@ -22882,117 +21609,6 @@ "align": false, "alignLevel": null } -<<<<<<< HEAD -======= - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "wait notify task details", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 0, - "fillGradient": 0, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 49 - }, - "hiddenSeries": false, - "id": 300, - "legend": { - "alignAsTable": true, - "avg": false, - "current": 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": [ - { - "exemplar": true, - "expr": "max(tiflash_pipeline_wait_on_notify_tasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance, type)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(tiflash_pipeline_wait_on_notify_tasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", - "hide": false, - "interval": "", - "legendFormat": "sum({{type}})", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Wait notify task details", - "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": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "title": "Pipeline Model", @@ -23005,11 +21621,7 @@ "h": 1, "w": 24, "x": 0, -<<<<<<< HEAD - "y": 14 -======= "y": 18 ->>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 248, "panels": [ @@ -23030,7 +21642,7 @@ "h": 8, "w": 12, "x": 0, - "y": 18 + "y": 15 }, "hiddenSeries": false, "id": 246, From 40417661def0e828dbcb503aa34a5ee1ac8bbe1f Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Tue, 8 Jul 2025 20:54:22 +0800 Subject: [PATCH 5/6] Update Grafana TiFlash-Proxy-Details Signed-off-by: JaySon-Huang --- metrics/grafana/tiflash_proxy_details.json | 3969 ++++++++++++++++---- 1 file changed, 3337 insertions(+), 632 deletions(-) diff --git a/metrics/grafana/tiflash_proxy_details.json b/metrics/grafana/tiflash_proxy_details.json index f33fdb0c0df..72bc556b24a 100644 --- a/metrics/grafana/tiflash_proxy_details.json +++ b/metrics/grafana/tiflash_proxy_details.json @@ -14,7 +14,7 @@ "type": "grafana", "id": "grafana", "name": "Grafana", - "version": "7.5.11" + "version": "7.5.17" }, { "type": "panel", @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1670499325053, + "iteration": 1742544258476, "links": [], "panels": [ { @@ -113,7 +113,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -220,7 +220,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -327,7 +327,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -434,7 +434,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -541,7 +541,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -662,7 +662,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -725,38 +725,1641 @@ { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The memory usage of raft entry cache per TiFlash instance", + "description": null, + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 25 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4560, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "((\n tiflash_proxy_tikv_server_mem_trace_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",name=~\"raftstore-.*\"}\n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{name}}", + "metric": "", + "query": "((\n tikv_server_mem_trace_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",name=~\"raftstore-.*\"}\n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "((\n tiflash_proxy_raft_engine_memory_usage\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-raft-engine", + "metric": "", + "query": "((\n raft_engine_memory_usage\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) ", + "refId": "B", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Memory trace", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + } + ], + "repeat": null, + "title": "Cluster", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 1 + }, + "id": 4558, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The count of operations per second", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 2 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4538, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_apply_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "write {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_apply_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_read_entry_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "read_entry {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_read_entry_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_read_message_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "read_message {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_read_message_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "C", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Operation", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time used in write operation", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 2 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4540, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [ + { + "alias": "/^count/", + "bars": false, + "dashLength": 1, + "dashes": true, + "fill": 2, + "fillBelowTo": null, + "lines": true, + "spaceLength": 1, + "transform": "negative-Y", + "yaxis": 2, + "zindex": -3 + }, + { + "alias": "/^avg/", + "bars": false, + "fill": 7, + "fillBelowTo": null, + "lines": true, + "yaxis": 1, + "zindex": 0 + } + ], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.9999,(\n sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99.99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.9999,(\n sum(rate(\n raft_engine_write_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "(sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg {{$additional_groupby}}", + "metric": "", + "query": "(sum(rate(\n raft_engine_write_duration_seconds_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n raft_engine_write_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "refId": "C", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": true, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "count {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "D", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Duration", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The I/O flow rate", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 9 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4554, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_size_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "write {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_size_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_background_rewrite_bytes_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "rewrite-{{type}} {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_background_rewrite_bytes_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "refId": "B", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Flow", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "99% duration breakdown of write operation", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 9 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4556, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_preprocess_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "wait {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_preprocess_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_leader_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "wal {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_leader_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_apply_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "apply {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_apply_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "C", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Duration Breakdown (99%)", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The bytes per write", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 16 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4550, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [ + { + "alias": "/^count/", + "bars": false, + "dashLength": 1, + "dashes": true, + "fill": 2, + "fillBelowTo": null, + "lines": true, + "spaceLength": 1, + "transform": "negative-Y", + "yaxis": 2, + "zindex": -3 + }, + { + "alias": "/^avg/", + "bars": false, + "fill": 7, + "fillBelowTo": null, + "lines": true, + "yaxis": 1, + "zindex": 0 + } + ], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.9999,(\n sum(rate(\n tiflash_proxy_raft_engine_write_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99.99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.9999,(\n sum(rate(\n raft_engine_write_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "(sum(rate(\n tiflash_proxy_raft_engine_write_size_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n tiflash_proxy_raft_engine_write_size_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg {{$additional_groupby}}", + "metric": "", + "query": "(sum(rate(\n raft_engine_write_size_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n raft_engine_write_size_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "refId": "C", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_size_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by () ", + "format": "time_series", + "hide": true, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "count {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_size_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "D", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Bytes / Written", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "999% duration breakdown of WAL write operation", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 16 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4552, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_write_leader_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_write_leader_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_sync_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "sync {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_sync_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_allocate_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "allocate {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_allocate_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "C", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_rotate_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "rotate {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_rotate_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "D", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "WAL Duration Breakdown (999%)", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The average number of files", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 23 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4546, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "avg((\n tiflash_proxy_raft_engine_log_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "avg((\n raft_engine_log_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "avg((\n tiflash_proxy_raft_engine_swap_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "swap {{$additional_groupby}}", + "metric": "", + "query": "avg((\n raft_engine_swap_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by ($additional_groupby) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "avg((\n tiflash_proxy_raft_engine_recycled_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}-recycle {{$additional_groupby}}", + "metric": "", + "query": "avg((\n raft_engine_recycled_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "refId": "C", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "File Count", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The 99% duration of operations other than write", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 23 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4548, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_read_entry_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "read_entry {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_read_entry_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_read_message_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "read_message {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_read_message_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_purge_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "purge {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_purge_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "C", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Other Durations (99%)", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The average number of log entries", "editable": true, "error": false, "fieldConfig": { - "defaults": {}, + "defaults": { + "links": [] + }, "overrides": [] }, - "fill": 0, - "fillGradient": 0, - "grid": {}, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 25 + "y": 30 }, + "height": null, "hiddenSeries": false, - "id": 4536, + "hideTimeOverride": false, + "id": 4542, + "interval": null, + "isNew": true, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", + "sort": "max", "sortDesc": true, "total": false, "values": true @@ -764,42 +2367,54 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", + "repeat": null, + "repeatDirection": null, "seriesOverrides": [], "spaceLength": 10, + "span": null, "stack": false, "steppedLine": false, "targets": [ { + "datasource": "${DS_TEST-CLUSTER}", "exemplar": true, - "expr": "tiflash_proxy_tikv_server_mem_trace_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"raftstore-.*\"}", + "expr": "avg((\n tiflash_proxy_raft_engine_log_entry_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", "format": "time_series", + "hide": false, + "instant": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{name}} {{instance}}", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "avg((\n raft_engine_log_entry_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", "refId": "A", - "step": 10 + "step": 10, + "target": "" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Entry Cache", + "title": "Entry Count", "tooltip": { - "msResolution": false, + "msResolution": true, "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, + "transformations": [], "type": "graph", "xaxis": { "buckets": null, @@ -810,14 +2425,16 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": null, + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { + "decimals": null, "format": "short", "label": null, "logBase": 1, @@ -828,12 +2445,218 @@ ], "yaxis": { "align": false, - "alignLevel": null + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The compression ratio per write", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 30 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4544, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [ + { + "alias": "/^count/", + "bars": false, + "dashLength": 1, + "dashes": true, + "fill": 2, + "fillBelowTo": null, + "lines": true, + "spaceLength": 1, + "transform": "negative-Y", + "yaxis": 2, + "zindex": -3 + }, + { + "alias": "/^avg/", + "bars": false, + "fill": 7, + "fillBelowTo": null, + "lines": true, + "yaxis": 1, + "zindex": 0 + } + ], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.9999,(\n sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99.99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.9999,(\n sum(rate(\n raft_engine_write_compression_ratio_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_compression_ratio_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "(sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg {{$additional_groupby}}", + "metric": "", + "query": "(sum(rate(\n raft_engine_write_compression_ratio_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n raft_engine_write_compression_ratio_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "refId": "C", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "count {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_compression_ratio_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "D", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Compression Ratio", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 } } ], - "repeat": null, - "title": "Cluster", + "title": "Raft Engine", "type": "row" }, { @@ -843,7 +2666,7 @@ "h": 1, "w": 24, "x": 0, - "y": 1 + "y": 2 }, "id": 2743, "panels": [ @@ -1864,7 +3687,7 @@ "h": 1, "w": 24, "x": 0, - "y": 2 + "y": 3 }, "id": 2744, "panels": [ @@ -3102,7 +4925,7 @@ "h": 1, "w": 24, "x": 0, - "y": 3 + "y": 4 }, "id": 2745, "panels": [ @@ -3715,7 +5538,7 @@ "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 5 }, "id": 2746, "panels": [ @@ -5275,7 +7098,7 @@ "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 6 }, "id": 2747, "panels": [ @@ -5287,13 +7110,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of requests that TiKV sends to PD", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 6 + "y": 7 }, + "hiddenSeries": false, "id": 1069, "legend": { "alignAsTable": true, @@ -5313,7 +7142,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5380,13 +7213,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed by requests that TiKV sends to PD", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 6 + "y": 7 }, + "hiddenSeries": false, "id": 1070, "legend": { "alignAsTable": true, @@ -5406,7 +7245,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5473,13 +7316,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": " \tThe total number of PD heartbeat messages", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 14 + "y": 15 }, + "hiddenSeries": false, "id": 1215, "legend": { "alignAsTable": true, @@ -5499,7 +7348,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5566,13 +7419,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The total number of peers validated by the PD worker", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 14 + "y": 15 }, + "hiddenSeries": false, "id": 1396, "legend": { "alignAsTable": true, @@ -5592,7 +7451,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5664,7 +7527,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 7 }, "id": 2748, "panels": [ @@ -5677,14 +7540,20 @@ "description": "The time consumed when Raft applies log", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 7 + "y": 8 }, + "hiddenSeries": false, "id": 31, "legend": { "alignAsTable": true, @@ -5703,7 +7572,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5789,14 +7662,20 @@ "description": "The time consumed for Raft to apply logs per TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 7 + "y": 8 }, + "hiddenSeries": false, "id": 32, "legend": { "alignAsTable": true, @@ -5815,7 +7694,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5884,14 +7767,20 @@ "description": "The time consumed when Raft appends log", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 14 + "y": 15 }, + "hiddenSeries": false, "id": 39, "legend": { "alignAsTable": true, @@ -5910,7 +7799,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5996,14 +7889,20 @@ "description": "The time consumed when Raft appends log on each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 14 + "y": 15 }, + "hiddenSeries": false, "id": 40, "legend": { "alignAsTable": true, @@ -6022,7 +7921,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6089,13 +7992,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The time consumed when Raft commits log", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 22 }, + "hiddenSeries": false, "id": 3690, "legend": { "alignAsTable": true, @@ -6112,7 +8021,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -6191,13 +8104,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The time consumed when Raft commits log on each TiKV instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 22 }, + "hiddenSeries": false, "id": 3688, "legend": { "alignAsTable": true, @@ -6214,7 +8133,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -6284,7 +8207,7 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 8 }, "id": 2749, "panels": [ @@ -6298,14 +8221,20 @@ "description": "The count of different ready type of Raft", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 8 + "y": 9 }, + "hiddenSeries": false, "id": 5, "legend": { "alignAsTable": true, @@ -6325,7 +8254,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6404,14 +8337,20 @@ "description": "The time consumed for peer processes to be ready in Raft", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 8 + "y": 9 }, + "hiddenSeries": false, "id": 118, "legend": { "alignAsTable": true, @@ -6431,7 +8370,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6509,14 +8452,20 @@ "description": "The time consumed by raftstore events (P99).99", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 17 }, + "hiddenSeries": false, "id": 123, "legend": { "alignAsTable": true, @@ -6536,7 +8485,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6616,7 +8569,7 @@ "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 9 }, "id": 2750, "panels": [ @@ -6630,14 +8583,20 @@ "description": "The number of Raft messages sent by each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 47 + "y": 10 }, + "hiddenSeries": false, "id": 1615, "legend": { "alignAsTable": true, @@ -6657,7 +8616,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6727,14 +8690,20 @@ "description": "The number of Raft messages flushed by each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 47 + "y": 10 }, + "hiddenSeries": false, "id": 1616, "legend": { "alignAsTable": true, @@ -6754,7 +8723,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6823,13 +8796,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The number of Raft messages received by each TiKV instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 54 + "y": 17 }, + "hiddenSeries": false, "id": 106, "legend": { "alignAsTable": true, @@ -6849,7 +8828,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6918,14 +8901,20 @@ "description": "The number of different types of Raft messages that are sent", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 54 + "y": 17 }, + "hiddenSeries": false, "id": 11, "legend": { "alignAsTable": true, @@ -6945,7 +8934,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7015,14 +9008,20 @@ "description": "The total number of vote messages that are sent in Raft", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 61 + "y": 24 }, + "hiddenSeries": false, "id": 25, "legend": { "alignAsTable": true, @@ -7042,7 +9041,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7112,14 +9115,20 @@ "description": "The number of dropped Raft messages per type", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 61 + "y": 24 }, + "hiddenSeries": false, "id": 1309, "legend": { "alignAsTable": true, @@ -7139,7 +9148,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7211,7 +9224,7 @@ "h": 1, "w": 24, "x": 0, - "y": 9 + "y": 10 }, "id": 2751, "panels": [ @@ -7223,13 +9236,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The proposal count of all Regions in a mio tick", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 48 + "y": 11 }, + "hiddenSeries": false, "id": 108, "legend": { "alignAsTable": true, @@ -7249,7 +9268,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7319,14 +9342,20 @@ "description": "The number of proposals per type", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 48 + "y": 11 }, + "hiddenSeries": false, "id": 7, "legend": { "alignAsTable": true, @@ -7346,7 +9375,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7417,14 +9450,20 @@ "description": "The number of read proposals which are made by each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 55 + "y": 18 }, + "hiddenSeries": false, "id": 119, "legend": { "alignAsTable": true, @@ -7444,7 +9483,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7515,14 +9558,20 @@ "description": "The number of write proposals which are made by each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 55 + "y": 18 }, + "hiddenSeries": false, "id": 120, "legend": { "alignAsTable": true, @@ -7542,7 +9591,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7612,14 +9665,20 @@ "description": "The wait time of each proposal", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 62 + "y": 25 }, + "hiddenSeries": false, "id": 41, "legend": { "alignAsTable": true, @@ -7638,7 +9697,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7724,14 +9787,20 @@ "description": "The wait time of each proposal in each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 62 + "y": 25 }, + "hiddenSeries": false, "id": 42, "legend": { "alignAsTable": true, @@ -7750,7 +9819,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7818,14 +9891,20 @@ "datasource": "${DS_TEST-CLUSTER}", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 69 + "y": 32 }, + "hiddenSeries": false, "id": 2535, "legend": { "alignAsTable": true, @@ -7844,7 +9923,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7929,14 +10012,20 @@ "datasource": "${DS_TEST-CLUSTER}", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 69 + "y": 32 }, + "hiddenSeries": false, "id": 2536, "legend": { "alignAsTable": true, @@ -7955,7 +10044,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8022,13 +10115,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The rate at which peers propose logs", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 76 + "y": 39 }, + "hiddenSeries": false, "id": 1975, "legend": { "alignAsTable": true, @@ -8047,7 +10146,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8114,14 +10217,20 @@ "datasource": "${DS_TEST-CLUSTER}", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 76 + "y": 39 }, + "hiddenSeries": false, "id": 4375, "legend": { "alignAsTable": true, @@ -8140,7 +10249,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8221,7 +10334,7 @@ "h": 1, "w": 24, "x": 0, - "y": 10 + "y": 11 }, "id": 2752, "panels": [ @@ -8235,14 +10348,20 @@ "description": "The number of admin proposals", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 11 + "y": 12 }, + "hiddenSeries": false, "id": 76, "legend": { "alignAsTable": true, @@ -8262,7 +10381,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8333,14 +10456,20 @@ "description": "The number of the processed apply command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 11 + "y": 12 }, + "hiddenSeries": false, "id": 77, "legend": { "alignAsTable": true, @@ -8360,7 +10489,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8431,14 +10564,20 @@ "description": "The number of raftstore split checksss", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 18 + "y": 19 }, + "hiddenSeries": false, "id": 70, "legend": { "alignAsTable": true, @@ -8458,7 +10597,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8529,14 +10672,20 @@ "description": "The time consumed when running split check in .9999", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 18 + "y": 19 }, + "hiddenSeries": false, "id": 71, "legend": { "alignAsTable": true, @@ -8557,7 +10706,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8630,7 +10783,7 @@ "h": 1, "w": 24, "x": 0, - "y": 11 + "y": 12 }, "id": 4200, "panels": [ @@ -8641,13 +10794,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The time used by each level in the unified read pool per second. Level 0 refers to small queries.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, "y": 13 }, + "hiddenSeries": false, "id": 4194, "legend": { "alignAsTable": true, @@ -8665,7 +10824,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -8730,13 +10893,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The chance that level 0 (small) tasks are scheduled in the unified read pool.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, "y": 13 }, + "hiddenSeries": false, "id": 4196, "legend": { "alignAsTable": true, @@ -8753,7 +10922,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -8818,13 +10991,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The number of concurrently running tasks in the unified read pool.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, "y": 21 }, + "hiddenSeries": false, "id": 4198, "legend": { "alignAsTable": true, @@ -8841,7 +11020,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -8911,7 +11094,7 @@ "h": 1, "w": 24, "x": 0, - "y": 12 + "y": 13 }, "id": 2754, "panels": [ @@ -8925,7 +11108,12 @@ "description": "The total count of different kinds of commands received", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -8933,6 +11121,7 @@ "x": 0, "y": 14 }, + "hiddenSeries": false, "id": 2, "legend": { "alignAsTable": true, @@ -8954,7 +11143,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9024,7 +11217,12 @@ "description": "The total number of engine asynchronous request errors", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9032,6 +11230,7 @@ "x": 12, "y": 14 }, + "hiddenSeries": false, "id": 8, "legend": { "alignAsTable": true, @@ -9053,7 +11252,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9124,7 +11327,12 @@ "description": "The time consumed by processing asynchronous snapshot requests", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9132,6 +11340,7 @@ "x": 0, "y": 22 }, + "hiddenSeries": false, "id": 15, "legend": { "alignAsTable": true, @@ -9152,7 +11361,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9238,7 +11451,12 @@ "description": "The time consumed by processing asynchronous write requests", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9246,6 +11464,7 @@ "x": 12, "y": 22 }, + "hiddenSeries": false, "id": 109, "legend": { "alignAsTable": true, @@ -9266,7 +11485,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9354,7 +11577,7 @@ "h": 1, "w": 24, "x": 0, - "y": 13 + "y": 14 }, "id": 2755, "panels": [ @@ -9366,7 +11589,12 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The total number of commands on each stage", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 10, "w": 12, @@ -9374,6 +11602,7 @@ "y": 15 }, "height": "400", + "hiddenSeries": false, "id": 167, "legend": { "alignAsTable": true, @@ -9394,7 +11623,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9469,7 +11702,12 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The total writing bytes of commands on each stage", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 10, "w": 12, @@ -9477,6 +11715,7 @@ "y": 15 }, "height": "400", + "hiddenSeries": false, "id": 3834, "legend": { "alignAsTable": true, @@ -9497,7 +11736,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9566,7 +11809,12 @@ "description": "The count of different priority commands", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9575,6 +11823,7 @@ "y": 25 }, "height": "", + "hiddenSeries": false, "id": 1, "legend": { "alignAsTable": true, @@ -9595,7 +11844,11 @@ "links": [], "maxPerRow": 2, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9700,7 +11953,12 @@ "description": "The count of pending commands per TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9709,6 +11967,7 @@ "y": 25 }, "height": "", + "hiddenSeries": false, "id": 193, "legend": { "alignAsTable": true, @@ -9729,7 +11988,11 @@ "links": [], "maxPerRow": 2, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9754,7 +12017,8 @@ "fill": true, "line": true, "op": "gt", - "value": 300 + "value": 300, + "visible": true } ], "timeFrom": null, @@ -9810,7 +12074,7 @@ "h": 1, "w": 24, "x": 0, - "y": 14 + "y": 15 }, "id": 2756, "panels": [ @@ -9822,7 +12086,12 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The total number of commands on each stage in commit command", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 10, "w": 24, @@ -9830,6 +12099,7 @@ "y": 16 }, "height": "400", + "hiddenSeries": false, "id": 168, "legend": { "alignAsTable": true, @@ -9850,7 +12120,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9934,7 +12208,12 @@ "description": "The time consumed when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9942,6 +12221,7 @@ "x": 0, "y": 26 }, + "hiddenSeries": false, "id": 3, "legend": { "alignAsTable": true, @@ -9963,7 +12243,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10059,7 +12343,12 @@ "description": "The time which is caused by latch wait in commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -10067,6 +12356,7 @@ "x": 12, "y": 26 }, + "hiddenSeries": false, "id": 194, "legend": { "alignAsTable": true, @@ -10088,7 +12378,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10184,7 +12478,12 @@ "description": "The count of keys read by a commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -10192,6 +12491,7 @@ "x": 0, "y": 34 }, + "hiddenSeries": false, "id": 195, "legend": { "alignAsTable": true, @@ -10213,7 +12513,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10309,7 +12613,12 @@ "description": "The count of keys written by a commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -10317,6 +12626,7 @@ "x": 12, "y": 34 }, + "hiddenSeries": false, "id": 373, "legend": { "alignAsTable": true, @@ -10338,7 +12648,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10434,6 +12748,10 @@ "description": "The keys scan details of each CF when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "grid": {}, "gridPos": { @@ -10464,6 +12782,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10541,6 +12860,10 @@ "description": "The keys scan details of lock CF when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "grid": {}, "gridPos": { @@ -10571,6 +12894,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10648,6 +12972,10 @@ "description": "The keys scan details of write CF when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "grid": {}, "gridPos": { @@ -10678,6 +13006,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10755,6 +13084,10 @@ "description": "The keys scan details of default CF when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "grid": {}, "gridPos": { @@ -10785,6 +13118,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10864,7 +13198,7 @@ "h": 1, "w": 24, "x": 0, - "y": 15 + "y": 16 }, "id": 2759, "panels": [ @@ -10877,14 +13211,20 @@ "description": "The rate of Raft snapshot messages sent", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 8, "x": 0, - "y": 16 + "y": 17 }, + "hiddenSeries": false, "id": 35, "legend": { "alignAsTable": true, @@ -10904,7 +13244,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10973,14 +13317,20 @@ "description": "The time consumed when handling snapshots", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 8, "x": 8, - "y": 16 + "y": 17 }, + "hiddenSeries": false, "id": 36, "legend": { "alignAsTable": true, @@ -11000,7 +13350,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11085,14 +13439,20 @@ "description": "The number of snapshots in different states", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 8, "x": 16, - "y": 16 + "y": 17 }, + "hiddenSeries": false, "id": 38, "legend": { "alignAsTable": true, @@ -11112,7 +13472,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11182,14 +13546,20 @@ "description": "The snapshot size (P99.99).9999", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 23 + "y": 24 }, + "hiddenSeries": false, "id": 44, "legend": { "alignAsTable": true, @@ -11209,7 +13579,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11279,14 +13653,20 @@ "description": "The number of KV within a snapshot in .9999", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 23 + "y": 24 }, + "hiddenSeries": false, "id": 43, "legend": { "alignAsTable": true, @@ -11306,7 +13686,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11379,7 +13763,7 @@ "h": 1, "w": 24, "x": 0, - "y": 16 + "y": 17 }, "id": 2760, "panels": [ @@ -11393,14 +13777,20 @@ "description": "The number of tasks handled by worker", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 18 }, + "hiddenSeries": false, "id": 59, "legend": { "alignAsTable": true, @@ -11422,7 +13812,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11492,14 +13886,20 @@ "description": " \tCurrent pending and running tasks of worker", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 18 }, + "hiddenSeries": false, "id": 1395, "legend": { "alignAsTable": true, @@ -11521,7 +13921,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11591,14 +13995,20 @@ "description": "The number of tasks handled by future_pool", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 29 + "y": 26 }, + "hiddenSeries": false, "id": 1876, "legend": { "alignAsTable": true, @@ -11620,7 +14030,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11690,14 +14104,20 @@ "description": "Current pending and running tasks of future_pool", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 29 + "y": 26 }, + "hiddenSeries": false, "id": 1877, "legend": { "alignAsTable": true, @@ -11719,7 +14139,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11791,7 +14215,7 @@ "h": 1, "w": 24, "x": 0, - "y": 17 + "y": 18 }, "id": 2761, "panels": [ @@ -11802,13 +14226,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 24 + "y": 19 }, + "hiddenSeries": false, "id": 2108, "legend": { "alignAsTable": true, @@ -11828,7 +14258,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": true, "renderer": "flot", @@ -11901,13 +14335,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 24 + "y": 19 }, + "hiddenSeries": false, "id": 2258, "legend": { "alignAsTable": true, @@ -11927,7 +14367,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": true, "renderer": "flot", @@ -11995,13 +14439,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 31 + "y": 26 }, + "hiddenSeries": false, "id": 2660, "legend": { "alignAsTable": true, @@ -12021,7 +14471,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": true, "renderer": "flot", @@ -12089,13 +14543,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 31 + "y": 26 }, + "hiddenSeries": false, "id": 2661, "legend": { "alignAsTable": true, @@ -12115,7 +14575,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": true, "renderer": "flot", @@ -12188,7 +14652,7 @@ "h": 1, "w": 24, "x": 0, - "y": 18 + "y": 19 }, "id": 2762, "panels": [ @@ -12200,13 +14664,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of get operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 25 + "y": 20 }, + "hiddenSeries": false, "id": 138, "legend": { "alignAsTable": true, @@ -12226,7 +14696,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12334,13 +14808,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing get operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 25 + "y": 20 }, + "hiddenSeries": false, "id": 82, "legend": { "alignAsTable": true, @@ -12360,7 +14840,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12458,13 +14942,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of seek operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 33 + "y": 28 }, + "hiddenSeries": false, "id": 129, "legend": { "alignAsTable": true, @@ -12484,7 +14974,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12604,13 +15098,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing seek operation", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 33 + "y": 28 }, + "hiddenSeries": false, "id": 125, "legend": { "alignAsTable": true, @@ -12630,7 +15130,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12728,14 +15232,146 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of write operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 36 + }, + "hiddenSeries": false, + "id": 139, + "legend": { + "alignAsTable": true, + "avg": false, + "current": 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.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "scopedVars": { + "db": { + "selected": false, + "text": "kv", + "value": "kv" + } + }, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=~\"write_done_by_self|write_done_by_other\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "done", + "refId": "A", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_timeout\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "timeout", + "refId": "B", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_with_wal\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "with_wal", + "refId": "C", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write operations", + "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": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The time consumed when executing write operation", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 41 + "x": 12, + "y": 36 }, - "id": 139, + "hiddenSeries": false, + "id": 126, "legend": { "alignAsTable": true, "avg": false, @@ -12754,7 +15390,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12771,35 +15411,43 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=~\"write_done_by_self|write_done_by_other\"}[1m]))", + "expr": "max(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_max\"})", "format": "time_series", "intervalFactor": 2, - "legendFormat": "done", + "legendFormat": "max", "refId": "A", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_timeout\"}[1m]))", + "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_percentile99\"})", "format": "time_series", "intervalFactor": 2, - "legendFormat": "timeout", + "legendFormat": "99%", "refId": "B", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_with_wal\"}[1m]))", + "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_percentile95\"})", "format": "time_series", "intervalFactor": 2, - "legendFormat": "with_wal", + "legendFormat": "95%", "refId": "C", "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_average\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "avg", + "refId": "D", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write operations", + "title": "Write duration", "tooltip": { "shared": true, "sort": 0, @@ -12815,9 +15463,9 @@ }, "yaxes": [ { - "format": "ops", + "format": "µs", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -12843,15 +15491,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The time consumed when executing write operation", - "fill": 0, + "description": " \tThe count of WAL sync operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 41 + "x": 0, + "y": 44 }, - "id": 126, + "id": 137, "legend": { "alignAsTable": true, "avg": false, @@ -12871,6 +15523,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12887,43 +15540,20 @@ "steppedLine": false, "targets": [ { - "expr": "max(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_max\"})", + "expr": "sum(rate(tiflash_proxy_tikv_engine_wal_file_synced{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m]))", "format": "time_series", "intervalFactor": 2, - "legendFormat": "max", + "legendFormat": "sync", + "metric": "", "refId": "A", "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_percentile99\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99%", - "refId": "B", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_percentile95\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "95%", - "refId": "C", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_average\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "avg", - "refId": "D", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write duration", + "title": "WAL sync operations", "tooltip": { "shared": true, "sort": 0, @@ -12939,9 +15569,9 @@ }, "yaxes": [ { - "format": "µs", + "format": "ops", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -12968,12 +15598,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing write wal operation", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 41 + "y": 44 }, "id": 130, "legend": { @@ -12995,6 +15629,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13091,15 +15726,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": " \tThe count of WAL sync operations", + "description": "The count of compaction and flush operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 49 + "y": 52 }, - "id": 137, + "id": 128, "legend": { "alignAsTable": true, "avg": false, @@ -13119,6 +15758,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13135,12 +15775,12 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_wal_file_synced{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_event_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "sync", - "metric": "", - "refId": "A", + "legendFormat": "{{type}}", + "metric": "tiflash_proxy_tikv_engine_event_total", + "refId": "B", "step": 10 } ], @@ -13148,7 +15788,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "WAL sync operations", + "title": "Compaction operations", "tooltip": { "shared": true, "sort": 0, @@ -13193,12 +15833,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing WAL sync operation", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 49 + "y": 52 }, "id": 135, "legend": { @@ -13221,6 +15865,7 @@ "maxPerRow": 2, "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13317,15 +15962,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The count of compaction and flush operations", + "description": "The time consumed when reading SST files", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 57 + "y": 60 }, - "id": 128, + "id": 140, "legend": { "alignAsTable": true, "avg": false, @@ -13345,6 +15994,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13361,20 +16011,47 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_event_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (type)", + "expr": "max(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_max\"})", "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{type}}", - "metric": "tiflash_proxy_tikv_engine_event_total", + "legendFormat": "max", + "metric": "", + "refId": "A", + "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_percentile99\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99%", + "metric": "", "refId": "B", "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_percentile95\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95%", + "metric": "", + "refId": "C", + "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_average\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "avg", + "metric": "", + "refId": "D", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compaction operations", + "title": "SST read duration", "tooltip": { "shared": true, "sort": 0, @@ -13390,9 +16067,9 @@ }, "yaxes": [ { - "format": "ops", + "format": "µs", "label": null, - "logBase": 1, + "logBase": 10, "max": null, "min": null, "show": true @@ -13419,12 +16096,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing the compaction and flush operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 57 + "y": 60 }, "id": 136, "legend": { @@ -13446,6 +16127,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13543,15 +16225,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The time consumed when reading SST files", + "description": "The block cache size. Broken down by column family if shared block cache is disabled.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 65 + "y": 68 }, - "id": 140, + "id": 102, "legend": { "alignAsTable": true, "avg": false, @@ -13571,6 +16257,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13587,47 +16274,19 @@ "steppedLine": false, "targets": [ { - "expr": "max(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_max\"})", + "expr": "topk(20, avg(tiflash_proxy_tikv_engine_block_cache_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by(cf, instance))", "format": "time_series", "intervalFactor": 2, - "legendFormat": "max", - "metric": "", + "legendFormat": "{{instance}}-{{cf}}", "refId": "A", "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_percentile99\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99%", - "metric": "", - "refId": "B", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_percentile95\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "95%", - "metric": "", - "refId": "C", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_average\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "avg", - "metric": "", - "refId": "D", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "SST read duration", + "title": "Block cache size", "tooltip": { "shared": true, "sort": 0, @@ -13643,9 +16302,9 @@ }, "yaxes": [ { - "format": "µs", + "format": "bytes", "label": null, - "logBase": 10, + "logBase": 1, "max": null, "min": null, "show": true @@ -13671,15 +16330,18 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The time which is caused by write stall", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 153 + "y": 68 }, - "id": 87, + "id": 2451, "legend": { "alignAsTable": true, "avg": false, @@ -13699,55 +16361,30 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", - "scopedVars": { - "db": { - "selected": false, - "text": "kv", - "value": "kv" - } - }, - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_max\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "max", - "metric": "", - "refId": "A", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_percentile99\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99%", - "metric": "", - "refId": "B", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_percentile95\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "95%", - "metric": "", - "refId": "C", - "step": 10 - }, + "scopedVars": { + "db": { + "selected": false, + "text": "kv", + "value": "kv" + } + }, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_average\"})", + "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (cf, reason)", "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "avg", + "legendFormat": "{{cf}} - {{reason}}", "metric": "", - "refId": "D", + "refId": "A", "step": 10 } ], @@ -13755,7 +16392,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write stall duration", + "title": "Compaction reason", "tooltip": { "shared": true, "sort": 0, @@ -13771,11 +16408,11 @@ }, "yaxes": [ { - "format": "µs", + "format": "short", "label": null, - "logBase": 10, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -13783,7 +16420,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -13799,15 +16436,20 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The memtable size of each column family", - "fill": 1, + "description": "The flow of different kinds of block cache operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 153 + "x": 0, + "y": 76 }, - "id": 103, + "height": "", + "id": 467, "legend": { "alignAsTable": true, "avg": false, @@ -13827,6 +16469,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13843,19 +16486,86 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_proxy_tikv_engine_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"mem-tables\"}) by (cf)", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_byte_read\"}[1m]))", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 2, - "legendFormat": "{{cf}}", + "legendFormat": "total_read", "refId": "A", "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_byte_write\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "total_written", + "refId": "C", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_data_bytes_insert\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "data_insert", + "metric": "", + "refId": "D", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_filter_bytes_insert\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "filter_insert", + "metric": "", + "refId": "B", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_filter_bytes_evict\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "filter_evict", + "metric": "", + "refId": "E", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_index_bytes_insert\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "index_insert", + "metric": "", + "refId": "F", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_index_bytes_evict\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "index_evict", + "metric": "", + "refId": "G", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memtable size", + "title": "Block cache flow", "tooltip": { "shared": true, "sort": 0, @@ -13871,19 +16581,19 @@ }, "yaxes": [ { - "format": "bytes", + "format": "Bps", "label": null, - "logBase": 1, + "logBase": 10, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -13900,12 +16610,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The hit rate of memtable", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 73 + "y": 76 }, "id": 88, "legend": { @@ -13927,6 +16641,7 @@ "links": [], "nullPointMode": "connected", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13999,15 +16714,20 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The block cache size. Broken down by column family if shared block cache is disabled.", - "fill": 1, + "description": "The flow of different kinds of operations on keys", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 81 + "y": 84 }, - "id": 102, + "height": "", + "id": 132, "legend": { "alignAsTable": true, "avg": false, @@ -14027,6 +16747,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14043,10 +16764,33 @@ "steppedLine": false, "targets": [ { - "expr": "topk(20, avg(tiflash_proxy_tikv_engine_block_cache_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by(cf, instance))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"keys_read\"}[1m]))", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 2, - "legendFormat": "{{instance}}-{{cf}}", + "legendFormat": "read", + "refId": "B", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"keys_written\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "written", + "refId": "C", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_num_corrupt_keys{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "corrupt", + "metric": "", "refId": "A", "step": 10 } @@ -14055,7 +16799,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Block cache size", + "title": "Keys flow", "tooltip": { "shared": true, "sort": 0, @@ -14071,11 +16815,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "ops", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -14083,7 +16827,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -14100,12 +16844,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The hit rate of block cache", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 81 + "y": 84 }, "id": 80, "legend": { @@ -14128,6 +16876,7 @@ "maxPerRow": 2, "nullPointMode": "connected", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14237,16 +16986,20 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow of different kinds of block cache operations", + "description": "The flow rate of read operations per type", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 89 + "y": 92 }, "height": "", - "id": 467, + "id": 85, "legend": { "alignAsTable": true, "avg": false, @@ -14266,6 +17019,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14282,86 +17036,31 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_byte_read\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_read\"}[1m]))", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "total_read", + "legendFormat": "get", "refId": "A", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_byte_write\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"iter_bytes_read\"}[1m]))", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "total_written", + "legendFormat": "scan", "refId": "C", "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_data_bytes_insert\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "data_insert", - "metric": "", - "refId": "D", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_filter_bytes_insert\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "filter_insert", - "metric": "", - "refId": "B", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_filter_bytes_evict\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "filter_evict", - "metric": "", - "refId": "E", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_index_bytes_insert\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "index_insert", - "metric": "", - "refId": "F", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_index_bytes_evict\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "index_evict", - "metric": "", - "refId": "G", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Block cache flow", + "title": "Read flow", "tooltip": { "shared": true, "sort": 0, @@ -14379,13 +17078,13 @@ { "format": "Bps", "label": null, - "logBase": 10, + "logBase": 1, "max": null, "min": "0", "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -14406,12 +17105,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of different kinds of block cache operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 89 + "y": 92 }, "id": 468, "legend": { @@ -14433,6 +17136,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14542,16 +17246,20 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow of different kinds of operations on keys", - "fill": 0, + "description": "The flow of different kinds of write operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 97 + "y": 100 }, "height": "", - "id": 132, + "id": 86, "legend": { "alignAsTable": true, "avg": false, @@ -14571,6 +17279,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14587,33 +17296,20 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"keys_read\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "read", - "refId": "B", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"keys_written\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"wal_file_bytes\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "written", + "legendFormat": "wal", "refId": "C", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_num_corrupt_keys{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_written\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "corrupt", - "metric": "", + "legendFormat": "write", "refId": "A", "step": 10 } @@ -14622,7 +17318,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Keys flow", + "title": "Write flow", "tooltip": { "shared": true, "sort": 0, @@ -14638,7 +17334,7 @@ }, "yaxes": [ { - "format": "ops", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -14667,12 +17363,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of keys in each column family", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 97 + "y": 100 }, "id": 131, "legend": { @@ -14694,6 +17394,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14768,16 +17469,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow rate of read operations per type", - "fill": 0, + "description": "The flow rate of compaction operations per type", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 105 + "y": 108 }, - "height": "", - "id": 85, + "id": 90, "legend": { "alignAsTable": true, "avg": false, @@ -14797,6 +17501,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14813,31 +17518,38 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_read\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_read\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "get", + "legendFormat": "read", "refId": "A", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"iter_bytes_read\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_written\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "scan", + "legendFormat": "written", "refId": "C", "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"flush_write_bytes\"}[1m]))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "flushed", + "refId": "B", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read flow", + "title": "Compaction flow", "tooltip": { "shared": true, "sort": 0, @@ -14861,7 +17573,7 @@ "show": true }, { - "format": "short", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -14882,12 +17594,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The bytes per read", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 105 + "y": 108 }, "id": 133, "legend": { @@ -14910,6 +17626,7 @@ "maxPerRow": 2, "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15006,16 +17723,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow of different kinds of write operations", + "description": "The read amplification per TiKV instance \t", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 113 + "y": 116 }, - "height": "", - "id": 86, + "id": 518, "legend": { "alignAsTable": true, "avg": false, @@ -15035,6 +17755,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15051,20 +17772,12 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"wal_file_bytes\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "wal", - "refId": "C", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_written\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_read_amp_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"read_amp_total_read_bytes\"}[1m])) by (instance) / sum(rate(tiflash_proxy_tikv_engine_read_amp_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", db=\"$db\", type=\"read_amp_estimate_useful_bytes\"}[1m])) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "write", + "legendFormat": "{{instance}}", + "metric": "", "refId": "A", "step": 10 } @@ -15073,7 +17786,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write flow", + "title": "Read amplication", "tooltip": { "shared": true, "sort": 0, @@ -15089,7 +17802,7 @@ }, "yaxes": [ { - "format": "Bps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -15118,12 +17831,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The bytes per write", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 113 + "y": 116 }, "id": 134, "legend": { @@ -15146,6 +17863,7 @@ "maxPerRow": 2, "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15242,15 +17960,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow rate of compaction operations per type", + "description": "The number of snapshot of each TiKV instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 121 + "y": 124 }, - "id": 90, + "id": 516, "legend": { "alignAsTable": true, "avg": false, @@ -15270,6 +17992,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15286,38 +18009,21 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_read\"}[1m]))", + "expr": "tiflash_proxy_tikv_engine_num_snapshots{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "read", + "legendFormat": "{{instance}}", + "metric": "", "refId": "A", "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_written\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "written", - "refId": "C", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"flush_write_bytes\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "flushed", - "refId": "B", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compaction flow", + "title": "Number of snapshots", "tooltip": { "shared": true, "sort": 0, @@ -15333,7 +18039,7 @@ }, "yaxes": [ { - "format": "Bps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -15341,7 +18047,7 @@ "show": true }, { - "format": "Bps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -15362,12 +18068,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The pending bytes to be compacted", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 121 + "y": 124 }, "id": 127, "legend": { @@ -15389,6 +18099,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15443,109 +18154,7 @@ "show": true }, { - "format": "Bps", - "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 read amplification per TiKV instance \t", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 129 - }, - "id": 518, - "legend": { - "alignAsTable": true, - "avg": false, - "current": 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", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "scopedVars": { - "db": { - "selected": false, - "text": "kv", - "value": "kv" - } - }, - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_read_amp_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"read_amp_total_read_bytes\"}[1m])) by (instance) / sum(rate(tiflash_proxy_tikv_engine_read_amp_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", db=\"$db\", type=\"read_amp_estimate_useful_bytes\"}[1m])) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "metric": "", - "refId": "A", - "step": 10 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Read amplication", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -15564,25 +18173,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The compression ratio of each level", + "description": "The number of SST files for different column families in each level", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 129 + "x": 0, + "y": 132 }, - "id": 863, + "id": 2002, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, - "min": false, + "min": true, "rightSide": true, "show": true, - "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -15593,6 +18204,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15609,21 +18221,18 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_proxy_tikv_engine_compression_ratio{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by (level)", + "expr": "avg(tiflash_proxy_tikv_engine_num_files_at_level{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by (cf, level)", "format": "time_series", - "hide": false, "intervalFactor": 2, - "legendFormat": "level - {{level}}", - "metric": "", - "refId": "A", - "step": 10 + "legendFormat": "cf-{{cf}}, level-{{level}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compression ratio", + "title": "Number files at each level", "tooltip": { "shared": true, "sort": 0, @@ -15643,7 +18252,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -15651,7 +18260,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -15667,15 +18276,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The number of snapshot of each TiKV instance", + "description": "The compression ratio of each level", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 137 + "x": 12, + "y": 132 }, - "id": 516, + "id": 863, "legend": { "alignAsTable": true, "avg": false, @@ -15695,6 +18308,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15711,11 +18325,11 @@ "steppedLine": false, "targets": [ { - "expr": "tiflash_proxy_tikv_engine_num_snapshots{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", + "expr": "avg(tiflash_proxy_tikv_engine_compression_ratio{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by (level)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}", + "legendFormat": "level - {{level}}", "metric": "", "refId": "A", "step": 10 @@ -15725,7 +18339,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Number of snapshots", + "title": "Compression ratio", "tooltip": { "shared": true, "sort": 0, @@ -15768,25 +18382,28 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The time that the oldest unreleased snapshot survivals", + "description": "Stall conditions changed of each column family", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 137 + "x": 0, + "y": 140 }, - "id": 517, + "id": 2381, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideZero": true, "max": true, - "min": false, + "min": true, "rightSide": true, "show": true, - "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -15797,6 +18414,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15813,21 +18431,18 @@ "steppedLine": false, "targets": [ { - "expr": "tiflash_proxy_tikv_engine_oldest_snapshot_duration{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", + "expr": "tiflash_proxy_tikv_engine_stall_conditions_changed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", "format": "time_series", - "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}", - "metric": "tiflash_proxy_tikv_engine_oldest_snapshot_duration", - "refId": "A", - "step": 10 + "legendFormat": "{{instance}}-{{cf}}-{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Oldest snapshots duration", + "title": "Stall conditions changed of each CF", "tooltip": { "shared": true, "sort": 0, @@ -15843,11 +18458,11 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -15855,7 +18470,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -15870,23 +18485,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of SST files for different column families in each level", + "decimals": 1, + "description": "The time that the oldest unreleased snapshot survivals", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 145 + "x": 12, + "y": 140 }, - "id": 2002, + "id": 517, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, - "min": true, + "min": false, "rightSide": true, "show": true, + "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -15897,6 +18518,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15913,18 +18535,21 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_proxy_tikv_engine_num_files_at_level{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by (cf, level)", + "expr": "tiflash_proxy_tikv_engine_oldest_snapshot_duration{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "cf-{{cf}}, level-{{level}}", - "refId": "A" + "legendFormat": "{{instance}}", + "metric": "tiflash_proxy_tikv_engine_oldest_snapshot_duration", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Number files at each level", + "title": "Oldest snapshots duration", "tooltip": { "shared": true, "sort": 0, @@ -15940,11 +18565,11 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -15952,7 +18577,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -15967,23 +18592,28 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The time consumed when ingesting SST files", + "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 145 + "x": 0, + "y": 148 }, - "id": 2003, + "id": 2452, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, - "min": true, + "min": false, "rightSide": true, "show": true, + "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -15994,6 +18624,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -16010,25 +18641,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99%", - "refId": "A" - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) / sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "expr": "sum(increase(tiflash_proxy_tikv_engine_write_stall_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (type)", "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "average", - "refId": "B" + "legendFormat": "{{type}}", + "metric": "", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Ingest SST duration seconds", + "title": "Write Stall Reason", "tooltip": { "shared": true, "sort": 0, @@ -16044,11 +18671,11 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -16056,7 +18683,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -16071,20 +18698,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Stall conditions changed of each column family", + "description": "The time consumed when ingesting SST files", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 153 + "x": 12, + "y": 148 }, - "id": 2381, + "id": 2003, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideZero": true, "max": true, "min": true, "rightSide": true, @@ -16099,6 +18729,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -16115,10 +18746,17 @@ "steppedLine": false, "targets": [ { - "expr": "tiflash_proxy_tikv_engine_stall_conditions_changed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", + "expr": "histogram_quantile(0.99, sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{instance}}-{{cf}}-{{type}}", + "legendFormat": "99%", + "refId": "A" + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) / sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "average", "refId": "B" } ], @@ -16126,7 +18764,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Stall conditions changed of each CF", + "title": "Ingest SST duration seconds", "tooltip": { "shared": true, "sort": 0, @@ -16142,7 +18780,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -16170,14 +18808,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "description": "The time which is caused by write stall", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 161 + "x": 12, + "y": 156 }, - "id": 2452, + "id": 87, "legend": { "alignAsTable": true, "avg": false, @@ -16197,6 +18840,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -16213,21 +18857,47 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tiflash_proxy_tikv_engine_write_stall_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (type)", + "expr": "max(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_max\"})", "format": "time_series", - "hide": false, "intervalFactor": 2, - "legendFormat": "{{type}}", + "legendFormat": "max", "metric": "", "refId": "A", "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_percentile99\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99%", + "metric": "", + "refId": "B", + "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_percentile95\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95%", + "metric": "", + "refId": "C", + "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_average\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "avg", + "metric": "", + "refId": "D", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Stall Reason", + "title": "Write stall duration", "tooltip": { "shared": true, "sort": 0, @@ -16243,11 +18913,11 @@ }, "yaxes": [ { - "format": "short", + "format": "µs", "label": null, - "logBase": 1, + "logBase": 10, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -16255,7 +18925,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -16271,14 +18941,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "description": "The memtable size of each column family", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 65 + "y": 164 }, - "id": 2451, + "id": 103, "legend": { "alignAsTable": true, "avg": false, @@ -16298,6 +18973,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -16314,12 +18990,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (cf, reason)", + "expr": "avg(tiflash_proxy_tikv_engine_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"mem-tables\"}) by (cf)", "format": "time_series", - "hide": false, "intervalFactor": 2, - "legendFormat": "{{cf}} - {{reason}}", - "metric": "", + "legendFormat": "{{cf}}", "refId": "A", "step": 10 } @@ -16328,7 +19002,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compaction reason", + "title": "Memtable size", "tooltip": { "shared": true, "sort": 0, @@ -16344,11 +19018,11 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -16356,7 +19030,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -16845,7 +19519,7 @@ } ], "refresh": "1m", - "schemaVersion": 18, + "schemaVersion": 27, "style": "dark", "tags": [], "templating": { @@ -16984,6 +19658,37 @@ "tagsQuery": "", "type": "query", "useTags": false + }, + { + "allValue": null, + "current": { + "selected": true, + "text": "none", + "value": "none" + }, + "description": null, + "error": null, + "hide": 0, + "includeAll": false, + "label": "additional_groupby", + "multi": false, + "name": "additional_groupby", + "options": [ + { + "selected": true, + "text": "none", + "value": "none" + }, + { + "selected": false, + "text": "instance", + "value": "instance" + } + ], + "query": "none,instance", + "queryValue": "", + "skipUrlSync": false, + "type": "custom" } ] }, From 28da97b1d908297ab4b8ca59bc50c4944c8f980e Mon Sep 17 00:00:00 2001 From: JaySon-Huang Date: Wed, 9 Jul 2025 12:22:44 +0800 Subject: [PATCH 6/6] Fix access to invalid mem Signed-off-by: JaySon-Huang --- dbms/src/Interpreters/InterpreterCreateQuery.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index d15e5887519..97bff2e4abe 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -68,7 +68,7 @@ class InterpreterCreateQuery : public IInterpreter ASTPtr query_ptr; Context & context; - std::string_view log_suffix; + std::string log_suffix; /// Using while loading database. ThreadPool * thread_pool = nullptr;