diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index c89497b8173..0007faa995c 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -60,6 +60,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 27009efd6a6..5be6b492c0c 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -318,6 +318,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 4c9282566bf..31ff839a2bb 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h @@ -76,19 +76,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: @@ -96,8 +83,14 @@ class ColumnFile UInt64 getId() const { return id; } virtual size_t getRows() const { return 0; } +<<<<<<< HEAD virtual size_t getBytes() const { return 0; }; virtual size_t getDeletes() const { return 0; }; +======= + virtual size_t getBytes() const { return 0; } + virtual size_t getAllocateBytes() const { return 0; } + virtual size_t getDeletes() const { return 0; } +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) virtual Type getType() const = 0; @@ -139,7 +132,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 1990e4aafff..1d9d99cc499 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); @@ -64,7 +65,13 @@ ColumnFileReaderPtr ColumnFileInMemory::getReader( return std::make_shared(*this, col_defs); } -bool ColumnFileInMemory::append( +void ColumnFileInMemory::disableAppend() +{ + disable_append = true; + // TODO: Call shrinkToFit() to release the extra memory of the cache block. +} + +ColumnFile::AppendResult ColumnFileInMemory::append( const DMContext & context, const Block & data, size_t offset, @@ -72,28 +79,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 81dd9a3365f..8f33f7fecf3 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; @@ -60,7 +74,12 @@ class ColumnFileInMemory : public ColumnFile Type getType() const override { return Type::INMEMORY_FILE; } size_t getRows() const override { return rows; } +<<<<<<< HEAD size_t getBytes() const override { return bytes; }; +======= + size_t getBytes() const override { return bytes; } + size_t getAllocateBytes() const override { return cache->block.allocatedBytes(); } +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) CachePtr getCache() { return cache; } @@ -75,9 +94,13 @@ class ColumnFileInMemory : public ColumnFile const ColumnDefinesPtr & col_defs) 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 42f1d253ec0..40ca207d072 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp @@ -247,8 +247,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 537795bcf95..03d0ffa761c 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -81,7 +81,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; @@ -202,6 +202,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 bd8cf9f3ced..244bbe117cb 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,15 +32,64 @@ class MemTableSet : public std::enable_shared_from_this , private boost::noncopyable { +<<<<<<< HEAD +private: +======= 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; + +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) // 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; @@ -50,18 +97,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. @@ -71,20 +107,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 /** @@ -149,5 +178,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 5932eb8ca10..33c4f832cb8 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -26,7 +26,15 @@ #include #include #include +<<<<<<< HEAD #include +======= +#include +#include +#include +#include +#include +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) #include #include #include @@ -74,6 +82,7 @@ namespace tests class DeltaMergeStoreTest; } +<<<<<<< HEAD inline static const PageIdU64 DELTA_MERGE_FIRST_SEGMENT_ID = 1; struct SegmentStats @@ -175,6 +184,31 @@ struct StoreStats }; class DeltaMergeStore : private boost::noncopyable +======= +struct LocalIndexStats +{ + UInt64 column_id{}; + UInt64 index_id{}; + String index_kind{}; + + UInt64 rows_stable_indexed{}; // Total rows + UInt64 rows_stable_not_indexed{}; // Total rows + UInt64 rows_delta_indexed{}; // Total rows + UInt64 rows_delta_not_indexed{}; // Total rows + + // If the index is finally failed to be built, then this is not empty + String error_message{}; +}; +using LocalIndexesStats = std::vector; + + +class DeltaMergeStore; +using DeltaMergeStorePtr = std::shared_ptr; + +class DeltaMergeStore + : private boost::noncopyable + , public std::enable_shared_from_this +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) { public: friend class ::DB::DM::tests::DeltaMergeStoreTest; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp index 411d09a9f22..ce2f64e2067 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -13,13 +13,16 @@ // limitations under the License. #include +<<<<<<< HEAD +======= +#include +#include +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) #include #include #include -namespace DB -{ -namespace DM +namespace DB::DM { StoreStats DeltaMergeStore::getStoreStats() @@ -31,11 +34,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) { @@ -64,6 +69,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(); } @@ -87,6 +93,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; @@ -153,8 +160,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(); @@ -164,16 +169,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(); @@ -193,5 +207,4 @@ SegmentsStats DeltaMergeStore::getSegmentsStats() } -} // 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 0dd77648403..144a8aa6dc5 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/In.h +++ b/dbms/src/Storages/DeltaMerge/Filter/In.h @@ -46,6 +46,20 @@ class In : public RSOperator return s + "]}"; }; + 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 89a7c95ddde..abbcea0c4c7 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 { RSResults results(pack_count, RSResult::Some); diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp new file mode 100644 index 00000000000..f0ae394824e --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.cpp @@ -0,0 +1,247 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::DM +{ +PushDownExecutorPtr PushDownExecutor::build( + const RSOperatorPtr & rs_operator, + const ANNQueryInfoPtr & ann_query_info, + const FTSQueryInfoPtr & fts_query_info, + const TiDB::ColumnInfos & table_scan_column_info, + const google::protobuf::RepeatedPtrField & pushed_down_filters, + const ColumnDefines & columns_to_read, + const ColumnRangePtr & column_range, + const Context & context, + const LoggerPtr & tracing_logger) +{ + // check if the ann_query_info is valid + if (ann_query_info) + { + RUNTIME_CHECK(ann_query_info->top_k() != std::numeric_limits::max()); + } + + if (pushed_down_filters.empty()) + { + LOG_DEBUG(tracing_logger, "Push down filter is empty"); + return std::make_shared(rs_operator, ann_query_info, fts_query_info, column_range); + } + std::unordered_map columns_to_read_map; + for (const auto & column : columns_to_read) + columns_to_read_map.emplace(column.id, column); + + // Get the columns of the filter, is a subset of columns_to_read + std::unordered_set filter_col_id_set; + for (const auto & expr : pushed_down_filters) + { + getColumnIDsFromExpr(expr, table_scan_column_info, filter_col_id_set); + } + auto filter_columns = std::make_shared(); + filter_columns->reserve(filter_col_id_set.size()); + for (const auto & cid : filter_col_id_set) + { + RUNTIME_CHECK_MSG( + columns_to_read_map.contains(cid), + "Filter ColumnID({}) not found in columns_to_read_map", + cid); + filter_columns->emplace_back(columns_to_read_map.at(cid)); + } + + // The source_columns_of_analyzer should be the same as the size of table_scan_column_info + // The columns_to_read is a subset of table_scan_column_info, when there are generated columns and extra table id column. + NamesAndTypes source_columns_of_analyzer; + source_columns_of_analyzer.reserve(table_scan_column_info.size()); + for (size_t i = 0; i < table_scan_column_info.size(); ++i) + { + auto const & ci = table_scan_column_info[i]; + const auto cid = ci.id; + if (ci.hasGeneratedColumnFlag()) + { + const auto & col_name = GeneratedColumnPlaceholderBlockInputStream::getColumnName(i); + const auto & data_type = getDataTypeByColumnInfoForComputingLayer(ci); + source_columns_of_analyzer.emplace_back(col_name, data_type); + continue; + } + if (cid == MutSup::extra_table_id_col_id) + { + source_columns_of_analyzer.emplace_back( + MutSup::extra_table_id_column_name, + MutSup::getExtraTableIdColumnType()); + continue; + } + RUNTIME_CHECK_MSG(columns_to_read_map.contains(cid), "ColumnID({}) not found in columns_to_read_map", cid); + source_columns_of_analyzer.emplace_back(columns_to_read_map.at(cid).name, columns_to_read_map.at(cid).type); + } + auto analyzer = std::make_unique(source_columns_of_analyzer, context); + + // Build the extra cast + ExpressionActionsPtr extra_cast = nullptr; + // need_cast_column should be the same size as table_scan_column_info and source_columns_of_analyzer + std::vector may_need_add_cast_column; + may_need_add_cast_column.reserve(table_scan_column_info.size()); + for (const auto & col : table_scan_column_info) + may_need_add_cast_column.push_back( + !col.hasGeneratedColumnFlag() && filter_col_id_set.contains(col.id) && col.id != -1); + ExpressionActionsChain chain; + auto & step = analyzer->initAndGetLastStep(chain); + auto & actions = step.actions; + if (auto [has_cast, casted_columns] + = analyzer->buildExtraCastsAfterTS(actions, may_need_add_cast_column, table_scan_column_info); + has_cast) + { + NamesWithAliases project_cols; + for (size_t i = 0; i < table_scan_column_info.size(); ++i) + { + if (filter_col_id_set.contains(table_scan_column_info[i].id)) + { + auto it = columns_to_read_map.find(table_scan_column_info[i].id); + RUNTIME_CHECK(it != columns_to_read_map.end(), table_scan_column_info[i].id); + project_cols.emplace_back(casted_columns[i], it->second.name); + } + } + actions->add(ExpressionAction::project(project_cols)); + + for (const auto & col : *filter_columns) + step.required_output.push_back(col.name); + + extra_cast = chain.getLastActions(); + chain.finalize(); + chain.clear(); + LOG_DEBUG(tracing_logger, "Extra cast for filter columns: {}", extra_cast->dumpActions()); + } + + // build filter expression actions + auto [before_where, filter_column_name, project_after_where] + = analyzer->buildPushDownFilter(pushed_down_filters, true); + LOG_DEBUG(tracing_logger, "Push down filter: {}", before_where->dumpActions()); + + // record current column defines + auto columns_after_cast = std::make_shared(); + if (extra_cast != nullptr) + { + columns_after_cast->reserve(columns_to_read.size()); + const auto & current_names_and_types = analyzer->getCurrentInputColumns(); + for (size_t i = 0; i < table_scan_column_info.size(); ++i) + { + if (table_scan_column_info[i].hasGeneratedColumnFlag() + || table_scan_column_info[i].id == MutSup::extra_table_id_col_id) + continue; + auto col = columns_to_read_map.at(table_scan_column_info[i].id); + RUNTIME_CHECK_MSG( + col.name == current_names_and_types[i].name, + "Column name mismatch, expect: {}, actual: {}", + col.name, + current_names_and_types[i].name); + columns_after_cast->push_back(col); + columns_after_cast->back().type = current_names_and_types[i].type; + } + } + + return std::make_shared( + rs_operator, + ann_query_info, + fts_query_info, + before_where, + project_after_where, + filter_columns, + filter_column_name, + extra_cast, + columns_after_cast, + column_range); +} + +PushDownExecutorPtr PushDownExecutor::build( + const SelectQueryInfo & query_info, + const ColumnDefines & columns_to_read, + const ColumnDefines & table_column_defines, + const google::protobuf::RepeatedPtrField & used_indexes, + const Context & context, + const LoggerPtr & tracing_logger) +{ + const auto & dag_query = query_info.dag_query; + if (unlikely(dag_query == nullptr)) + return EMPTY_FILTER; + + const auto & columns_to_read_info = dag_query->source_columns; + // build rough set operator + const auto rs_operator = RSOperator::build( + dag_query, + columns_to_read_info, + table_column_defines, + context.getSettingsRef().dt_enable_rough_set_filter, + tracing_logger); + // build column_range + const auto column_range = rs_operator && !used_indexes.empty() ? rs_operator->buildSets(used_indexes) : nullptr; + // build ann_query_info + ANNQueryInfoPtr ann_query_info = nullptr; + if (dag_query->ann_query_info.query_type() != tipb::ANNQueryType::InvalidQueryType) + ann_query_info = std::make_shared(dag_query->ann_query_info); + FTSQueryInfoPtr fts_query_info = nullptr; + if (dag_query->fts_query_info.query_type() != tipb::FTSQueryType::FTSQueryTypeInvalid) + fts_query_info = std::make_shared(dag_query->fts_query_info); + // build push down filter + const auto & pushed_down_filters = dag_query->pushed_down_filters; + if (unlikely(context.getSettingsRef().force_push_down_all_filters_to_scan) && !dag_query->filters.empty()) + { + google::protobuf::RepeatedPtrField merged_filters{ + pushed_down_filters.begin(), + pushed_down_filters.end()}; + merged_filters.MergeFrom(dag_query->filters); + return PushDownExecutor::build( + rs_operator, + ann_query_info, + fts_query_info, + columns_to_read_info, + merged_filters, + columns_to_read, + column_range, + context, + tracing_logger); + } + return PushDownExecutor::build( + rs_operator, + ann_query_info, + fts_query_info, + columns_to_read_info, + pushed_down_filters, + columns_to_read, + column_range, + 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/PushDownExecutor.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h new file mode 100644 index 00000000000..5beeb0ab0f2 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownExecutor.h @@ -0,0 +1,132 @@ +// 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 + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + +#include +#include +#include +#include +#include + +namespace DB +{ +struct SelectQueryInfo; +} + +namespace DB::DM +{ + +class PushDownExecutor; +using PushDownExecutorPtr = std::shared_ptr; +inline static const PushDownExecutorPtr EMPTY_FILTER{}; + +class PushDownExecutor +{ +public: + PushDownExecutor( + const RSOperatorPtr & rs_operator_, + const ANNQueryInfoPtr & ann_query_info_, + const FTSQueryInfoPtr & fts_query_info_, + const ExpressionActionsPtr & before_where_, + const ExpressionActionsPtr & project_after_where_, + const ColumnDefinesPtr & filter_columns_, + const String filter_column_name_, + const ExpressionActionsPtr & extra_cast_, + const ColumnDefinesPtr & columns_after_cast_, + const ColumnRangePtr & column_range_) + : rs_operator(rs_operator_) + , before_where(before_where_) + , project_after_where(project_after_where_) + , filter_column_name(std::move(filter_column_name_)) + , filter_columns(filter_columns_) + , extra_cast(extra_cast_) + , columns_after_cast(columns_after_cast_) + , ann_query_info(ann_query_info_) + , fts_query_info(fts_query_info_) + , column_range(column_range_) + {} + + explicit PushDownExecutor( + const RSOperatorPtr & rs_operator_, + const ANNQueryInfoPtr & ann_query_info_ = nullptr, + const FTSQueryInfoPtr & fts_query_info_ = nullptr, + const ColumnRangePtr & column_range_ = nullptr) + : rs_operator(rs_operator_) + , ann_query_info(ann_query_info_) + , fts_query_info(fts_query_info_) + , column_range(column_range_) + {} + + explicit PushDownExecutor(const ANNQueryInfoPtr & ann_query_info_) + : ann_query_info(ann_query_info_) + {} + + explicit PushDownExecutor(const FTSQueryInfoPtr & fts_query_info_) + : fts_query_info(fts_query_info_) + {} + + Poco::JSON::Object::Ptr toJSONObject() const; + + // Use by StorageDisaggregated. + static PushDownExecutorPtr build( + const DM::RSOperatorPtr & rs_operator, + const ANNQueryInfoPtr & ann_query_info, + const FTSQueryInfoPtr & fts_query_info, + const TiDB::ColumnInfos & table_scan_column_info, + const google::protobuf::RepeatedPtrField & pushed_down_filters, + const ColumnDefines & columns_to_read, + const ColumnRangePtr & column_range, + const Context & context, + const LoggerPtr & tracing_logger); + + // Use by StorageDeltaMerge. + static DM::PushDownExecutorPtr build( + const SelectQueryInfo & query_info, + const ColumnDefines & columns_to_read, + const ColumnDefines & table_column_defines, + const google::protobuf::RepeatedPtrField & used_indexes, + const Context & context, + const LoggerPtr & tracing_logger); + + // Rough set operator + RSOperatorPtr rs_operator; + // Filter expression actions and the name of the tmp filter column + // Used construct the FilterBlockInputStream + const ExpressionActionsPtr before_where; + // The projection after the filter, used to remove the tmp filter column + // Used to construct the ExpressionBlockInputStream + // 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 + const ColumnDefinesPtr filter_columns; + // The expression actions used to cast the timestamp/datetime column + const ExpressionActionsPtr extra_cast; + // If the extra_cast is not null, the types of the columns may be changed + const ColumnDefinesPtr columns_after_cast; + // The ann_query_info contains the information of the ANN index + const ANNQueryInfoPtr ann_query_info; + // The FTSQueryInfo contains the information of the FTS index + const FTSQueryInfoPtr fts_query_info; + // The column_range contains the column values of the pushed down filters + const ColumnRangePtr column_range; +}; + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h b/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h index 14387e00d26..455a6eb7ad9 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 : public std::enable_shared_from_this 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; @@ -81,6 +87,14 @@ class ColCmpVal : public RSOperator R"(","col":")" + attr.col_name + // R"(","value":")" + 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; + } }; @@ -110,6 +124,18 @@ class LogicalOp : public RSOperator s.pop_back(); return s + "]}"; } + 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; + } }; #define GET_RSINDEX_FROM_PARAM_NOT_FOUND_RETURN_DIRECTLY(param, attr, rsindex, res) \ diff --git a/dbms/src/Storages/DeltaMerge/Filter/Unsupported.h b/dbms/src/Storages/DeltaMerge/Filter/Unsupported.h index f132c84534c..72e5067ec30 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/Unsupported.h +++ b/dbms/src/Storages/DeltaMerge/Filter/Unsupported.h @@ -47,6 +47,14 @@ class Unsupported : public RSOperator R"(","is_not":")" + DB::toString(is_not) + "\"}"; } + 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, Some); diff --git a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp index 611703fd77d..84fc6576ab1 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp @@ -303,7 +303,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 8dae573e5d8..58d778c36fe 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 @@ -155,6 +156,30 @@ String ScanContext::toJson() const }; json->set("region_num_of_instance", to_json_array(region_num_of_instance)); +<<<<<<< HEAD +======= + if (vector_idx_load_from_cache.load() // + + vector_idx_load_from_disk.load() // + + vector_idx_load_from_s3.load() + > 0) + { + Poco::JSON::Object::Ptr vec_idx = new Poco::JSON::Object(); + vec_idx->set("tot_load", vector_idx_load_time_ms.load()); + vec_idx->set("load_s3", vector_idx_load_from_s3.load()); + vec_idx->set("load_disk", vector_idx_load_from_disk.load()); + vec_idx->set("load_cache", vector_idx_load_from_cache.load()); + vec_idx->set("tot_search", vector_idx_search_time_ms.load()); + vec_idx->set("read_vec", vector_idx_read_vec_time_ms.load()); + vec_idx->set("read_others", vector_idx_read_others_time_ms.load()); + json->set("vector_idx", vec_idx); + } + + if (pushdown_executor) + { + json->set("pushdown", pushdown_executor->toJSONObject()); + } + +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) 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 78321dd8737..6a0e951cebf 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`. @@ -62,7 +64,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 @@ -84,7 +85,58 @@ class ScanContext // Building bitmap std::atomic build_bitmap_time_ns{0}; +<<<<<<< HEAD +======= + std::atomic vector_idx_load_from_s3{0}; + std::atomic vector_idx_load_from_disk{0}; + std::atomic vector_idx_load_from_cache{0}; + std::atomic vector_idx_load_time_ms{0}; + std::atomic vector_idx_search_time_ms{0}; + std::atomic vector_idx_search_visited_nodes{0}; + std::atomic vector_idx_search_discarded_nodes{0}; + std::atomic vector_idx_read_vec_time_ms{0}; + std::atomic vector_idx_read_others_time_ms{0}; + + 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 89937f04794..6150081788b 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -107,6 +107,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 @@ -252,7 +253,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 87618701904..0d347efea00 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -732,6 +732,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 17c06451145..8033dd20922 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp @@ -551,6 +551,7 @@ try CATCH +<<<<<<< HEAD TEST_F(SegmentOperationTest, DeltaIndexError) try { @@ -656,6 +657,8 @@ try } CATCH +======= +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) 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 75c64b40368..b3491007979 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -48,6 +48,7 @@ #include #include #include +#include #include #include #include @@ -61,7 +62,14 @@ #include #include +<<<<<<< HEAD #include +======= +namespace CurrentMetrics +{ +extern const Metric DT_NumStorageDeltaMerge; +} // namespace CurrentMetrics +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) namespace DB { @@ -88,7 +96,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()) @@ -970,13 +982,25 @@ BlockInputStreams StorageDeltaMerge::read( RUNTIME_CHECK(query_info.mvcc_query_info != nullptr); const auto & mvcc_query_info = *query_info.mvcc_query_info; +<<<<<<< HEAD auto ranges = parseMvccQueryInfo(mvcc_query_info, num_streams, context, query_info.req_id, 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 filter = parsePushDownFilter(query_info, columns_to_read, context, tracing_logger); auto runtime_filter_list = parseRuntimeFilterList(query_info, context); const auto & scan_context = mvcc_query_info.scan_context; + scan_context->pushdown_executor = pushdown_executor; auto streams = store->read( context, @@ -984,8 +1008,13 @@ BlockInputStreams StorageDeltaMerge::read( columns_to_read, ranges, num_streams, +<<<<<<< HEAD /*max_version=*/mvcc_query_info.read_tso, filter, +======= + /*start_ts=*/mvcc_query_info.start_ts, + pushdown_executor, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) runtime_filter_list, query_info.dag_query == nullptr ? 0 : query_info.dag_query->rf_max_wait_time_ms, query_info.req_id, @@ -1064,13 +1093,25 @@ void StorageDeltaMerge::read( RUNTIME_CHECK(query_info.mvcc_query_info != nullptr); const auto & mvcc_query_info = *query_info.mvcc_query_info; +<<<<<<< HEAD auto ranges = parseMvccQueryInfo(mvcc_query_info, num_streams, context, query_info.req_id, 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 filter = parsePushDownFilter(query_info, columns_to_read, context, tracing_logger); auto runtime_filter_list = parseRuntimeFilterList(query_info, context); const auto & scan_context = mvcc_query_info.scan_context; + scan_context->pushdown_executor = pushdown_executor; store->read( exec_context_, @@ -1080,8 +1121,13 @@ void StorageDeltaMerge::read( columns_to_read, ranges, num_streams, +<<<<<<< HEAD /*max_version=*/mvcc_query_info.read_tso, filter, +======= + /*start_ts=*/mvcc_query_info.start_ts, + pushdown_executor, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) 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 dcb2d9e08e5..297cb64bf67 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -283,6 +283,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. @@ -313,9 +316,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 c10165141c2..cceddf90b5c 100644 --- a/dbms/src/Storages/System/StorageSystemDTSegments.cpp +++ b/dbms/src/Storages/System/StorageSystemDTSegments.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -57,6 +58,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()}, @@ -140,6 +142,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 e4dbdbc2f64..df4647f1519 100644 --- a/dbms/src/Storages/System/StorageSystemDTTables.cpp +++ b/dbms/src/Storages/System/StorageSystemDTTables.cpp @@ -40,6 +40,7 @@ StorageSystemDTTables::StorageSystemDTTables(const std::string & name_) {"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()}, @@ -51,6 +52,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()}, @@ -154,6 +156,7 @@ BlockInputStreams StorageSystemDTTables::read( res_columns[j++]->insert(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); @@ -165,6 +168,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 5e4041cb754..b303a89954a 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 d27b54cad73..594de84bcdd 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 08fd86163da..b0e48cc3089 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -79,6 +79,7 @@ #include #include #include +#include #include #include #include @@ -672,21 +673,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. @@ -761,7 +747,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 e11c62ac3fb..9b5edc37cef 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -2759,11 +2759,13 @@ "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 44 +<<<<<<< HEAD +======= + "x": 0, + "y": 51 }, "hiddenSeries": false, - "id": 268, + "id": 295, "legend": { "alignAsTable": true, "avg": false, @@ -2806,31 +2808,43 @@ "targets": [ { "exemplar": true, - "expr": "sum by (instance) (rate(tiflash_proxy_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"SegmentReader.*\"}[1m]))", + "expr": "sum by (instance) (rate(tiflash_proxy_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"LocalIndexPool*\"}[1m]))", "format": "time_series", "hide": false, "instant": false, "interval": "", "intervalFactor": 2, - "legendFormat": "{{name}} {{instance}}", + "legendFormat": "pool-{{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, + "expr": "count by (instance) (tiflash_proxy_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"LocalIndexPool*\"})", + "hide": true, "interval": "", "intervalFactor": 2, "legendFormat": "Limit", "refId": "B" + }, + { + "exemplar": true, + "expr": "sum by (instance) (rate(tiflash_proxy_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"LocalIndexSched*\"}[1m]))", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "sched-{{instance}}", + "refId": "C", + "step": 40 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Segment Reader", + "title": "Local Index Pool", "tooltip": { "msResolution": false, "shared": true, @@ -2868,42 +2882,33 @@ "align": false, "alignLevel": null } - } - ], - "title": "Threads CPU", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 2 - }, - "id": 6, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": null, + "description": "", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 3 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "x": 12, + "y": 44 }, "hiddenSeries": false, - "id": 9, + "id": 268, "legend": { "alignAsTable": true, "avg": false, @@ -2912,13 +2917,16 @@ "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 }, @@ -2927,25 +2935,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": 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 by (instance) (rate(tiflash_proxy_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"SegmentReader.*\"}[1m]))", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 2, + "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": "Request QPS", + "title": "Segment Reader", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -2960,8 +2992,8 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", + "decimals": 1, + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -2969,39 +3001,55 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { "align": false, "alignLevel": null } - }, + } + ], + "title": "Threads CPU", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 2 + }, + "id": 327, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, + "x": 0, "y": 3 }, "hiddenSeries": false, - "id": 2, + "id": 329, "legend": { "alignAsTable": true, "avg": false, @@ -3010,20 +3058,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, @@ -3031,18 +3082,30 @@ "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(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": "Executor QPS", + "title": "Threads state", "tooltip": { "shared": true, "sort": 0, @@ -3062,11 +3125,11 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3085,6 +3148,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] @@ -3094,33 +3158,36 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 10 + "x": 12, + "y": 3 }, "hiddenSeries": false, - "id": 11, + "id": 331, "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, @@ -3129,43 +3196,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_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": "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": "{{name}}-{{io}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request Duration", + "title": "Threads IO", "tooltip": { "shared": true, "sort": 0, @@ -3181,11 +3226,11 @@ }, "yaxes": [ { - "format": "s", + "format": "Bps", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -3208,20 +3253,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, + "x": 0, "y": 10 }, "hiddenSeries": false, - "id": 12, + "id": 333, "legend": { "alignAsTable": true, "avg": false, @@ -3230,20 +3276,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, @@ -3251,18 +3300,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_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": "{{reason}}", - "refId": "A" + "legendFormat": "{{instance}} - {{name}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Error QPS", + "title": "Thread Voluntary Context Switches", "tooltip": { "shared": true, "sort": 0, @@ -3282,11 +3335,11 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3305,6 +3358,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] @@ -3314,33 +3368,36 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 17 + "x": 12, + "y": 10 }, "hiddenSeries": false, - "id": 13, + "id": 335, "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, @@ -3348,39 +3405,22 @@ "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))", - "format": "time_series", - "intervalFactor": 1, - "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))", + "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": "80-{{type}}", - "refId": "D" + "legendFormat": "{{instance}} - {{name}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request Handle Duration", + "title": "Thread Nonvoluntary Context Switches", "tooltip": { "shared": true, "sort": 0, @@ -3396,11 +3436,11 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -3416,7 +3456,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, @@ -3432,11 +3487,11 @@ "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 17 + "x": 0, + "y": 4 }, "hiddenSeries": false, - "id": 14, + "id": 9, "legend": { "alignAsTable": true, "avg": false, @@ -3466,10 +3521,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_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "interval": "", "intervalFactor": 1, "legendFormat": "{{type}}", "refId": "A" @@ -3479,7 +3532,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Response Bytes/Seconds", + "title": "Request QPS", "tooltip": { "shared": true, "sort": 0, @@ -3495,7 +3548,8 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": null, + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3503,7 +3557,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3526,24 +3580,26 @@ "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 24 + "x": 12, + "y": 4 }, "hiddenSeries": false, - "id": 63, + "id": 2, "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, @@ -3554,7 +3610,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3563,39 +3619,18 @@ "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))", + "expr": "sum(rate(tiflash_coprocessor_executor_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_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": "Cop task memory usage", + "title": "Executor QPS", "tooltip": { "shared": true, "sort": 0, @@ -3611,7 +3646,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3619,7 +3654,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3642,26 +3677,26 @@ "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 24 + "x": 0, + "y": 11 }, "hiddenSeries": false, - "id": 165, + "id": 11, "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, @@ -3682,19 +3717,43 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_exchange_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "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": "{{type}}", + "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" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Exchange Bytes/Seconds", + "title": "Request Duration", "tooltip": { "shared": true, "sort": 0, @@ -3710,7 +3769,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -3746,11 +3805,11 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 31 + "x": 12, + "y": 11 }, "hiddenSeries": false, - "id": 100, + "id": 12, "legend": { "alignAsTable": true, "avg": false, @@ -3780,12 +3839,10 @@ "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": "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": "{{instance}}-{{type}}", + "legendFormat": "{{reason}}", "refId": "A" } ], @@ -3793,7 +3850,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Threads of Rpc", + "title": "Error QPS", "tooltip": { "shared": true, "sort": 0, @@ -3817,7 +3874,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3840,26 +3897,26 @@ "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 31 + "x": 0, + "y": 18 }, "hiddenSeries": false, - "id": 77, + "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, @@ -3879,18 +3936,39 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tiflash_coprocessor_handling_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (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", "intervalFactor": 1, - "legendFormat": "{{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": "Handling Request Number", + "title": "Request Handle Duration", "tooltip": { "shared": true, "sort": 0, @@ -3906,8 +3984,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -3915,7 +3992,7 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3943,11 +4020,11 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 38 + "x": 12, + "y": 18 }, "hiddenSeries": false, - "id": 102, + "id": 14, "legend": { "alignAsTable": true, "avg": false, @@ -3978,11 +4055,11 @@ "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(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": "{{instance}}-{{type}}", + "legendFormat": "{{type}}", "refId": "A" } ], @@ -3990,7 +4067,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Threads", + "title": "Response Bytes/Seconds", "tooltip": { "shared": true, "sort": 0, @@ -4006,7 +4083,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4037,26 +4114,24 @@ "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 38 + "x": 0, + "y": 25 }, "hiddenSeries": false, - "id": 101, + "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, @@ -4067,7 +4142,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4076,20 +4151,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_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": "Max Threads of Rpc", + "title": "Cop task memory usage", "tooltip": { "shared": true, "sort": 0, @@ -4105,7 +4199,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4132,7 +4226,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The MPP query count in TiFlash", "fieldConfig": { "defaults": {}, "overrides": [] @@ -4142,11 +4235,11 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 45 + "x": 12, + "y": 25 }, "hiddenSeries": false, - "id": 157, + "id": 165, "legend": { "alignAsTable": true, "avg": false, @@ -4177,11 +4270,11 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_mpp_task_manager{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) 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" } ], @@ -4189,7 +4282,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MPP Query count", + "title": "Exchange Bytes/Seconds", "tooltip": { "shared": true, "sort": 0, @@ -4205,7 +4298,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4241,11 +4334,11 @@ "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 45 + "x": 0, + "y": 32 }, "hiddenSeries": false, - "id": 103, + "id": 100, "legend": { "alignAsTable": true, "avg": false, @@ -4276,7 +4369,7 @@ "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": "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, @@ -4288,7 +4381,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Max Threads", + "title": "Threads of Rpc", "tooltip": { "shared": true, "sort": 0, @@ -4340,11 +4433,11 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 52 + "x": 12, + "y": 32 }, "hiddenSeries": false, - "id": 199, + "id": 77, "legend": { "alignAsTable": true, "avg": false, @@ -4374,12 +4467,10 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "tiflash_mpp_task_monitor{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "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}}", + "legendFormat": "{{type}}", "refId": "A" } ], @@ -4387,7 +4478,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Time of the Longest Live MPP Task", + "title": "Handling Request Number", "tooltip": { "shared": true, "sort": 0, @@ -4403,7 +4494,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": null, + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4411,7 +4503,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4439,11 +4531,11 @@ "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 52 + "x": 0, + "y": 39 }, "hiddenSeries": false, - "id": 166, + "id": 102, "legend": { "alignAsTable": true, "avg": false, @@ -4474,11 +4566,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!~\".*max\", type!~\"rpc.*\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4486,7 +4578,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Data size in send and receive queue", + "title": "Threads", "tooltip": { "shared": true, "sort": 0, @@ -4502,7 +4594,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4522,66 +4614,49 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Coprocessor", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 3 - }, - "id": 105, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the min_tso of each instance", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 0, - "y": 4 + "x": 12, + "y": 39 }, "hiddenSeries": false, - "id": 107, + "id": 101, "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, @@ -4590,10 +4665,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_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type=~\".*max\"}) by (instance, type)", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4601,7 +4677,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Min TSO", + "title": "Max Threads of Rpc", "tooltip": { "shared": true, "sort": 0, @@ -4617,13 +4693,12 @@ }, "yaxes": [ { - "decimals": null, "format": "none", - "label": "TSO", + "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true }, { "format": "short", @@ -4645,29 +4720,26 @@ "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.", + "description": "The MPP query count in TiFlash", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 4 + "x": 0, + "y": 46 }, "hiddenSeries": false, - "id": 109, + "id": 157, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -4676,13 +4748,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": [], @@ -4692,54 +4765,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)", - "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, + "expr": "max(tiflash_mpp_task_manager{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance, type)", + "format": "time_series", "interval": "", + "intervalFactor": 1, "legendFormat": "{{instance}}-{{type}}", - "queryType": "randomWalk", - "refId": "E" + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Estimated Thread Usage and Limit", + "title": "MPP Query count", "tooltip": { "shared": true, "sort": 0, @@ -4755,12 +4793,11 @@ }, "yaxes": [ { - "decimals": null, "format": "none", - "label": "Threads", - "logBase": 10, + "label": null, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -4783,29 +4820,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 + "x": 12, + "y": 46 }, "hiddenSeries": false, - "id": 111, + "id": 103, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -4814,13 +4847,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": [], @@ -4830,27 +4864,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": "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=\"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": "Max Threads", "tooltip": { "shared": true, "sort": 0, @@ -4866,12 +4892,11 @@ }, "yaxes": [ { - "decimals": null, "format": "none", - "label": "Queries", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -4894,29 +4919,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 + "x": 0, + "y": 53 }, "hiddenSeries": false, - "id": 113, + "id": 199, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -4925,13 +4946,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": [], @@ -4941,27 +4963,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_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_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": "Time of the Longest Live MPP Task", "tooltip": { "shared": true, "sort": 0, @@ -4977,12 +4991,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "Tasks", + "format": "s", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5005,29 +5018,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 + "x": 12, + "y": 53 }, "hiddenSeries": false, - "id": 117, + "id": 166, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5036,13 +5045,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": [], @@ -5052,19 +5062,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": "tiflash_exchange_queueing_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "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": "Data size in send and receive queue", "tooltip": { "shared": true, "sort": 0, @@ -5080,12 +5090,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "", + "format": "bytes", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5101,6 +5110,8 @@ "align": false, "alignLevel": null } +<<<<<<< HEAD +======= }, { "aliasColors": {}, @@ -5108,8 +5119,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the time of waiting for schedule", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5117,18 +5126,18 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 20 + "x": 0, + "y": 60 }, "hiddenSeries": false, - "id": 115, + "id": 297, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5137,13 +5146,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": [], @@ -5153,37 +5163,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": "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}}-80", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{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": "Network Transmission", "tooltip": { "shared": true, "sort": 0, @@ -5199,12 +5191,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "s", - "label": "Time", + "format": "bytes", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5220,30 +5211,14 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Task Scheduler", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 4 - }, - "id": 16, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Executed DDL jobs per minute", + "description": "The establish calldata details", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5253,21 +5228,21 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 5 + "x": 12, + "y": 60 }, "hiddenSeries": false, - "id": 19, + "id": 299, "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, @@ -5287,41 +5262,20 @@ "steppedLine": false, "targets": [ { - "expr": "avg(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "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": "{{type}}", + "legendFormat": "{{instance}}-{{type}}", "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": "Establish calldata details", "tooltip": { "shared": true, "sort": 0, @@ -5337,8 +5291,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "opm", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -5346,7 +5299,7 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5358,61 +5311,79 @@ "align": false, "alignLevel": null } - }, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + } + ], + "repeat": null, + "title": "Coprocessor", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 105, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Executed DDL apply 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": 12, + "x": 0, "y": 5 }, "hiddenSeries": false, - "id": 18, + "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, - "renderer": "flot", + "pointradius": 1, + "points": true, + "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, "stack": false, "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=\"min_tso\"}) by (instance, resource_group)", + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "A" } ], @@ -5420,7 +5391,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Apply OPM", + "title": "Min TSO", "tooltip": { "shared": true, "sort": 0, @@ -5437,15 +5408,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, @@ -5464,89 +5435,93 @@ "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": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 0, - "y": 12 + "x": 12, + "y": 5 }, "hiddenSeries": false, - "id": 20, + "id": 109, "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=\"thread_soft_limit\"}) 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=\"estimated_thread_usage\"}) 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}}", + "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" }, { - "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}}", + "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" }, { - "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}}", + "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" } ], @@ -5554,7 +5529,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Apply Duration", + "title": "Estimated Thread Usage and Limit", "tooltip": { "shared": true, "sort": 0, @@ -5570,19 +5545,20 @@ }, "yaxes": [ { - "format": "s", - "label": null, - "logBase": 1, + "decimals": null, + "format": "none", + "label": "Threads", + "logBase": 10, "max": null, - "min": "0", + "min": null, "show": true }, { "format": "short", "label": null, "logBase": 1, - "max": "2", - "min": "0", + "max": null, + "min": null, "show": true } ], @@ -5590,32 +5566,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 queries", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, @@ -5624,57 +5587,52 @@ "h": 8, "w": 12, "x": 0, - "y": 6 + "y": 13 }, "hiddenSeries": false, - "id": 41, + "id": 111, "legend": { "alignAsTable": false, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "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_queries_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_queries_count\"}) by (instance, type, resource_group)", "hide": false, - "intervalFactor": 1, - "legendFormat": "write block", + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "B" } ], @@ -5682,7 +5640,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Command OPS", + "title": "Active and Waiting Queries Count", "tooltip": { "shared": true, "sort": 0, @@ -5699,19 +5657,19 @@ "yaxes": [ { "decimals": null, - "format": "ops", - "label": null, + "format": "none", + "label": "Queries", "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 } ], @@ -5723,12 +5681,15 @@ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": null, + "description": "the count of active/ waiting tasks", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, @@ -5737,12 +5698,12 @@ "h": 8, "w": 12, "x": 12, - "y": 6 + "y": 13 }, "hiddenSeries": false, - "id": 38, + "id": 113, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": false, @@ -5754,98 +5715,48 @@ }, "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\"} +\ntiflash_system_profile_event_WriteBufferAIOWriteBytes{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, + "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": "", - "intervalFactor": 1, - "legendFormat": "amp-total-{{instance}}", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "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]) +\nrate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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=\"active_tasks_count\"}) by (instance, type, resource_group)", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "amp-5min-{{instance}}", + "legendFormat": "{{instance}}-{{type}}-{{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]) +\nrate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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]) +\nrate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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]) +\nrate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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": "Active and Waiting Tasks Count", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, - "transformations": [], "type": "graph", "xaxis": { "buckets": null, @@ -5857,15 +5768,15 @@ "yaxes": [ { "decimals": null, - "format": "short", - "label": null, + "format": "none", + "label": "Tasks", "logBase": 1, - "max": "20", - "min": "0", + "max": null, + "min": null, "show": true }, { - "format": "binBps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5884,26 +5795,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage engine read tasks", + "decimals": null, + "description": "the usage of estimated threads exceeded the hard limit where errors occur.", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, - "w": 24, + "w": 12, "x": 0, - "y": 14 + "y": 21 }, "hiddenSeries": false, - "id": 40, + "id": 117, "legend": { - "alignAsTable": true, - "avg": true, + "alignAsTable": false, + "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -5912,14 +5826,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": [], @@ -5928,19 +5841,20 @@ "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" + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"hard_limit_exceeded_count\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}", + "queryType": "randomWalk", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Tasks OPS", + "title": "Hard Limit Exceeded Count", "tooltip": { "shared": true, "sort": 0, @@ -5957,15 +5871,15 @@ "yaxes": [ { "decimals": null, - "format": "ops", + "format": "none", "label": "", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5984,8 +5898,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of (maybe foreground) tasks of storage in bytes", + "decimals": null, + "description": "the time of waiting for schedule", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5995,71 +5909,74 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 22 + "x": 12, + "y": 21 }, - "height": "", "hiddenSeries": false, - "id": 253, + "id": 115, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, - "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": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "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)", - "format": "time_series", + "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": "", - "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A", - "step": 10 + "legendFormat": "{{instance}}-{{resource_group}}-100", + "queryType": "randomWalk", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "SubTasks Write Throughput (bytes)", + "title": "Task Waiting Duration", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6072,35 +5989,51 @@ }, "yaxes": [ { - "format": "binBps", - "label": null, + "decimals": null, + "format": "s", + "label": "Time", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { "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 rows", + "description": "Executed DDL jobs per minute", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6108,34 +6041,28 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 22 + "x": 0, + "y": 6 }, - "height": "", "hiddenSeries": false, - "id": 255, + "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 }, @@ -6144,37 +6071,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_rows{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 (rows)", + "title": "Schema Internal DDL OPM", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6187,7 +6127,8 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": null, + "format": "opm", "label": null, "logBase": 1, "max": null, @@ -6195,12 +6136,12 @@ "show": true }, { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -6214,7 +6155,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage's internal sub tasks", + "description": "Executed DDL apply jobs per minute", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6222,22 +6163,21 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 7, "w": 12, - "x": 0, - "y": 30 + "x": 12, + "y": 6 }, "hiddenSeries": false, - "id": 39, + "id": 18, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": false, "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, - "sideWidth": null, "total": false, "values": false }, @@ -6256,16 +6196,13 @@ "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": true, + "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", + "expr": "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": 2, - "legendFormat": "{{type}}", + "intervalFactor": 1, + "legendFormat": "triggle-by-{{type}}", "refId": "A" } ], @@ -6273,7 +6210,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Small Internal Tasks OPS", + "title": "Schema Apply OPM", "tooltip": { "shared": true, "sort": 0, @@ -6289,8 +6226,8 @@ }, "yaxes": [ { - "decimals": 1, - "format": "ops", + "decimals": null, + "format": "opm", "label": null, "logBase": 1, "max": null, @@ -6298,12 +6235,12 @@ "show": true }, { - "format": "opm", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": false + "min": null, + "show": true } ], "yaxis": { @@ -6317,33 +6254,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 7, "w": 12, - "x": 12, - "y": 30 + "x": 0, + "y": 13 }, "hiddenSeries": false, - "id": 42, + "id": 20, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "max": true, + "current": false, + "hideZero": false, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, - "sort": null, - "sortDesc": null, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -6357,27 +6292,59 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/^applying/", + "yaxis": 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)", + "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": "max-{{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 Duration", + "title": "Schema Apply Duration", "tooltip": { "shared": true, "sort": 0, @@ -6393,7 +6360,6 @@ }, "yaxes": [ { - "decimals": 1, "format": "s", "label": null, "logBase": 1, @@ -6402,56 +6368,77 @@ "show": true }, { - "format": "s", + "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": "Total number 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": 0, - "y": 35 + "y": 7 }, "hiddenSeries": false, - "id": 130, + "id": 310, "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 }, @@ -6460,28 +6447,46 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 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": "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": "{{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": "Large Internal Tasks OPS", + "title": "CPU Usage (irate)", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -6497,7 +6502,7 @@ "yaxes": [ { "decimals": 1, - "format": "ops", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -6505,11 +6510,11 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": false } ], @@ -6524,21 +6529,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration 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": 12, - "y": 35 + "y": 7 }, "hiddenSeries": false, - "id": 131, + "id": 309, "legend": { "alignAsTable": true, "avg": false, @@ -6547,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 }, @@ -6564,28 +6574,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": false, "targets": [ { "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval]))) by (le,type) / 1000000000)", + "expr": "sum 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": "max-{{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 Duration", + "title": "Segment Reader", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -6601,7 +6632,7 @@ "yaxes": [ { "decimals": 1, - "format": "s", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -6609,11 +6640,11 @@ "show": true }, { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": false } ], @@ -6628,7 +6659,6 @@ "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": [] @@ -6636,25 +6666,22 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, + "h": 8, + "w": 12, "x": 0, - "y": 40 + "y": 15 }, "hiddenSeries": false, - "id": 50, + "id": 316, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", + "sort": "current", "sortDesc": true, "total": false, "values": true @@ -6662,7 +6689,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -6676,53 +6703,24 @@ "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)", + "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": 1, - "legendFormat": "RW-{{instance}}", - "refId": "D" + "legendFormat": "{{type}}-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Opened File Count", + "title": "Request QPS by instance", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -6735,6 +6733,7 @@ }, "yaxes": [ { + "decimals": null, "format": "none", "label": null, "logBase": 1, @@ -6743,12 +6742,12 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -6762,33 +6761,35 @@ "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)", + "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": 8, - "y": 40 + "h": 8, + "w": 12, + "x": 12, + "y": 15 }, + "height": "", "hiddenSeries": false, - "id": 22, + "id": 314, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, + "hideEmpty": 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 @@ -6805,6 +6806,7 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -6812,31 +6814,42 @@ "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_ReadBufferFromFileDescriptorReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", + "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "Open-{{instance}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "File Descriptor-{{instance}}", + "refId": "A", + "step": 10 }, { "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(rate(tiflash_system_profile_event_PSMReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "OpenFail-{{instance}}", + "legendFormat": "Page-{{instance}}", "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "PageBackGround-{{instance}}", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "File Open OPS", + "title": "Read Throughput by instance", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -6849,7 +6862,7 @@ }, "yaxes": [ { - "format": "ops", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -6861,8 +6874,8 @@ "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true } ], "yaxis": { @@ -6876,7 +6889,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "OPS and duration of fsync operations.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", + "description": "The total count of different kinds of commands received", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6884,13 +6897,13 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 40 + "h": 8, + "w": 12, + "x": 0, + "y": 23 }, "hiddenSeries": false, - "id": 52, + "id": 318, "legend": { "alignAsTable": true, "avg": false, @@ -6899,9 +6912,6 @@ "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, @@ -6919,7 +6929,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/max-fsync/", + "alias": "/delete_range|ingest/", "yaxis": 2 } ], @@ -6928,28 +6938,27 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_FileFSync{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "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": "ops-fsync-{{instance}}", - "refId": "A" + "legendFormat": "write block-{{instance}}", + "refId": "C" }, { - "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(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", + "format": "time_series", "hide": false, - "interval": "", - "legendFormat": "max-fsync-{{instance}}", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "{{type}}-{{instance}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "FSync Status", + "title": "Write Command OPS By Instance", "tooltip": { "shared": true, "sort": 0, @@ -6974,11 +6983,11 @@ "show": true }, { - "format": "s", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -6993,7 +7002,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of different kinds of read operations", + "decimals": 1, + "description": "The throughput of write by instance", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7001,30 +7011,34 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 0, - "y": 47 + "x": 12, + "y": 23 }, + "height": "", "hiddenSeries": false, - "id": 46, + "id": 312, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, - "hideZero": true, + "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 as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -7033,49 +7047,45 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + } + ], "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]))", + "exemplar": true, + "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": "File Descriptor", - "refId": "D" + "legendFormat": "write-{{instance}}", + "refId": "A", + "step": 10 }, { - "expr": "sum(rate(tiflash_system_profile_event_WriteBufferAIOWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "AIO", - "refId": "F" + "exemplar": true, + "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": "ingest-{{instance}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Disk Write OPS", + "title": "Write Throughput By Instance", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -7088,8 +7098,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -7097,46 +7106,59 @@ "show": true }, { - "format": "none", + "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": 12, - "y": 47 + "x": 0, + "y": 8 }, "hiddenSeries": false, - "id": 47, + "id": 325, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -7147,13 +7169,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": [], @@ -7162,40 +7184,20 @@ "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]))", + "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" - }, - { - "expr": "sum(rate(tiflash_system_profile_event_ReadBufferAIORead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "AIO", - "refId": "F" + "legendFormat": "keyspace-{{instance}}", + "refId": "I" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Disk Read OPS", + "title": "Number of Keyspaces", "tooltip": { "shared": true, "sort": 2, @@ -7211,21 +7213,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": { @@ -7239,8 +7240,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of different kinds of write operations", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7250,25 +7250,19 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 54 + "x": 12, + "y": 8 }, - "height": "", "hiddenSeries": false, - "id": 60, + "id": 324, "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 }, @@ -7281,52 +7275,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]))", - "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]))", + "exemplar": true, + "expr": "tiflash_system_current_metric_DT_NumStorageDeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "PageBackGround", - "refId": "C" + "legendFormat": "tables-{{instance}}", + "refId": "I" }, { - "expr": "sum(rate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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, "interval": "", "intervalFactor": 1, - "legendFormat": "AIO", - "refId": "D" + "legendFormat": "tables-all-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write flow", + "title": "Number of Physical Tables", "tooltip": { "shared": true, "sort": 2, @@ -7342,20 +7323,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": { @@ -7369,8 +7350,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of different kinds of read operations", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7380,25 +7360,19 @@ "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 54 + "x": 0, + "y": 16 }, - "height": "", "hiddenSeries": false, - "id": 59, + "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 }, @@ -7411,51 +7385,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]))", - "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]))", + "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": "PageBackGround", - "refId": "C" + "legendFormat": "segments-{{instance}}", + "refId": "I" }, { - "expr": "sum(rate(tiflash_system_profile_event_ReadBufferAIOReadBytes{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": "AIO", - "refId": "D" + "legendFormat": "mem_table-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read flow", + "title": "Number of Segments", "tooltip": { "shared": true, "sort": 2, @@ -7471,20 +7433,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": { @@ -7495,30 +7457,29 @@ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The current processing number of segments' background management", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 62 + "x": 12, + "y": 16 }, "hiddenSeries": false, - "id": 67, + "id": 323, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -7528,13 +7489,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": [], @@ -7543,36 +7504,33 @@ "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)", + "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": "seg_split-{{instance}}", - "refId": "B" + "legendFormat": "bytes-{{instance}}", + "refId": "I" }, { - "expr": "avg(tiflash_system_current_metric_DT_SegmentMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "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": "seg_merge-{{instance}}", - "refId": "C" + "legendFormat": "bytes-allocated-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Current Data Management Tasks", + "title": "Bytes of MemTables", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -7585,21 +7543,20 @@ }, "yaxes": [ { - "decimals": 0, - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7613,7 +7570,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Errors of DeltaIndex", + "description": "The memory usage of mark cache and minmax index cache", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7623,26 +7580,29 @@ "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 62 + "x": 0, + "y": 24 }, "hiddenSeries": false, - "id": 237, + "id": 238, "legend": { "alignAsTable": true, "avg": false, - "current": 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", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -7651,27 +7611,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": [ { "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", + "expr": "tiflash_system_asynchronous_metric_MarkCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "DeltaIndexError-{{instance}}", + "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_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": "DeltaIndexError", + "title": "Mark Cache and Minmax Index Cache Memory Usage", "tooltip": { "shared": true, "sort": 0, @@ -7687,8 +7669,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "cps", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -7696,11 +7677,11 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": false } ], @@ -7715,7 +7696,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The storage I/O limiter metrics.", + "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": [] @@ -7725,26 +7706,23 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 70 + "x": 12, + "y": 24 }, "hiddenSeries": false, - "id": 84, + "id": 169, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -7759,24 +7737,77 @@ "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_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": "I/O Limiter Throughput", + "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, @@ -7787,21 +7818,20 @@ }, "yaxes": [ { - "decimals": 0, - "format": "binBps", + "format": "percentunit", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "short", + "format": "percent", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7815,7 +7845,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": [] @@ -7825,26 +7855,27 @@ "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 70 + "x": 0, + "y": 32 }, "hiddenSeries": false, - "id": 266, + "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 }, @@ -7853,36 +7884,52 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "$$hashKey": "object:563", - "alias": "/-/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "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, + "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"current_size\"}) by (instance)", "interval": "", - "intervalFactor": 2, - "legendFormat": "{{type}}", + "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 Pending Rate and Duration", + "title": "Schema of Column File", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -7895,18 +7942,15 @@ }, "yaxes": [ { - "$$hashKey": "object:230", - "decimals": 0, - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "$$hashKey": "object:231", - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -7918,35 +7962,50 @@ "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": "I/O Limiter current pending count.", + "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": 86, + "id": 41, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": false, "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": false @@ -7954,18 +8013,18 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/pending/", + "alias": "/delete_range|ingest/", "yaxis": 2 } ], @@ -7974,68 +8033,27 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_system_current_metric_RateLimiterPendingWriteRequest{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "other-current-{{instance}}", - "refId": "A", - "hide": true - }, - { - "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" + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" }, { - "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))", + "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, - "interval": "", - "legendFormat": "{{type}}-pending-P99", - "refId": "G" + "intervalFactor": 1, + "legendFormat": "write block", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Current Pending Count", + "title": "Write Command OPS", "tooltip": { "shared": true, "sort": 0, @@ -8051,8 +8069,8 @@ }, "yaxes": [ { - "decimals": 0, - "format": "short", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -8060,11 +8078,11 @@ "show": true }, { - "format": "s", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -8076,10 +8094,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": [] @@ -8089,26 +8107,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": 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 }, @@ -8119,7 +8137,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "cache_hit_ratio", + "alias": "/fs|write/", "yaxis": 2 } ], @@ -8129,50 +8147,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\"} +\ntiflash_system_profile_event_WriteBufferAIOWriteBytes{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]) +\nrate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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]) +\nrate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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]) +\nrate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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]) +\nrate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{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, @@ -8184,19 +8228,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 } ], @@ -8211,7 +8255,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The information of read thread scheduling.", + "description": "Total number of storage engine read tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8220,22 +8264,22 @@ "fillGradient": 0, "gridPos": { "h": 8, - "w": 12, - "x": 12, - "y": 78 + "w": 24, + "x": 0, + "y": 17 }, "hiddenSeries": false, - "id": 269, + "id": 40, "legend": { "alignAsTable": true, - "avg": false, - "current": false, - "max": false, + "avg": true, + "current": true, + "max": true, "min": false, "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -8249,25 +8293,17 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "$$hashKey": "object:308", - "alias": "/push_block/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ru_exhausted|sche_active_segment_limit|sche_from_cache|sche_new_task|sche_no_pool|sche_no_ru|sche_no_segment|sche_no_slot|push_block_bytes\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_storage_read_tasks_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", - "hide": false, - "interval": "", + "instant": false, "intervalFactor": 2, - "legendFormat": "{{type}}", + "legendFormat": "{{instance}}", "refId": "A" } ], @@ -8275,7 +8311,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Thread Scheduling", + "title": "Read Tasks OPS", "tooltip": { "shared": true, "sort": 0, @@ -8291,22 +8327,20 @@ }, "yaxes": [ { - "$$hashKey": "object:321", "decimals": null, "format": "ops", - "label": null, + "label": "", "logBase": 1, "max": null, "min": "0", "show": true }, { - "$$hashKey": "object:322", - "format": "binBps", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -8321,29 +8355,36 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", + "decimals": 1, + "description": "The throughput of (maybe foreground) tasks of storage in bytes", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 78 + "x": 0, + "y": 25 }, + "height": "", "hiddenSeries": false, - "id": 88, + "id": 253, "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 }, @@ -8356,12 +8397,13 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/max_snapshot_lifetime/", + "alias": "/total/", "yaxis": 2 } ], @@ -8370,117 +8412,25 @@ "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\"}", + "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": "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" + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Snapshots", + "title": "SubTasks Write Throughput (bytes)", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -8493,7 +8443,7 @@ }, "yaxes": [ { - "format": "short", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -8501,12 +8451,12 @@ "show": true }, { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { @@ -8520,118 +8470,84 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "cache misses or cache hits of mark_cache.\nBased on this infactor, we can check whether mark_cache is large enough", + "decimals": 1, + "description": "The throughput of (maybe foreground) tasks of storage in rows", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 86 + "x": 12, + "y": 25 }, + "height": "", "hiddenSeries": false, - "id": 169, + "id": 255, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, "min": false, + "rightSide": true, "show": true, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "max(tiflash_system_profile_event_MarkCacheMisses{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "interval": "", - "legendFormat": "mark cache misses", - "queryType": "randomWalk", - "refId": "A" - }, - { - "exemplar": true, - "expr": "max(tiflash_system_profile_event_MarkCacheHits{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_storage_subtask_throughput_rows{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", "hide": false, "interval": "", - "legendFormat": "mark cache hits", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Effectiveness of Mark Cache", + "title": "SubTasks Write Throughput (rows)", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, - "transformations": [ - { - "id": "calculateField", - "options": { - "alias": "mark cache count total", - "binary": { - "left": "mark cache misses", - "operator": "+", - "reducer": "sum", - "right": "mark cache hits" - }, - "mode": "binary", - "reduce": { - "reducer": "sum" - } - } - }, - { - "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, @@ -8642,15 +8558,15 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "binBps", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "percent", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -8669,84 +8585,66 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Information about schema of column file, to learn the memory usage of schema", + "description": "Total number of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 5, "w": 12, - "x": 12, - "y": 86 + "x": 0, + "y": 33 }, "hiddenSeries": false, - "id": 168, + "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, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": false, + "steppedLine": true, "targets": [ { "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", - "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", - "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)", + "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": "", - "legendFormat": "miss_count", - "refId": "D" + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema of Column File", + "title": "Small Internal Tasks OPS", "tooltip": { "shared": true, "sort": 0, @@ -8762,20 +8660,21 @@ }, "yaxes": [ { - "format": "short", + "decimals": 1, + "format": "ops", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "short", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, - "show": true + "min": "0", + "show": false } ], "yaxis": { @@ -8789,28 +8688,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 5, "w": 12, - "x": 0, - "y": 165 + "x": 12, + "y": 33 }, "hiddenSeries": false, - "id": 292, + "id": 42, "legend": { - "alignAsTable": false, - "avg": true, - "current": false, - "max": false, + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, "min": false, "rightSide": true, "show": true, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -8822,7 +8724,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.17", + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -8832,47 +8734,21 @@ "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))", + "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": 1, - "legendFormat": "999-{{type}}", + "intervalFactor": 2, + "legendFormat": "max-{{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", + "title": "Small Internal Tasks Duration", "tooltip": { "shared": true, "sort": 0, @@ -8888,8 +8764,8 @@ }, "yaxes": [ { - "$$hashKey": "object:222", - "format": "µs", + "decimals": 1, + "format": "s", "label": null, "logBase": 1, "max": null, @@ -8897,12 +8773,11 @@ "show": true }, { - "$$hashKey": "object:223", - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": false } ], @@ -8917,7 +8792,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The memory usage of mark cache and minmax index cache", + "description": "Total number of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8925,31 +8800,29 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 5, "w": 12, "x": 0, - "y": 94 + "y": 38 }, "hiddenSeries": false, - "id": 238, + "id": 130, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, + "sideWidth": null, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -8958,49 +8831,27 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/limit/", - "color": "#F2495C", - "hideTooltip": true, - "legend": false, - "linewidth": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": false, + "steppedLine": true, "targets": [ { "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_MarkCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "hide": false, - "interval": "", - "legendFormat": "mark_cache_{{instance}}", - "refId": "L" - }, - { - "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_MinMaxIndexFiles{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "sum(rate(tiflash_storage_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": "", - "legendFormat": "minmax_index_cache_{{instance}}", + "intervalFactor": 2, + "legendFormat": "{{type}}", "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", + "title": "Large Internal Tasks OPS", "tooltip": { "shared": true, "sort": 0, @@ -9016,7 +8867,8 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": 1, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -9024,11 +8876,11 @@ "show": true }, { - "format": "short", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": false } ], @@ -9036,30 +8888,14 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Storage", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 6 - }, - "id": 64, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The stall duration of write and delete range", + "description": "Duration of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9067,13 +8903,13 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, - "w": 24, - "x": 0, - "y": 7 + "h": 5, + "w": 12, + "x": 12, + "y": 38 }, "hiddenSeries": false, - "id": 62, + "id": 131, "legend": { "alignAsTable": true, "avg": false, @@ -9082,9 +8918,8 @@ "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -9100,33 +8935,19 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "99-delta_merge", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_write_stall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type, instance))", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "99-{{type}}-{{instance}}", - "refId": "B" - }, - { - "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_write_stall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type, instance) / 1000000000)", + "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": 1, - "legendFormat": "max-{{type}}-{{instance}}", + "intervalFactor": 2, + "legendFormat": "max-{{type}}", "refId": "A" } ], @@ -9134,7 +8955,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Stall Duration", + "title": "Large Internal Tasks Duration", "tooltip": { "shared": true, "sort": 0, @@ -9150,6 +8971,7 @@ }, "yaxes": [ { + "decimals": 1, "format": "s", "label": null, "logBase": 1, @@ -9163,7 +8985,7 @@ "logBase": 1, "max": null, "min": "0", - "show": true + "show": false } ], "yaxis": { @@ -9177,8 +8999,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of write and delta's background management", + "description": "The number of currently opened file descriptors.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9186,14 +9007,13 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, - "w": 12, + "h": 7, + "w": 8, "x": 0, - "y": 15 + "y": 43 }, - "height": "", "hiddenSeries": false, - "id": 70, + "id": 50, "legend": { "alignAsTable": true, "avg": false, @@ -9222,38 +9042,55 @@ "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(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[1m]))", + "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": "write+ingest", - "refId": "A", - "step": 10 + "legendFormat": "W-{{instance}}", + "refId": "B" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"}[1m]))", + "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": "ManageDelta", - "refId": "B" + "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": "Write & Delta Management Throughput", + "title": "Opened File Count", "tooltip": { "shared": true, "sort": 0, @@ -9269,7 +9106,7 @@ }, "yaxes": [ { - "format": "binBps", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -9277,7 +9114,7 @@ "show": true }, { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -9296,8 +9133,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of write and delta's background management", + "description": "The number of open file descriptors action.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9305,20 +9141,19 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 15 + "h": 7, + "w": 8, + "x": 8, + "y": 43 }, - "height": "", "hiddenSeries": false, - "id": 137, + "id": 22, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -9341,7 +9176,6 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -9349,28 +9183,28 @@ "targets": [ { "exemplar": true, - "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"})", + "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": "write+ingest", - "refId": "C" + "legendFormat": "Open-{{instance}}", + "refId": "A" }, { "exemplar": true, - "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"})", + "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": "ManageDelta", - "refId": "D" + "legendFormat": "OpenFail-{{instance}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write & Delta Management Total", + "title": "File Open OPS", "tooltip": { "shared": true, "sort": 0, @@ -9386,7 +9220,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -9394,7 +9228,7 @@ "show": true }, { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -9413,8 +9247,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of write by instance", + "description": "OPS and duration of fsync operations.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9422,20 +9255,17 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 9, - "w": 24, - "x": 0, - "y": 23 + "h": 7, + "w": 8, + "x": 16, + "y": 43 }, - "height": "", "hiddenSeries": false, - "id": 89, + "id": 52, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, "max": true, "min": false, "rightSide": true, @@ -9449,7 +9279,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -9458,10 +9288,9 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/total/", + "alias": "/max-fsync/", "yaxis": 2 } ], @@ -9471,21 +9300,19 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write\"}[1m])) by (instance)", + "expr": "sum(rate(tiflash_system_profile_event_FileFSync{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", - "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "write-{{instance}}", - "refId": "A", - "step": 10 + "legendFormat": "ops-fsync-{{instance}}", + "refId": "A" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest\"}[1m])) by (instance)", + "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": "ingest-{{instance}}", + "legendFormat": "max-fsync-{{instance}}", "refId": "B" } ], @@ -9493,7 +9320,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Throughput By Instance", + "title": "FSync Status", "tooltip": { "shared": true, "sort": 0, @@ -9509,7 +9336,8 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -9517,12 +9345,12 @@ "show": true }, { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -9536,7 +9364,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total count of different kinds of commands received", + "description": "The number of different kinds of read operations", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9544,17 +9372,19 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 9, - "w": 24, + "h": 7, + "w": 12, "x": 0, - "y": 32 + "y": 50 }, "hiddenSeries": false, - "id": 90, + "id": 46, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -9574,41 +9404,49 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/delete_range|ingest/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", + "expr": "sum(rate(tiflash_system_profile_event_PSMWriteIOCalls{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "write block-{{instance}}", + "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(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", + "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "{{type}}-{{instance}}", + "legendFormat": "File Descriptor", "refId": "D" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_WriteBufferAIOWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "AIO", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Command OPS By Instance", + "title": "Disk Write OPS", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -9630,11 +9468,11 @@ "show": true }, { - "format": "opm", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -9642,56 +9480,38 @@ "align": false, "alignLevel": null } - } - ], - "title": "Storage Write Stall", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 7 - }, - "id": 119, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The disk usage of PageStorage instances in each TiFlash node", - "editable": true, - "error": false, + "description": "The number of different kinds of read operations", "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, - "grid": {}, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 0, - "y": 8 + "x": 12, + "y": 50 }, "hiddenSeries": false, - "id": 128, + "id": 47, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "hideEmpty": false, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, "total": false, "values": true }, @@ -9707,65 +9527,47 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/^valid_rate/", - "yaxis": 2 - }, - { - "alias": "/size/", - "linewidth": 3 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_BlobDiskBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "hide": false, - "interval": "", + "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": "blob_disk_size-{{instance}}", + "legendFormat": "Page", "refId": "A" }, { - "exemplar": true, - "expr": "sum(tiflash_system_asynchronous_metric_BlobValidBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", + "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": "blob_valid_size-{{instance}}", - "refId": "B" + "legendFormat": "PageFile", + "refId": "C" }, { - "exemplar": true, - "expr": "sum((tiflash_system_asynchronous_metric_BlobValidBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) / (tiflash_system_asynchronous_metric_BlobDiskBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"})) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "blob_valid_rate-{{instance}}", - "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" }, { - "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_LogDiskBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "sum(rate(tiflash_system_profile_event_ReadBufferAIORead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "log_size-{{instance}}", - "refId": "E", - "step": 10 + "intervalFactor": 1, + "legendFormat": "AIO", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PageStorage Disk Usage", + "title": "Disk Read OPS", "tooltip": { - "msResolution": false, "shared": true, "sort": 2, "value_type": "individual" @@ -9780,7 +9582,8 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -9788,11 +9591,11 @@ "show": true }, { - "format": "percentunit", + "format": "none", "label": null, "logBase": 1, - "max": "1.1", - "min": "0", + "max": null, + "min": null, "show": true } ], @@ -9808,40 +9611,42 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The number of files of PageStorage instances in each TiFlash node", - "editable": true, - "error": false, + "description": "The flow of different kinds of write operations", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 8 + "x": 0, + "y": 57 }, + "height": "", "hiddenSeries": false, - "id": 129, + "id": 60, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "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 as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -9850,38 +9655,50 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(tiflash_system_asynchronous_metric_BlobFileNums{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "blob_file-{{instance}}", + "legendFormat": "File Descriptor", "refId": "A", "step": 10 }, { - "exemplar": true, - "expr": "sum(tiflash_system_asynchronous_metric_LogNums{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "log_file-{{instance}}", + "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" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "AIO", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PageStorage File Num", + "title": "Write flow", "tooltip": { - "msResolution": false, "shared": true, "sort": 2, "value_type": "individual" @@ -9896,7 +9713,7 @@ }, "yaxes": [ { - "format": "short", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -9904,10 +9721,10 @@ "show": true }, { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, - "max": "1.1", + "max": null, "min": "0", "show": true } @@ -9918,103 +9735,164 @@ } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "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": 0, - "y": 16 + "x": 12, + "y": 57 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 133, + "height": "", + "hiddenSeries": false, + "id": 59, "legend": { - "show": true + "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 }, - "pluginVersion": "6.1.6", - "reverseYBuckets": false, + "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": [ { - "exemplar": true, - "expr": "sum(delta(tiflash_storage_page_write_batch_size_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"v3\"}[1m])) by (le)", - "format": "heatmap", - "interval": "", - "legendFormat": "{{le}}", - "queryType": "randomWalk", - "refId": "A" + "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" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_ReadBufferAIOReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "AIO", + "refId": "D" } ], - "title": "PageStorage WriteBatch Size", + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Read flow", "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 2, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "bytes", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null + "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": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 16 + "x": 0, + "y": 65 }, "hiddenSeries": false, - "id": 158, + "id": 67, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, @@ -10036,58 +9914,36 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "expr": "avg(tiflash_system_current_metric_DT_DeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", - "hide": true, - "interval": "", + "hide": false, "intervalFactor": 1, - "legendFormat": "{{type}}-max", + "legendFormat": "delta_merge-{{instance}}", "refId": "A" }, { - "exemplar": true, - "expr": "histogram_quantile(0.999, sum(rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "avg(tiflash_system_current_metric_DT_SegmentSplit{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", - "hide": false, - "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}-999", + "legendFormat": "seg_split-{{instance}}", "refId": "B" }, { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, - "interval": "", - "legendFormat": "{{type}}-99", + "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": "histogram_quantile(0.95, sum(rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, - "interval": "", - "legendFormat": "{{type}}-95", - "refId": "D" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.80, sum(rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, - "interval": "", - "legendFormat": "{{type}}-80", - "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Page write Duration", + "title": "Current Data Management Tasks", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -10100,7 +9956,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": 0, + "format": "short", "label": null, "logBase": 1, "max": null, @@ -10108,7 +9965,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -10127,34 +9984,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Errors of DeltaIndex", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 24 + "x": 12, + "y": 65 }, "hiddenSeries": false, - "id": 163, + "id": 237, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": false, - "hideEmpty": false, - "hideZero": true, - "max": false, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sort": "max", - "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -10175,13 +10029,12 @@ "targets": [ { "exemplar": true, - "expr": "sum(increase(tiflash_storage_page_gc_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "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, - "instant": false, "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "DeltaIndexError-{{instance}}", "refId": "A" } ], @@ -10189,10 +10042,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Page GC Tasks OPM", + "title": "DeltaIndexError", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -10205,7 +10058,8 @@ }, "yaxes": [ { - "format": "opm", + "decimals": null, + "format": "cps", "label": null, "logBase": 1, "max": null, @@ -10213,12 +10067,12 @@ "show": true }, { - "format": "short", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, - "show": true + "min": "0", + "show": false } ], "yaxis": { @@ -10232,6 +10086,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -10241,23 +10096,19 @@ "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 24 + "x": 0, + "y": 70 }, "hiddenSeries": false, - "id": 162, + "id": 84, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, @@ -10270,7 +10121,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -10279,35 +10130,22 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_page_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "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, "instant": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{type}}-max", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_page_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{type}}-99", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Page GC Duration", + "title": "I/O Limiter Throughput", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -10320,7 +10158,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": 0, + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -10347,36 +10186,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The number of pages of all TiFlash instance", - "editable": true, - "error": false, + "description": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 32 + "x": 12, + "y": 70 }, "hiddenSeries": false, - "id": 164, + "id": 266, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -10389,34 +10221,39 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ + "seriesOverrides": [ + { + "$$hashKey": "object:563", + "alias": "/-/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_PagesInMem{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "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": "num_pages-{{instance}}", - "refId": "A", - "step": 10 + "legendFormat": "{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Numer of Pages", + "title": "I/O Limiter Pending Rate and Duration", "tooltip": { - "msResolution": false, "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -10429,7 +10266,9 @@ }, "yaxes": [ { - "format": "short", + "$$hashKey": "object:230", + "decimals": 0, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -10437,7 +10276,8 @@ "show": true }, { - "format": "short", + "$$hashKey": "object:231", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -10456,99 +10296,118 @@ "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": "I/O Limiter current pending count.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 32 + "y": 70 }, "hiddenSeries": false, - "id": 123, + "id": 86, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "hideZero": true, + "current": false, "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/pending/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolV2Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "avg(tiflash_system_current_metric_RateLimiterPendingWriteRequest{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-V2", + "intervalFactor": 1, + "legendFormat": "other-current-{{instance}}", "refId": "A", - "step": 10 + "hide": true }, { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolV3Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-V3", + "legendFormat": "bgwrite-current-{{instance}}", "refId": "B" }, { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolMixMode{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-MixMode", + "legendFormat": "fgwrite-current-{{instance}}", "refId": "C" }, { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolUniPS{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-UniPS", + "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": "Number of Tables", + "title": "I/O Limiter Current Pending Count", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -10563,6 +10422,7 @@ }, "yaxes": [ { + "decimals": 0, "format": "short", "label": null, "logBase": 1, @@ -10571,7 +10431,7 @@ "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -10590,43 +10450,40 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "The information of data sharing cache hit ratio. Data sharing cache is purpose-built for OLAP workload that can reduce repeated data reads of concurrent table scanning.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 40 +<<<<<<< HEAD + "y": 78 +======= + "y": 73 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, - "height": "", "hiddenSeries": false, - "id": 198, + "id": 132, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -10635,32 +10492,60 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "cache_hit_ratio", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(tiflash_storage_page_data_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", + "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", - "step": 10 + "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": "PageStorage stored bytes by type", + "title": "Data Sharing", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -10673,7 +10558,8 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -10681,7 +10567,7 @@ "show": true }, { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -10700,7 +10586,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The num of pending writers in PageStorage", + "description": "The information of read thread scheduling.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -10711,30 +10597,29 @@ "h": 8, "w": 12, "x": 12, - "y": 40 +<<<<<<< HEAD + "y": 78 +======= + "y": 73 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "hiddenSeries": false, - "id": 231, + "id": 269, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": false, - "max": true, + "current": false, + "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -10743,19 +10628,25 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "$$hashKey": "object:308", + "alias": "/push_block/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_PSPendingWriterNum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_storage_read_thread_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ru_exhausted|sche_active_segment_limit|sche_from_cache|sche_new_task|sche_no_pool|sche_no_ru|sche_no_segment|sche_no_slot|push_block_bytes\"}[1m])) by (type)", "format": "time_series", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "size-{{instance}}", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" } ], @@ -10763,7 +10654,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PageStorage Pending Writers Num", + "title": "Read Thread Scheduling", "tooltip": { "shared": true, "sort": 0, @@ -10779,7 +10670,9 @@ }, "yaxes": [ { - "format": "none", + "$$hashKey": "object:321", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -10787,12 +10680,13 @@ "show": true }, { - "format": "short", + "$$hashKey": "object:322", + "format": "binBps", "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true } ], "yaxis": { @@ -10806,26 +10700,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", +<<<<<<< HEAD +======= "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 9, - "w": 24, + "h": 8, + "w": 12, "x": 0, - "y": 48 + "y": 81 }, "hiddenSeries": false, - "id": 232, + "id": 292, "legend": { "alignAsTable": true, - "avg": false, - "current": true, - "max": true, + "avg": true, + "current": false, + "max": false, "min": false, "rightSide": true, "show": true, @@ -10844,31 +10739,53 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_page_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, 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": 1, - "legendFormat": "{{type}}-{{instance}}", + "legendFormat": "999-{{type}}", "refId": "A" - } - ], - "thresholds": [], + }, + { + "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": "PS Command OPS By Instance", + "title": "Read Thread Internal Duration", "tooltip": { "shared": true, "sort": 2, @@ -10884,8 +10801,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "µs", "label": null, "logBase": 1, "max": null, @@ -10893,97 +10809,187 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { "align": false, "alignLevel": null } - } - ], - "title": "PageStorage", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 8 - }, - "id": 34, - "panels": [ + }, { "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": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 0, - "y": 9 + "x": 12, + "y": 81 }, "hiddenSeries": false, - "id": 167, + "id": 88, "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, "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": [], + "seriesOverrides": [ + { + "alias": "/max_snapshot_lifetime/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_stale_read_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "tiflash_system_current_metric_DT_SegmentReadTasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "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}}", + "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": "Stale Read OPS", + "title": "Read Snapshots", "tooltip": { "shared": true, "sort": 0, @@ -10999,8 +11005,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -11008,11 +11013,11 @@ "show": true }, { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -11027,41 +11032,40 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", +<<<<<<< HEAD + "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": 7, + "h": 8, "w": 12, - "x": 12, - "y": 9 + "x": 0, + "y": 86 }, "hiddenSeries": false, - "id": 35, + "id": 169, "legend": { - "alignAsTable": false, "avg": false, "current": false, "max": false, "min": false, - "rightSide": false, "show": true, "total": false, "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": [], @@ -11070,48 +11074,101 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_raft_read_index_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", + "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": "Raft Read Index OPS", + "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": "ops", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, + "transformations": [ { - "format": "none", + "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": { @@ -11125,6 +11182,10 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Information about schema of column file, to learn the memory usage of schema", +======= + "description": "", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "fieldConfig": { "defaults": {}, "overrides": [] @@ -11132,89 +11193,101 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, +<<<<<<< HEAD + "x": 12, + "y": 86 + }, + "hiddenSeries": false, + "id": 168, +======= "x": 0, - "y": 16 + "y": 89 }, "hiddenSeries": false, - "id": 37, + "id": 291, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": false, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, +<<<<<<< HEAD +======= "links": [], - "nullPointMode": "null as zero", +>>>>>>> 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": "/timeout/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_wait_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le) / 1000000000)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "max", + "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", + "queryType": "randomWalk", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_wait_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "99", + "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" }, { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_wait_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "95", + "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" }, { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_raft_wait_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "80", + "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(increase(tiflash_system_profile_event_RaftWaitIndexTimeout{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", +======= + "expr": "tiflash_memory_usage_by_class{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-timeout", - "refId": "E" + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "B" +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Wait Index Duration", +<<<<<<< HEAD + "title": "Schema of Column File", +======= + "title": "Memory by class", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "tooltip": { "shared": true, "sort": 0, @@ -11230,20 +11303,23 @@ }, "yaxes": [ { - "format": "s", +<<<<<<< HEAD + "format": "short", +======= + "format": "bytes", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "decimals": 2, - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -11258,7 +11334,10 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of currently applying snapshots.", +<<<<<<< HEAD +======= + "description": "", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "fieldConfig": { "defaults": {}, "overrides": [] @@ -11266,20 +11345,18 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 16 + "y": 89 }, "hiddenSeries": false, - "id": 36, + "id": 289, "legend": { - "alignAsTable": false, "avg": false, "current": false, "max": false, "min": false, - "rightSide": false, "show": true, "total": false, "values": false @@ -11287,13 +11364,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": [], @@ -11302,42 +11379,34 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le) / 1000000000)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "max", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "exemplar": true, + "expr": "tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"alloc_.*\"}", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "99", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "95", - "refId": "C" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "exemplar": true, + "expr": "-tiflash_storages_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"dealloc_.*\"}", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "80", - "refId": "D" + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Batch Read Index Duration", + "title": "Memory by thread", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -11350,11 +11419,11 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -11377,47 +11446,62 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of applying Raft write logs", - "editable": true, - "error": false, + "description": "The compression ratio of different compression algorithm", "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 1, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 7, - "w": 24, + "w": 12, "x": 0, - "y": 23 +<<<<<<< HEAD + "y": 165 }, "hiddenSeries": false, - "id": 82, + "id": 292, + "legend": { + "alignAsTable": false, + "avg": true, + "current": false, +======= + "y": 97 + }, + "hiddenSeries": false, + "id": 294, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, - "max": true, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "max": false, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, +<<<<<<< HEAD "links": [], "nullPointMode": "null as zero", +======= + "nullPointMode": "null", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "options": { "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", +<<<<<<< HEAD + "pluginVersion": "7.5.17", "pointradius": 5, +======= + "pluginVersion": "7.5.11", + "pointradius": 2, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "points": false, "renderer": "flot", "seriesOverrides": [], @@ -11427,73 +11511,69 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_apply_write_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", +<<<<<<< HEAD + "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": 2, - "legendFormat": " 100%-{{type}}", - "refId": "D", - "step": 4 + "intervalFactor": 1, + "legendFormat": "999-{{type}}", +======= + "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", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "refId": "A" }, { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_apply_write_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", +<<<<<<< HEAD + "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": 2, - "legendFormat": " 99%-{{type}}", - "metric": "", - "refId": "A", - "step": 4 + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" }, { "exemplar": true, - "expr": "sum(rate(tiflash_raft_apply_write_command_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m])) / sum(rate(tiflash_raft_apply_write_command_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m])) ", + "expr": "histogram_quantile(0.95, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", "interval": "", - "intervalFactor": 2, - "legendFormat": "avg-write", - "refId": "C", - "step": 4 + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" }, { "exemplar": true, - "expr": "sum(rate(tiflash_raft_apply_write_command_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"admin\"}[1m])) / sum(rate(tiflash_raft_apply_write_command_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"admin\"}[1m])) ", + "expr": "histogram_quantile(0.80, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", - "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "avg-admin", - "refId": "E", - "step": 4 - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_raft_apply_write_command_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", cluster_id=~\".*$tidb_cluster\", instance=~\"$instance\", type=\"flush_region\"}[1m])) / sum(rate(tiflash_raft_apply_write_command_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", cluster_id=~\".*$tidb_cluster\", instance=~\"$instance\", type=\"flush_region\"}[1m]))", + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "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": "avg-flush_region", + "legendFormat": "lightweight", "refId": "B" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_raft_write_data_to_storage_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", cluster_id=~\".*$tidb_cluster\", instance=~\"$instance\", type=\"decode\"}[1m])) / sum(rate(tiflash_raft_write_data_to_storage_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", cluster_id=~\".*$tidb_cluster\", instance=~\"$instance\", type=\"decode\"}[1m]) ) ", - "hide": false, - "interval": "", - "legendFormat": "avg-decode", - "refId": "F" +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Apply Raft write logs Duration", +<<<<<<< HEAD + "title": "Read Thread Internal Duration", +======= + "title": "Compression Ratio", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "tooltip": { - "msResolution": false, "shared": true, - "sort": 2, - "value_type": "cumulative" + "sort": 0, + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -11505,13 +11585,29 @@ }, "yaxes": [ { - "format": "s", +<<<<<<< HEAD + "$$hashKey": "object:222", + "format": "µs", "label": null, "logBase": 1, "max": null, "min": "0", "show": true }, + { + "$$hashKey": "object:223", +======= +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, +<<<<<<< HEAD + "show": false +======= + "show": true + }, { "format": "short", "label": null, @@ -11519,6 +11615,7 @@ "max": null, "min": null, "show": true +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "yaxis": { @@ -11532,47 +11629,53 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "editable": true, - "error": false, +<<<<<<< HEAD + "description": "The memory usage of mark cache and minmax index cache", +======= + "description": "The count of the compression algorithm used by each data part", +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 7, - "w": 24, + "w": 12, +<<<<<<< HEAD "x": 0, - "y": 30 + "y": 94 +======= + "x": 12, + "y": 97 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "hiddenSeries": false, - "id": 242, + "id": 238, "legend": { "alignAsTable": true, "avg": false, "current": true, +<<<<<<< HEAD "max": true, +======= + "max": false, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, - "total": false, + "total": true, "values": true }, "lines": true, "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -11582,25 +11685,23 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", + "expr": "sum(rate(tiflash_storage_pack_compression_algorithm_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "99%-{{type}}", - "refId": "D", - "step": 4 + "legendFormat": "{{type}}", + "queryType": "randomWalk", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Heavy Raft Apply Duration", + "title": "Compression Algorithm Count", "tooltip": { - "msResolution": false, "shared": true, - "sort": 2, - "value_type": "cumulative" + "sort": 0, + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -11612,11 +11713,11 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -11634,317 +11735,182 @@ } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of decoding Region data into blocks when writing Region data to the storage layer. (Mixed with \"write logs\" and \"apply Snapshot\" operations)", "fieldConfig": { "defaults": {}, "overrides": [] }, + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 37 + "y": 104 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 76, + "hiddenSeries": false, + "id": 301, "legend": { - "show": true + "alignAsTable": true, + "avg": true, + "current": false, + "max": false, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true }, + "lines": true, + "linewidth": 1, "links": [], - "reverseYBuckets": false, - "targets": [ + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ { - "expr": "sum(delta(tiflash_raft_write_data_to_storage_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"decode\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2 } ], - "timeFrom": null, - "timeShift": null, - "title": "Region write Duration (decode)", - "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of writing Region data blocks to the storage layer (Mixed with \"write logs\" and \"apply Snapshot\" operations)", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 37 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 87, - "legend": { - "show": true - }, - "links": [], - "reverseYBuckets": false, + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "sum(delta(tiflash_raft_write_data_to_storage_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 2, - "legendFormat": "{{le}}", + "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": "Region write Duration (write blocks)", + "title": "Mark Cache and Minmax Index Cache Memory Usage", "tooltip": { - "show": true, - "showHistogram": true + "shared": true, + "sort": 2, + "value_type": "individual" }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of applying Raft write logs", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 44 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 81, - "legend": { - "show": true + "values": [] }, - "links": [], - "reverseYBuckets": false, - "targets": [ + "yaxes": [ { - "exemplar": true, - "expr": "sum(delta(tiflash_raft_apply_write_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m])) by (le)", - "format": "heatmap", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Apply Raft write logs Duration [Heatmap]", - "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of applying Raft write logs", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 44 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 93, - "legend": { - "show": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, { - "expr": "sum(delta(tiflash_raft_apply_write_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"admin\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false } ], - "timeFrom": null, - "timeShift": null, - "title": "Apply Raft admin logs Duration [Heatmap]", - "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, + "yaxis": { + "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": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 51 + "y": 10 }, "hiddenSeries": false, - "id": 235, + "id": 84, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -11955,7 +11921,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -11964,12 +11930,11 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_raft_raft_events_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "expr": "sum(rate(tiflash_storage_io_limiter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" } ], @@ -11977,7 +11942,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Events QPS", + "title": "I/O Limiter Throughput", "tooltip": { "shared": true, "sort": 2, @@ -11993,8 +11958,8 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "decimals": 0, + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -12002,7 +11967,7 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -12021,30 +11986,33 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Current limit bytes per second of Storage I/O limiter", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 51 + "y": 10 }, "hiddenSeries": false, - "id": 241, + "id": 305, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -12055,21 +12023,22 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": false, + "stack": true, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_raft_raft_frequent_events_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "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": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" } ], @@ -12077,10 +12046,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Frequent Events QPS", + "title": "I/O Limiter Threshold", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -12093,8 +12062,8 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "decimals": 0, + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -12102,7 +12071,7 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -12116,305 +12085,229 @@ } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", + "description": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] }, + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 58 + "y": 18 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 234, + "hiddenSeries": false, + "id": 266, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true }, + "lines": true, + "linewidth": 1, "links": [], - "reverseYBuckets": false, + "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(delta(tiflash_raft_raft_log_gap_count_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"applied_index\"}[1m])) by (le, type)", - "format": "heatmap", - "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": "{{le}}", - "refId": "B" - }, - { - "exemplar": true, - "expr": "sum(delta(tiflash_raft_raft_log_gap_count_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"compact_index\"}[1m])) by (le, type)", - "hide": false, - "interval": "", - "legendFormat": "{{le}}", + "legendFormat": "{{type}}", "refId": "A" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Raft Log Gap Heatmap", + "title": "I/O Limiter Pending Rate and Duration", "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 0, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "none", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "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 + } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", + "description": "I/O Limiter current pending count.", "fieldConfig": { "defaults": {}, "overrides": [] }, + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 58 + "y": 18 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 243, + "hiddenSeries": false, + "id": 86, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false }, + "lines": true, + "linewidth": 1, "links": [], - "reverseYBuckets": false, + "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": "sum(delta(tiflash_raft_entry_size_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"normal\"}[1m])) by (le, type)", - "format": "heatmap", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", + "legendFormat": "bgwrite-current-{{instance}}", "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Raft Entry Batch Size Heatmap", - "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 65 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 244, - "legend": { - "show": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ + }, { "exemplar": true, - "expr": "sum(delta(tiflash_raft_region_flush_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"unflushed\"}[1m])) by (le, type)", - "format": "heatmap", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" + "legendFormat": "fgwrite-current-{{instance}}", + "refId": "C" }, { "exemplar": true, - "expr": "sum(delta(tiflash_raft_region_flush_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"flushed\"}[1m])) by (le, type)", - "format": "heatmap", - "hide": true, + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Region Size Heatmap", - "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "bytes", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 65 - }, - "hiddenSeries": false, - "id": 249, - "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": [ + "legendFormat": "bgread-current-{{instance}}", + "refId": "D" + }, { "exemplar": true, - "expr": "sum(rate(tiflash_raft_ongoing_snapshot_total_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", - "format": "time_series", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" + "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": "Ongoing raft snapshot", + "title": "I/O Limiter Current Pending Count", "tooltip": { "shared": true, "sort": 0, @@ -12430,15 +12323,16 @@ }, "yaxes": [ { - "format": "bytes", + "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, @@ -12450,13 +12344,30 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Rate Limiter", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 10 + }, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "id": 64, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The stall duration of write and delete range", "fieldConfig": { "defaults": {}, "overrides": [] @@ -12464,23 +12375,30 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 12, + "h": 8, + "w": 24, "x": 0, - "y": 72 +<<<<<<< HEAD + "y": 7 +======= + "y": 10 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "hiddenSeries": false, - "id": 240, + "id": 62, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -12494,18 +12412,33 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "99-delta_merge", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_raft_eager_gc_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_write_stall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type, instance))", "format": "time_series", + "hide": true, "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "99-{{type}}-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_write_stall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type, instance) / 1000000000)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "max-{{type}}-{{instance}}", "refId": "A" } ], @@ -12513,7 +12446,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Eager GC OPS", + "title": "Write Stall Duration", "tooltip": { "shared": true, "sort": 0, @@ -12529,8 +12462,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -12538,11 +12470,11 @@ "show": true }, { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -12557,33 +12489,39 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of Raft logs eager GC tasks", - "editable": true, - "error": false, + "decimals": 1, + "description": "The throughput of write and delta's background management", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, - "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 72 + "x": 0, +<<<<<<< HEAD + "y": 15 +======= + "y": 18 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, + "height": "", "hiddenSeries": false, - "id": 239, + "id": 70, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": false, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "current", + "sideWidth": 250, + "sort": "max", "sortDesc": true, "total": false, "values": true @@ -12591,7 +12529,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -12600,6 +12538,7 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -12607,58 +12546,34 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_eager_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[1m]))", "format": "time_series", + "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": " 99%-{{type}}", - "metric": "", + "intervalFactor": 1, + "legendFormat": "write+ingest", "refId": "A", - "step": 4 - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_eager_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 2, - "legendFormat": "95%-{{type}}", - "refId": "B", - "step": 4 - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_raft_eager_gc_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type) / sum(rate(tiflash_raft_eager_gc_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "avg-{{type}}", - "refId": "C", - "step": 4 + "step": 10 }, { "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_eager_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"}[1m]))", "format": "time_series", - "hide": true, "interval": "", - "intervalFactor": 2, - "legendFormat": " 100%-{{type}}", - "refId": "D", - "step": 4 + "intervalFactor": 1, + "legendFormat": "ManageDelta", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Eager GC Duration", + "title": "Write & Delta Management Throughput", "tooltip": { - "msResolution": false, "shared": true, - "sort": 2, - "value_type": "cumulative" + "sort": 0, + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -12670,7 +12585,7 @@ }, "yaxes": [ { - "format": "s", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -12678,12 +12593,12 @@ "show": true }, { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -12697,21 +12612,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of currently applying snapshots.", + "decimals": 1, + "description": "The throughput of write and delta's background management", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 24, - "x": 0, - "y": 79 + "h": 8, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 15 +======= + "y": 18 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, + "height": "", "hiddenSeries": false, - "id": 75, + "id": 137, "legend": { "alignAsTable": true, "avg": false, @@ -12722,13 +12643,16 @@ "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 }, @@ -12737,33 +12661,36 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(tiflash_system_current_metric_RaftNumSnapshotsPendingApply{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "exemplar": true, + "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"})", "format": "time_series", - "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "Pending-{{instance}}", - "refId": "B" + "legendFormat": "write+ingest", + "refId": "C" }, { - "expr": "sum(tiflash_system_current_metric_RaftNumPrehandlingSubTasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "exemplar": true, + "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"})", "format": "time_series", - "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "PrehandleSubtasks-{{instance}}", - "refId": "A" + "legendFormat": "ManageDelta", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Applying snapshots Count", + "title": "Write & Delta Management Total", "tooltip": { "shared": true, "sort": 0, @@ -12779,7 +12706,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -12787,12 +12714,12 @@ "show": true }, { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -12801,216 +12728,5802 @@ } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of pre-decode when applying region snapshot", + "decimals": 1, + "description": "The throughput of write by instance", "fieldConfig": { "defaults": {}, "overrides": [] }, + "fill": 0, + "fillGradient": 0, "gridPos": { - "h": 7, - "w": 12, + "h": 9, + "w": 24, "x": 0, - "y": 86 +<<<<<<< HEAD + "y": 23 +======= + "y": 26 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 72, + "height": "", + "hiddenSeries": false, + "id": 89, "legend": { - "show": true + "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": [], - "reverseYBuckets": false, + "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": [ { - "expr": "sum(delta(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"snapshot_predecode\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 2, - "legendFormat": "{{le}}", + "exemplar": true, + "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": "write-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "exemplar": true, + "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": "ingest-{{instance}}", "refId": "B" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Snapshot Predecode Duration", + "title": "Write Throughput By Instance", "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 0, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "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 + } }, { - "cards": { - "cardPadding": null, - "cardRound": null + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total count of different kinds of commands received", + "fieldConfig": { + "defaults": {}, + "overrides": [] }, - "color": { + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 9, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 32 +======= + "y": 35 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 90, + "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": [ + { + "alias": "/delete_range|ingest/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "write block-{{instance}}", + "refId": "C" + }, + { + "expr": "sum(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{type}}-{{instance}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Command OPS By Instance", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "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": "opm", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Storage Write Stall", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 7 +======= + "y": 11 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "id": 119, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The disk usage of PageStorage instances in each TiFlash node", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 11 + }, + "hiddenSeries": false, + "id": 128, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 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": [ + { + "alias": "/^valid_rate/", + "yaxis": 2 + }, + { + "alias": "/size/", + "linewidth": 3 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_BlobDiskBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "blob_disk_size-{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_asynchronous_metric_BlobValidBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "blob_valid_size-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum((tiflash_system_asynchronous_metric_BlobValidBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) / (tiflash_system_asynchronous_metric_BlobDiskBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"})) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "blob_valid_rate-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_LogDiskBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "log_size-{{instance}}", + "refId": "E", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PageStorage Disk Usage", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "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": "percentunit", + "label": null, + "logBase": 1, + "max": "1.1", + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The number of files of PageStorage instances in each TiFlash node", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 11 + }, + "hiddenSeries": false, + "id": 129, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 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": "sum(tiflash_system_asynchronous_metric_BlobFileNums{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "blob_file-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_asynchronous_metric_LogNums{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "log_file-{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PageStorage File Num", + "tooltip": { + "msResolution": false, + "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": "0", + "show": true + }, + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": "1.1", + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 19 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 133, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_storage_page_write_batch_size_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"v3\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "PageStorage WriteBatch Size", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 19 + }, + "hiddenSeries": false, + "id": 158, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}-max", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}-999", + "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-99", + "refId": "C" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-95", + "refId": "D" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.80, sum(rate(tiflash_storage_page_write_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-80", + "refId": "E" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Page write Duration", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 27 + }, + "hiddenSeries": false, + "id": 163, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": false, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sort": "max", + "sortDesc": 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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(increase(tiflash_storage_page_gc_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Page GC Tasks OPM", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "opm", + "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}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 27 + }, + "hiddenSeries": false, + "id": 162, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 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_page_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}-max", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_page_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}-99", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Page GC Duration", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The number of pages of all TiFlash instance", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 35 + }, + "hiddenSeries": false, + "id": 164, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "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", + "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": "tiflash_system_asynchronous_metric_PagesInMem{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "num_pages-{{instance}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Numer of Pages", + "tooltip": { + "msResolution": false, + "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": "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 num of pending writers in PageStorage", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 35 + }, + "hiddenSeries": false, + "id": 231, + "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": [ + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_PSPendingWriterNum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "size-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PageStorage Pending Writers Num", + "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}", + "decimals": 1, + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 43 + }, + "height": "", + "hiddenSeries": false, + "id": 198, + "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": [ + { + "exemplar": true, + "expr": "sum(tiflash_storage_page_data_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PageStorage stored bytes by type", + "tooltip": { + "shared": true, + "sort": 2, + "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": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "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, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 43 + }, + "hiddenSeries": false, + "id": 123, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideZero": true, + "max": false, + "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", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_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": "", + "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": "Number of Tables", + "tooltip": { + "msResolution": false, + "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 + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 9, + "w": 24, + "x": 0, + "y": 51 + }, + "hiddenSeries": false, + "id": 232, + "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": [ + { + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_page_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PS Command OPS By Instance", + "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": "opm", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "PageStorage", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 8 +======= + "y": 12 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "id": 34, + "panels": [ + { + "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": 12 + }, + "hiddenSeries": false, + "id": 167, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_stale_read_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Stale Read 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": 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}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 12 + }, + "hiddenSeries": false, + "id": 35, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tiflash_raft_read_index_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Read Index 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": 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}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, +<<<<<<< HEAD +======= + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 19 + }, + "hiddenSeries": false, + "id": 270, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_learner_read_failures_count{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": "Learner Read Failures", + "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": "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}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 19 + }, + "hiddenSeries": false, + "id": 271, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_read_index_events_count{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": "Read Index Events", + "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": "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}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 16 +======= + "y": 26 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 37, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": 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": [ + { + "alias": "/timeout/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_wait_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le) / 1000000000)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "max", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_wait_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_wait_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "95", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_raft_wait_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "80", + "refId": "D" + }, + { + "expr": "sum(increase(tiflash_system_profile_event_RaftWaitIndexTimeout{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-timeout", + "refId": "E" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Wait Index Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "decimals": 2, + "format": "opm", + "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": "The number of currently applying snapshots.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 16 +======= + "y": 26 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 36, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le) / 1000000000)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "max", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "95", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "80", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Batch Read Index Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of applying Raft write logs", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 23 +======= + "y": 33 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 82, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_apply_write_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": " 100%-{{type}}", + "refId": "D", + "step": 4 + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_apply_write_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": " 99%-{{type}}", + "metric": "", + "refId": "A", + "step": 4 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_apply_write_command_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m])) / sum(rate(tiflash_raft_apply_write_command_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m])) ", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg-write", + "refId": "C", + "step": 4 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_apply_write_command_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"admin\"}[1m])) / sum(rate(tiflash_raft_apply_write_command_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"admin\"}[1m])) ", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg-admin", + "refId": "E", + "step": 4 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_apply_write_command_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", cluster_id=~\".*$tidb_cluster\", instance=~\"$instance\", type=\"flush_region\"}[1m])) / sum(rate(tiflash_raft_apply_write_command_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", cluster_id=~\".*$tidb_cluster\", instance=~\"$instance\", type=\"flush_region\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "avg-flush_region", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_write_data_to_storage_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", cluster_id=~\".*$tidb_cluster\", instance=~\"$instance\", type=\"decode\"}[1m])) / sum(rate(tiflash_raft_write_data_to_storage_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", cluster_id=~\".*$tidb_cluster\", instance=~\"$instance\", type=\"decode\"}[1m]) ) ", + "hide": false, + "interval": "", + "legendFormat": "avg-decode", + "refId": "F" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Apply Raft write logs Duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 30 +======= + "y": 40 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 242, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "99%-{{type}}", + "refId": "D", + "step": 4 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Heavy Raft Apply Duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of decoding Region data into blocks when writing Region data to the storage layer. (Mixed with \"write logs\" and \"apply Snapshot\" operations)", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 37 +======= + "y": 47 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 76, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(delta(tiflash_raft_write_data_to_storage_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"decode\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Region write Duration (decode)", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of writing Region data blocks to the storage layer (Mixed with \"write logs\" and \"apply Snapshot\" operations)", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 37 +======= + "y": 47 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 87, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(delta(tiflash_raft_write_data_to_storage_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Region write Duration (write blocks)", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of applying Raft write logs", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 44 +======= + "y": 54 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 81, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_apply_write_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Apply Raft write logs Duration [Heatmap]", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of applying Raft write logs", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 44 +======= + "y": 54 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 93, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(delta(tiflash_raft_apply_write_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"admin\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Apply Raft admin logs Duration [Heatmap]", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "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, +<<<<<<< HEAD + "y": 51 +======= + "y": 61 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 235, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_raft_events_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Events QPS", + "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}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 51 +======= + "y": 61 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 241, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_raft_frequent_events_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Frequent Events QPS", + "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 + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 58 +======= + "y": 68 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 234, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_raft_log_gap_count_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"applied_index\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_raft_log_gap_count_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"compact_index\"}[1m])) by (le, type)", + "hide": false, + "interval": "", + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Raft Log Gap Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 58 +======= + "y": 68 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 243, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_entry_size_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"normal\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Raft Entry Batch Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 65 +======= + "y": 75 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 244, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_region_flush_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"unflushed\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_region_flush_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"flushed\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": true, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Region Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 65 +======= + "y": 75 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 249, + "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": "sum(rate(tiflash_raft_ongoing_snapshot_total_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Ongoing raft snapshot", + "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 + } + }, + { +<<<<<<< HEAD +======= + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 82 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 261, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_write_flow_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write_committed\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Write Committed Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 82 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 262, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_write_flow_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"snapshot_uncommitted\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Uncommitted Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 89 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 265, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_write_flow_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest_uncommitted\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Ingest Uncommitted Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 89 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 264, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_write_flow_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"big_write_to_region\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Big Write To Region Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "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, +<<<<<<< HEAD + "y": 72 +======= + "y": 96 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 240, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_eager_gc_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Eager GC 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": 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": "Duration of Raft logs eager GC tasks", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 72 +======= + "y": 96 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 239, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_eager_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": " 99%-{{type}}", + "metric": "", + "refId": "A", + "step": 4 + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_eager_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 2, + "legendFormat": "95%-{{type}}", + "refId": "B", + "step": 4 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_eager_gc_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type) / sum(rate(tiflash_raft_eager_gc_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg-{{type}}", + "refId": "C", + "step": 4 + }, + { + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_eager_gc_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 2, + "legendFormat": " 100%-{{type}}", + "refId": "D", + "step": 4 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Eager GC Duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of currently applying snapshots.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 79 +======= + "y": 103 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 75, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": 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": [ + { + "expr": "sum(tiflash_system_current_metric_RaftNumSnapshotsPendingApply{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "Pending-{{instance}}", + "refId": "B" + }, + { + "expr": "sum(tiflash_system_current_metric_RaftNumPrehandlingSubTasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "PrehandleSubtasks-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Applying snapshots 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": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of pre-decode when applying region snapshot", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 86 +======= + "y": 110 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 72, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(delta(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"snapshot_predecode\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Predecode Duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of SST to DT in pre-decode when applying region snapshot", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 86 +======= + "y": 110 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 127, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(delta(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"snapshot_predecode_sst2dt\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Predecode SST to DT Duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { +<<<<<<< HEAD +======= + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 117 + }, + "hiddenSeries": false, + "id": 263, + "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": "sum(tiflash_system_current_metric_MemoryTrackingKVStore{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "KVStore memory", + "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}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 117 + }, + "hiddenSeries": false, + "id": 272, + "legend": { + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "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_raft_proxy_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_raft_proxy_thread_memory_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"dealloc_.*\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Memory by thread", + "tooltip": { + "shared": true, + "sort": 2, + "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 + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 124 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 290, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_command_throughput_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"prehandle_snapshot\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Prehandle Throughput Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 93 +======= + "y": 124 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 250, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_snapshot_total_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"approx_raft_snapshot\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The keys flow of different kinds of Raft operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 100 +======= + "y": 131 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "height": "", + "hiddenSeries": false, + "id": 71, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "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_raft_process_keys{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Keys flow", + "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", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { +<<<<<<< HEAD +======= + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, + "y": 138 + }, + "height": "", + "hiddenSeries": false, + "id": 260, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "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": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft throughput", + "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", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of pre-decode when applying region snapshot", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 107 +======= + "y": 145 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 73, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(delta(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"snapshot_flush\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Flush Duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of ingesting SST", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 107 +======= + "y": 145 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 74, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(delta(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"ingest_sst\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Ingest SST Duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Latency that TiKV sends raft log to TiFlash.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, +<<<<<<< HEAD + "y": 114 +======= + "y": 152 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 92, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(delta(tiflash_raft_upstream_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Upstream Latency [Heatmap]", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Latency that TiKV sends raft log to TiFlash.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, +<<<<<<< HEAD + "y": 114 +======= + "y": 152 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "hiddenSeries": false, + "id": 91, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_upstream_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le) / 1000000000)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": " 100%", + "metric": "", + "refId": "A", + "step": 4 + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_upstream_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": " 99%", + "metric": "", + "refId": "B", + "step": 4 + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_upstream_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "95%", + "refId": "C", + "step": 4 + }, + { + "expr": "sum(rate(tiflash_raft_upstream_latency_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) / sum(rate(tiflash_raft_upstream_latency_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) ", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "avg", + "refId": "D", + "step": 4 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Upstream Latency", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } +<<<<<<< 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": 159 + }, + "hiddenSeries": false, + "id": 296, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "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": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_proxy_tikv_server_raft_append_rejects{}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Log Replication Rejected", + "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": "none", + "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, + "title": "Raft", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 9 +======= + "y": 13 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "id": 95, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 13 + }, + "hiddenSeries": false, + "id": 99, + "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": [ + { + "alias": "/^RS Filter/", + "yaxis": 2 + }, + { + "alias": "/^PK/", + "yaxis": 2 + }, + { + "alias": "/^No Filter/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "avg((rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]) - rate(tiflash_system_profile_event_DMFileFilterAftRoughSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) / (rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "1min-{{instance}}", + "refId": "B" + }, + { + "expr": "avg((rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) - rate(tiflash_system_profile_event_DMFileFilterAftRoughSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m])) / (rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]))) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "5min-{{instance}}", + "refId": "C" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_DMFileFilterNoFilter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": true, + "instant": false, + "intervalFactor": 1, + "legendFormat": "No Filter-{{instance}}", + "refId": "A" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": true, + "instant": false, + "intervalFactor": 1, + "legendFormat": "PK Filter-{{instance}}", + "refId": "D" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_DMFileFilterAftRoughSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "RS Filter-{{instance}}", + "refId": "E" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Rough Set Filter Rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "decimals": null, + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { "cardColor": "#b4ff00", "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", + "colorScheme": "interpolateOranges", "exponent": 0.5, - "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of SST to DT in pre-decode when applying region snapshot", "fieldConfig": { "defaults": {}, - "overrides": [] + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 13 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 97, + "legend": { + "show": false + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_storage_rough_set_filter_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "Rough Set Filter Rate Histogram", + "tooltip": { + "show": true, + "showHistogram": false + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "percent", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + } + ], + "title": "Rough Set Filter Rate Histogram", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, +<<<<<<< HEAD + "y": 10 +======= + "y": 14 + }, + "id": 274, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 0, + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "/delete_range|ingest/" + }, + "properties": [ + { + "id": "unit", + "value": "opm" + } + ] + } + ] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 14 + }, + "hiddenSeries": false, + "id": 286, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "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": [], + "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 by (type, instance) (\n tiflash_vector_index_active_instances{\n k8s_cluster=\"$k8s_cluster\", \n tidb_cluster=\"$tidb_cluster\", \n instance=~\"$instance\"\n }\n)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "In-Memory Vector Index Instances", + "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": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "ops", + "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}", + "decimals": 0, + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "/delete_range|ingest/" + }, + "properties": [ + { + "id": "unit", + "value": "opm" + } + ] + } + ] }, + "fill": 0, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 86 + "y": 14 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 127, + "hiddenSeries": false, + "id": 288, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": false, + "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": [], - "reverseYBuckets": false, + "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(delta(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"snapshot_predecode_sst2dt\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 2, - "legendFormat": "{{le}}", + "exemplar": true, + "expr": "tiflash_vector_index_memory_usage{\n k8s_cluster=\"$k8s_cluster\", \n tidb_cluster=\"$tidb_cluster\", \n instance=~\"$instance\"\n}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "tiflash_process_rss_by_type_bytes{\n k8s_cluster=\"$k8s_cluster\", \n tidb_cluster=\"$tidb_cluster\", \n instance=~\"$instance\",\n type=\"file\"\n}", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-RssFile", "refId": "B" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Snapshot Predecode SST to DT Duration", + "title": "Vector Index Estimated Memory Usage", "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 0, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "yaxes": [ + { + "decimals": 0, + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "description": "", "fieldConfig": { "defaults": {}, - "overrides": [] + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "/delete_range|ingest/" + }, + "properties": [ + { + "id": "unit", + "value": "opm" + } + ] + } + ] }, + "fill": 0, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 93 + "x": 0, + "y": 22 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 250, + "hiddenSeries": false, + "id": 282, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": false, + "hideEmpty": false, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true }, + "lines": true, + "linewidth": 1, "links": [], - "reverseYBuckets": false, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/download/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(delta(tiflash_raft_snapshot_total_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"approx_raft_snapshot\"}[1m])) by (le)", - "format": "heatmap", + "expr": "histogram_quantile(\n 0.999, \n sum(rate(\n tiflash_vector_index_duration_bucket{\n k8s_cluster=\"$k8s_cluster\", \n tidb_cluster=\"$tidb_cluster\", \n instance=~\"$instance\",\n type!=\"build\"\n }\n [$__rate_interval]\n )) by (le, type)\n)", + "format": "time_series", + "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Snapshot Size Heatmap", + "title": "99.9% Vector Search Duration (Per Request)", "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 0, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "bytes", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "yaxes": [ + { + "decimals": 1, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "decimals": 1, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, @@ -13019,26 +18532,38 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The keys flow of different kinds of Raft operations", + "description": "", "fieldConfig": { "defaults": {}, - "overrides": [] + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "/delete_range|ingest/" + }, + "properties": [ + { + "id": "unit", + "value": "opm" + } + ] + } + ] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 24, - "x": 0, - "y": 100 + "h": 8, + "w": 12, + "x": 12, + "y": 22 }, - "height": "", "hiddenSeries": false, - "id": 71, + "id": 284, "legend": { "alignAsTable": true, "avg": false, - "current": true, + "current": false, "hideEmpty": false, "hideZero": false, "max": true, @@ -13046,7 +18571,7 @@ "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", + "sort": "max", "sortDesc": true, "total": false, "values": true @@ -13054,7 +18579,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -13063,27 +18588,27 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_raft_process_keys{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "exemplar": true, + "expr": "histogram_quantile(\n 0.999, \n sum(rate(\n tiflash_vector_index_duration_bucket{\n k8s_cluster=\"$k8s_cluster\", \n tidb_cluster=\"$tidb_cluster\", \n instance=~\"$instance\",\n type=\"build\"\n }\n [$__rate_interval]\n )) by (le, type)\n)", "format": "time_series", "hide": false, - "intervalFactor": 2, + "interval": "", + "intervalFactor": 1, "legendFormat": "{{type}}", - "refId": "A", - "step": 10 + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Keys flow", + "title": "99.9% Vector Index Build Duration (Per DMFile Column)", "tooltip": { "shared": true, "sort": 0, @@ -13099,7 +18624,8 @@ }, "yaxes": [ { - "format": "short", + "decimals": 1, + "format": "s", "label": null, "logBase": 1, "max": null, @@ -13107,227 +18633,262 @@ "show": true }, { - "format": "short", + "decimals": 1, + "format": "s", "label": null, "logBase": 1, "max": null, "min": "0", - "show": true + "show": false } ], "yaxis": { "align": false, "alignLevel": null } - }, + } + ], + "title": "Vector Search", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 15 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) + }, + "id": 171, + "panels": [ { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of pre-decode when applying region snapshot", "fieldConfig": { "defaults": {}, "overrides": [] }, + "fill": 0, + "fillGradient": 0, "gridPos": { - "h": 7, - "w": 12, + "h": 8, + "w": 24, "x": 0, - "y": 107 + "y": 15 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 73, + "hiddenSeries": false, + "id": 173, "legend": { - "show": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(delta(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"snapshot_flush\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Snapshot Flush Duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "alignAsTable": true, + "avg": false, + "current": false, + "max": true, + "min": false, + "rightSide": true, "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of ingesting SST", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 107 + "sideWidth": 250, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 74, - "legend": { - "show": true + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true }, - "links": [], - "reverseYBuckets": false, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "sum(delta(tiflash_raft_command_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"ingest_sst\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_disaggregated_breakdown_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "interval": "", + "legendFormat": "99%-{{type}}", + "queryType": "randomWalk", + "refId": "A" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Ingest SST Duration", + "title": "Read Duration Breakdown", "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 2, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Latency that TiKV sends raft log to TiFlash.", + "description": "PageStorage Checkpoint Duration", "fieldConfig": { "defaults": {}, "overrides": [] }, + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 114 + "y": 23 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 92, + "hiddenSeries": false, + "id": 187, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true }, + "lines": true, + "linewidth": 1, "links": [], - "reverseYBuckets": false, + "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(delta(tiflash_raft_upstream_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "B" + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-99", + "refId": "C" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-90", + "refId": "D" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.80, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-80", + "refId": "E" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Upstream Latency [Heatmap]", + "title": "Checkpoint Upload Duration", "tooltip": { - "show": true, - "showHistogram": true - }, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 2, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, @@ -13335,32 +18896,34 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Latency that TiKV sends raft log to TiFlash.", - "editable": true, - "error": false, + "decimals": 1, + "description": "The flow of checkpoint operations", "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 1, "fillGradient": 0, - "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 114 + "y": 23 }, + "height": "", "hiddenSeries": false, - "id": 91, + "id": 174, "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, @@ -13369,7 +18932,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -13378,57 +18941,42 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_raft_upstream_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le) / 1000000000)", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_checkpoint_flow{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"incremental\"}[1m]))", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 2, - "legendFormat": " 100%", - "metric": "", + "legendFormat": "incremental", "refId": "A", - "step": 4 - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_upstream_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": " 99%", - "metric": "", - "refId": "B", - "step": 4 - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_upstream_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "95%", - "refId": "C", - "step": 4 + "step": 10 }, { - "expr": "sum(rate(tiflash_raft_upstream_latency_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) / sum(rate(tiflash_raft_upstream_latency_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) ", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_checkpoint_flow{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"compaction\"}[1m]))", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "avg", - "refId": "D", - "step": 4 + "interval": "", + "intervalFactor": 1, + "legendFormat": "compaction", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Upstream Latency", + "title": "Checkpoint Upload flow", "tooltip": { - "msResolution": false, "shared": true, - "sort": 0, - "value_type": "cumulative" + "sort": 2, + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -13440,7 +18988,7 @@ }, "yaxes": [ { - "format": "s", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -13452,7 +19000,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -13460,58 +19008,50 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Raft", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 9 - }, - "id": 95, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The keys of checkpoint operations. All keys are uploaded in the checkpoint. Grouped by key types.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 10 + "y": 31 }, + "height": "", "hiddenSeries": false, - "id": 99, + "id": 196, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "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 as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -13520,75 +19060,32 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/^RS Filter/", - "yaxis": 2 - }, - { - "alias": "/^PK/", - "yaxis": 2 - }, - { - "alias": "/^No Filter/", - "yaxis": 2 - } - ], + "repeatedByRow": true, + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "avg((rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]) - rate(tiflash_system_profile_event_DMFileFilterAftRoughSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) / (rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (instance)", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_checkpoint_keys_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "hide": false, - "intervalFactor": 1, - "legendFormat": "1min-{{instance}}", - "refId": "B" - }, - { - "expr": "avg((rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) - rate(tiflash_system_profile_event_DMFileFilterAftRoughSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m])) / (rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]))) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "5min-{{instance}}", - "refId": "C" - }, - { - "expr": "sum(rate(tiflash_system_profile_event_DMFileFilterNoFilter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "hide": true, - "instant": false, - "intervalFactor": 1, - "legendFormat": "No Filter-{{instance}}", - "refId": "A" - }, - { - "expr": "sum(rate(tiflash_system_profile_event_DMFileFilterAftPKAndPackSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "hide": true, - "instant": false, - "intervalFactor": 1, - "legendFormat": "PK Filter-{{instance}}", - "refId": "D" - }, - { - "expr": "sum(rate(tiflash_system_profile_event_DMFileFilterAftRoughSet{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "RS Filter-{{instance}}", - "refId": "E" + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Rough Set Filter Rate", + "title": "Checkpoint Upload keys speed by type (all)", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -13601,8 +19098,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "percentunit", + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -13610,12 +19106,11 @@ "show": true }, { - "decimals": null, "format": "short", - "label": "", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -13624,134 +19119,58 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateOranges", - "exponent": 0.5, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 10 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 97, - "legend": { - "show": false - }, - "pluginVersion": "6.1.6", - "reverseYBuckets": false, - "targets": [ - { - "exemplar": true, - "expr": "sum(delta(tiflash_storage_rough_set_filter_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", - "format": "heatmap", - "interval": "", - "legendFormat": "{{le}}", - "queryType": "randomWalk", - "refId": "A" - } - ], - "title": "Rough Set Filter Rate Histogram", - "tooltip": { - "show": true, - "showHistogram": false - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 0, - "format": "percent", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null - } - ], - "title": "Rough Set Filter Rate Histogram", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 10 - }, - "id": 171, - "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The flow of checkpoint operations. Group by key types", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, - "w": 24, - "x": 0, - "y": 11 + "w": 12, + "x": 12, + "y": 31 }, + "height": "", "hiddenSeries": false, - "id": 173, + "id": 197, "legend": { "alignAsTable": true, "avg": false, - "current": false, + "current": true, + "hideEmpty": 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 }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -13759,18 +19178,21 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_disaggregated_breakdown_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_storage_checkpoint_flow_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "hide": false, "interval": "", - "legendFormat": "99%-{{type}}", - "queryType": "randomWalk", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Duration Breakdown", + "title": "Checkpoint Upload flow by type (incremental+compaction)", "tooltip": { "shared": true, "sort": 2, @@ -13786,7 +19208,7 @@ }, "yaxes": [ { - "format": "s", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -13798,7 +19220,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -13813,31 +19235,34 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "PageStorage Checkpoint Duration", + "decimals": 1, + "description": "The number of files of owned by each TiFlash node", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 19 + "y": 39 }, "hiddenSeries": false, - "id": 187, + "id": 176, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, - "sort": "max", - "sortDesc": true, + "sideWidth": null, "total": false, "values": true }, @@ -13851,54 +19276,32 @@ "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_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, - "interval": "", - "legendFormat": "{{type}}-99", - "refId": "C" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, - "interval": "", - "legendFormat": "{{type}}-90", - "refId": "D" - }, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.80, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, + "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"num_files\"}) by (instance)", + "format": "time_series", + "hide": false, "interval": "", - "legendFormat": "{{type}}-80", - "refId": "E" + "intervalFactor": 2, + "legendFormat": "checkpoint_data-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Checkpoint Upload Duration", + "title": "Remote File Num", "tooltip": { + "msResolution": false, "shared": true, "sort": 2, "value_type": "individual" @@ -13913,7 +19316,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -13921,11 +19324,11 @@ "show": true }, { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, - "max": null, - "min": null, + "max": "1.1", + "min": "0", "show": true } ], @@ -13941,42 +19344,40 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow of checkpoint operations", + "description": "The remote store usage owned by each TiFlash node", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 19 + "y": 39 }, - "height": "", "hiddenSeries": false, - "id": 174, + "id": 175, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -13985,39 +19386,54 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/^valid_rate/", + "yaxis": 2 + }, + { + "alias": "/size/", + "linewidth": 3 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_checkpoint_flow{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"incremental\"}[1m]))", - "format": "time_series", + "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"total_size\"}) by (instance)", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "incremental", - "refId": "A", - "step": 10 + "legendFormat": "remote_size-{{instance}}", + "refId": "A" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_checkpoint_flow{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"compaction\"}[1m]))", - "format": "time_series", + "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"valid_size\"}) by (instance)", + "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "compaction", + "intervalFactor": 2, + "legendFormat": "valid_size-{{instance}}", "refId": "B" + }, + { + "exemplar": true, + "expr": "sum((tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"valid_size\"}) / (tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"total_size\"})) by (instance)", + "hide": true, + "interval": "", + "legendFormat": "valid_rate-{{instance}}", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Checkpoint Upload flow", + "title": "Remote Store Usage", "tooltip": { + "msResolution": false, "shared": true, "sort": 2, "value_type": "individual" @@ -14032,7 +19448,7 @@ }, "yaxes": [ { - "format": "binBps", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -14040,10 +19456,10 @@ "show": true }, { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, - "max": null, + "max": "1.1", "min": "0", "show": true } @@ -14059,36 +19475,30 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The keys of checkpoint operations. All keys are uploaded in the checkpoint. Grouped by key types.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 27 + "y": 47 }, - "height": "", "hiddenSeries": false, - "id": 196, + "id": 189, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, + "hideEmpty": true, + "hideZero": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -14104,7 +19514,6 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -14112,24 +19521,22 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_checkpoint_keys_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_disaggregated_object_lock_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "hide": false, "interval": "", - "intervalFactor": 2, + "intervalFactor": 1, "legendFormat": "{{type}}", - "refId": "A", - "step": 10 + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Checkpoint Upload keys speed by type (all)", + "title": "Remote Object Lock Request QPS", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -14142,7 +19549,8 @@ }, "yaxes": [ { - "format": "ops", + "decimals": null, + "format": "none", "label": null, "logBase": 1, "max": null, @@ -14150,11 +19558,11 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -14169,52 +19577,45 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of checkpoint operations. Group by key types", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 27 + "y": 47 }, - "height": "", "hiddenSeries": false, - "id": 197, + "id": 191, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": true, + "current": false, "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", + "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, + "pointradius": 2, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -14222,21 +19623,18 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_checkpoint_flow_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "hide": false, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_disaggregated_object_lock_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "interval": "", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 10 + "legendFormat": "99%-{{type}}", + "queryType": "randomWalk", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Checkpoint Upload flow by type (incremental+compaction)", + "title": "Remote Object Lock Duration", "tooltip": { "shared": true, "sort": 2, @@ -14252,7 +19650,7 @@ }, "yaxes": [ { - "format": "binBps", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -14264,7 +19662,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -14279,73 +19677,86 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The number of files of owned by each TiFlash node", - "editable": true, - "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 35 + "y": 55 }, "hiddenSeries": false, - "id": 176, + "id": 193, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "max": false, + "current": false, + "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, + "sort": "max", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "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": [], + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + }, + { + "alias": "/one_store/", + "yaxis": 2 + }, + { + "alias": "/clean_locks/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"num_files\"}) by (instance)", - "format": "time_series", - "hide": false, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_s3_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "interval": "", - "intervalFactor": 2, - "legendFormat": "checkpoint_data-{{instance}}", - "refId": "A", - "step": 10 + "legendFormat": "99%-{{type}}", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_s3_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "90%-{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote File Num", + "title": "Remote GC Duration Breakdown", "tooltip": { - "msResolution": false, "shared": true, "sort": 2, "value_type": "individual" @@ -14360,7 +19771,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -14368,11 +19779,11 @@ "show": true }, { - "format": "percentunit", + "format": "s", "label": null, "logBase": 1, - "max": "1.1", - "min": "0", + "max": null, + "min": null, "show": true } ], @@ -14388,7 +19799,7 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The remote store usage owned by each TiFlash node", + "description": "", "editable": true, "error": false, "fieldConfig": { @@ -14402,19 +19813,23 @@ "h": 8, "w": 12, "x": 12, - "y": 35 + "y": 55 }, "hiddenSeries": false, - "id": 175, + "id": 195, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": false, "max": false, "min": false, "rightSide": true, "show": true, "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -14423,59 +19838,34 @@ "links": [], "nullPointMode": "null as zero", "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/^valid_rate/", - "yaxis": 2 - }, - { - "alias": "/size/", - "linewidth": 3 - } - ], + "alertThreshold": false + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"total_size\"}) by (instance)", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "remote_size-{{instance}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"valid_size\"}) by (instance)", - "hide": false, + "expr": "sum(tiflash_storage_s3_gc_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance,type)", + "format": "time_series", "interval": "", "intervalFactor": 2, - "legendFormat": "valid_size-{{instance}}", - "refId": "B" - }, - { - "exemplar": true, - "expr": "sum((tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"valid_size\"}) / (tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"total_size\"})) by (instance)", - "hide": true, - "interval": "", - "legendFormat": "valid_rate-{{instance}}", - "refId": "C" + "legendFormat": "{{instance}}-{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote Store Usage", + "title": "Remote GC Status", "tooltip": { "msResolution": false, "shared": true, @@ -14492,7 +19882,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -14500,11 +19890,11 @@ "show": true }, { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, - "max": "1.1", - "min": "0", + "max": null, + "min": null, "show": true } ], @@ -14519,68 +19909,107 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "Request Unit for tidb-serverless charging", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 43 + "y": 63 }, "hiddenSeries": false, - "id": 189, + "id": 201, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, + "hideEmpty": false, "hideZero": false, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { - "alertThreshold": true + "alertThreshold": false }, "percentage": false, "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/sum/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_disaggregated_object_lock_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_storage_sync_replica_ru[1m])) by (keyspace_id)", "format": "time_series", "interval": "", - "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "replica-sync-rate-{{keyspace_id}}", + "refId": "A", + "step": 10 + }, + { + "exemplar": true, + "expr": "sum(increase(tiflash_storage_sync_replica_ru[24h])) by (keyspace_id)", + "hide": false, + "interval": "", + "legendFormat": "replica-sync-sum-24h-{{keyspace_id}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_compute_request_unit[1m])) by (cluster_id)", + "hide": false, + "interval": "", + "legendFormat": "query-rate-{{cluster_id}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(increase(tiflash_compute_request_unit[24h])) by (cluster_id)", + "hide": false, + "interval": "", + "legendFormat": "query-sum-24h-{{cluster_id}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote Object Lock Request QPS", + "title": "Request Unit", "tooltip": { + "msResolution": false, "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -14593,8 +20022,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", + "format": "cps", "label": null, "logBase": 1, "max": null, @@ -14602,11 +20030,11 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -14621,6 +20049,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Memory Usage of Storage Tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -14631,10 +20060,10 @@ "h": 8, "w": 12, "x": 12, - "y": 43 + "y": 63 }, "hiddenSeries": false, - "id": 191, + "id": 233, "legend": { "alignAsTable": true, "avg": false, @@ -14643,21 +20072,19 @@ "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -14667,21 +20094,45 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_disaggregated_object_lock_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(tiflash_system_current_metric_MemoryTrackingQueryStorageTask{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, "interval": "", - "legendFormat": "99%-{{type}}", - "queryType": "randomWalk", - "refId": "A" + "legendFormat": "MemoryTrackingQueryStorageTask-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_MemoryTrackingFetchPages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "MemoryTrackingFetchPages-{{instance}}", + "refId": "D" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_DT_DeltaIndexCacheSize{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "DeltaIndexCacheSize-{{instance}}", + "refId": "E" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_MemoryTrackingSharedColumnData{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "SharedColumnData-{{instance}}", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote Object Lock Duration", + "title": "Memory Usage of Storage Tasks", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -14694,7 +20145,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": null, + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -14702,12 +20154,12 @@ "show": true }, { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, - "min": null, - "show": true + "min": "0", + "show": false } ], "yaxis": { @@ -14721,6 +20173,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "DeltaIndex cache of ReadNodes", "fieldConfig": { "defaults": {}, "overrides": [] @@ -14731,46 +20184,36 @@ "h": 8, "w": 12, "x": 0, - "y": 51 + "y": 71 }, "hiddenSeries": false, - "id": 193, + "id": 236, "legend": { "alignAsTable": true, "avg": false, - "current": false, - "max": true, + "current": true, + "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/total/", - "yaxis": 2 - }, - { - "alias": "/one_store/", - "yaxis": 2 - }, - { - "alias": "/clean_locks/", + "alias": "/hit_ratio/", "yaxis": 2 } ], @@ -14780,29 +20223,31 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_s3_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_storage_delta_index_cache{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type, instance)", + "format": "time_series", + "hide": false, "interval": "", - "legendFormat": "99%-{{type}}", - "queryType": "randomWalk", - "refId": "A" + "intervalFactor": 1, + "legendFormat": "{{type}}-{{instance}}", + "refId": "B" }, { "exemplar": true, - "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_s3_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, + "expr": "sum(rate(tiflash_storage_delta_index_cache{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"hit\"}[1m])) by (instance) /sum(rate(tiflash_storage_delta_index_cache{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "hide": false, "interval": "", - "legendFormat": "90%-{{type}}", - "refId": "B" + "legendFormat": "hit_ratio-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote GC Duration Breakdown", + "title": "DeltaIndexCache", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -14815,7 +20260,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -14823,11 +20269,11 @@ "show": true }, { - "format": "s", + "format": "percentunit", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -14835,6 +20281,8 @@ "align": false, "alignLevel": null } +<<<<<<< HEAD +======= }, { "aliasColors": {}, @@ -14842,38 +20290,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, "description": "", - "editable": true, - "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 51 + "y": 71 }, "hiddenSeries": false, - "id": 195, + "id": 252, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -14882,38 +20321,42 @@ "links": [], "nullPointMode": "null as zero", "options": { - "alertThreshold": false + "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/hit_ratio/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(tiflash_storage_s3_gc_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance,type)", + "expr": "sum(rate(tiflash_fap_task_state{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", + "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote GC Status", + "title": "FAP state", "tooltip": { - "msResolution": false, "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -14926,7 +20369,8 @@ }, "yaxes": [ { - "format": "short", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -14934,11 +20378,11 @@ "show": true }, { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -14953,38 +20397,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "Request Unit for tidb-serverless charging", - "editable": true, - "error": false, + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 59 + "y": 79 }, "hiddenSeries": false, - "id": 201, + "id": 251, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -14993,7 +20428,7 @@ "links": [], "nullPointMode": "null as zero", "options": { - "alertThreshold": false + "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", @@ -15002,7 +20437,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/sum/", + "alias": "/hit_ratio/", "yaxis": 2 } ], @@ -15012,48 +20447,23 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_sync_replica_ru[1m])) by (keyspace_id)", + "expr": "sum(rate(tiflash_fap_task_result{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "replica-sync-rate-{{keyspace_id}}", - "refId": "A", - "step": 10 - }, - { - "exemplar": true, - "expr": "sum(increase(tiflash_storage_sync_replica_ru[24h])) by (keyspace_id)", "hide": false, "interval": "", - "legendFormat": "replica-sync-sum-24h-{{keyspace_id}}", + "intervalFactor": 1, + "legendFormat": "{{type}}", "refId": "B" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_compute_request_unit[1m])) by (cluster_id)", - "hide": false, - "interval": "", - "legendFormat": "query-rate-{{cluster_id}}", - "refId": "C" - }, - { - "exemplar": true, - "expr": "sum(increase(tiflash_compute_request_unit[24h])) by (cluster_id)", - "hide": false, - "interval": "", - "legendFormat": "query-sum-24h-{{cluster_id}}", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request Unit", + "title": "FAP result", "tooltip": { - "msResolution": false, "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -15066,7 +20476,8 @@ }, "yaxes": [ { - "format": "cps", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -15074,7 +20485,7 @@ "show": true }, { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -15093,7 +20504,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Memory Usage of Storage Tasks", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -15104,15 +20515,15 @@ "h": 8, "w": 12, "x": 12, - "y": 59 + "y": 79 }, "hiddenSeries": false, - "id": 233, + "id": 253, "legend": { "alignAsTable": true, "avg": false, - "current": false, - "max": true, + "current": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -15131,49 +20542,32 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/hit_ratio/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(tiflash_system_current_metric_MemoryTrackingQueryStorageTask{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "MemoryTrackingQueryStorageTask-{{instance}}", - "refId": "C" - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_MemoryTrackingFetchPages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "MemoryTrackingFetchPages-{{instance}}", - "refId": "D" - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_DT_DeltaIndexCacheSize{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "DeltaIndexCacheSize-{{instance}}", - "refId": "E" - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_MemoryTrackingSharedColumnData{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_fap_nomatch_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)\n", + "format": "time_series", "hide": false, "interval": "", - "legendFormat": "SharedColumnData-{{instance}}", - "refId": "F" + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory Usage of Storage Tasks", + "title": "FAP no match reason", "tooltip": { "shared": true, "sort": 0, @@ -15190,7 +20584,7 @@ "yaxes": [ { "decimals": null, - "format": "bytes", + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -15203,7 +20597,7 @@ "logBase": 1, "max": null, "min": "0", - "show": false + "show": true } ], "yaxis": { @@ -15217,7 +20611,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "DeltaIndex cache of ReadNodes", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -15228,10 +20622,10 @@ "h": 8, "w": 12, "x": 0, - "y": 67 + "y": 87 }, "hiddenSeries": false, - "id": 236, + "id": 254, "legend": { "alignAsTable": true, "avg": false, @@ -15267,28 +20661,20 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_delta_index_cache{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type, instance)", + "expr": "histogram_quantile(0.999, sum(round(1000000000*rate(tiflash_fap_task_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))) by (le, type) / 1000000000)", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}-{{instance}}", + "legendFormat": "{{type}}", "refId": "B" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_delta_index_cache{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"hit\"}[1m])) by (instance) /sum(rate(tiflash_storage_delta_index_cache{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "hit_ratio-{{instance}}", - "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "DeltaIndexCache", + "title": "FAP time by stage", "tooltip": { "shared": true, "sort": 0, @@ -15305,7 +20691,7 @@ "yaxes": [ { "decimals": null, - "format": "ops", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -15325,6 +20711,7 @@ "align": false, "alignLevel": null } +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) } ], "title": "Disaggregated", @@ -15337,7 +20724,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 11 +======= + "y": 16 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 172, "panels": [ @@ -15358,7 +20749,7 @@ "h": 8, "w": 12, "x": 0, - "y": 12 + "y": 16 }, "hiddenSeries": false, "id": 178, @@ -15476,7 +20867,7 @@ "h": 8, "w": 12, "x": 12, - "y": 12 + "y": 16 }, "hiddenSeries": false, "id": 179, @@ -15650,7 +21041,7 @@ "h": 8, "w": 12, "x": 0, - "y": 20 + "y": 24 }, "hiddenSeries": false, "id": 182, @@ -15768,7 +21159,7 @@ "h": 8, "w": 12, "x": 12, - "y": 20 + "y": 24 }, "hiddenSeries": false, "id": 180, @@ -15895,7 +21286,7 @@ "h": 8, "w": 12, "x": 0, - "y": 28 + "y": 32 }, "hiddenSeries": false, "id": 185, @@ -16022,7 +21413,7 @@ "h": 8, "w": 12, "x": 12, - "y": 28 + "y": 32 }, "hiddenSeries": false, "id": 186, @@ -16124,7 +21515,7 @@ "h": 8, "w": 12, "x": 0, - "y": 36 + "y": 40 }, "hiddenSeries": false, "id": 188, @@ -16243,7 +21634,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 12 +======= + "y": 17 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 202, "panels": [ @@ -16264,7 +21659,7 @@ "h": 8, "w": 12, "x": 0, - "y": 13 + "y": 17 }, "hiddenSeries": false, "id": 224, @@ -16364,7 +21759,7 @@ "h": 8, "w": 12, "x": 12, - "y": 13 + "y": 17 }, "hiddenSeries": false, "id": 226, @@ -16472,7 +21867,7 @@ "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 25 }, "hiddenSeries": false, "id": 228, @@ -16575,7 +21970,7 @@ "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 25 }, "hiddenSeries": false, "id": 222, @@ -16748,7 +22143,7 @@ "h": 8, "w": 12, "x": 0, - "y": 29 + "y": 33 }, "hiddenSeries": false, "id": 230, @@ -16903,7 +22298,7 @@ "h": 8, "w": 12, "x": 12, - "y": 29 + "y": 33 }, "hiddenSeries": false, "id": 218, @@ -17033,7 +22428,7 @@ "h": 8, "w": 12, "x": 0, - "y": 37 + "y": 41 }, "hiddenSeries": false, "id": 220, @@ -17163,7 +22558,7 @@ "h": 8, "w": 12, "x": 12, - "y": 37 + "y": 41 }, "hiddenSeries": false, "id": 216, @@ -17271,6 +22666,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", @@ -17283,7 +22789,11 @@ "h": 1, "w": 24, "x": 0, +<<<<<<< HEAD "y": 13 +======= + "y": 18 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "id": 248, "panels": [ @@ -17304,7 +22814,11 @@ "h": 8, "w": 12, "x": 0, +<<<<<<< HEAD "y": 14 +======= + "y": 18 +>>>>>>> 6344098691 (metrics: Enhance the o11y of TiFlash storage layer (#10275)) }, "hiddenSeries": false, "id": 246,