diff --git a/CMakeLists.txt b/CMakeLists.txt index 96dd43d73e4..0943c9f9699 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -24,6 +24,10 @@ set(CMAKE_MACOSX_RPATH 1) option(TIFLASH_ENABLE_LLVM_DEVELOPMENT "enable facilities for development with LLVM" OFF) +if(NOT CMAKE_PREFIX_PATH AND DEFINED ENV{CMAKE_PREFIX_PATH}) + message(STATUS "Reading CMAKE_PREFIX_PATH from env... $ENV{CMAKE_PREFIX_PATH}") + set(CMAKE_PREFIX_PATH "$ENV{CMAKE_PREFIX_PATH}") +endif() if(CMAKE_PREFIX_PATH) # append paths for cmake to check libs set(ENV{LD_LIBRARY_PATH} 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/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index a85954e478d..8702e6ea361 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -347,6 +347,13 @@ namespace DB F(type_fg_write_alloc_bytes, {"type", "fg_write_alloc_bytes"}), \ F(type_bg_write_req_bytes, {"type", "bg_write_req_bytes"}), \ F(type_bg_write_alloc_bytes, {"type", "bg_write_alloc_bytes"})) \ + M(tiflash_storage_io_limiter_curr, \ + "Current limit bytes per second of Storage I/O limiter", \ + Gauge, \ + F(type_fg_read_bytes, {"type", "fg_read_bytes"}), \ + F(type_bg_read_bytes, {"type", "bg_read_bytes"}), \ + F(type_fg_write_bytes, {"type", "fg_write_bytes"}), \ + F(type_bg_write_bytes, {"type", "bg_write_bytes"})) \ M(tiflash_storage_rough_set_filter_rate, \ "Bucketed histogram of rough set filter rate", \ Histogram, \ diff --git a/dbms/src/Encryption/RateLimiter.cpp b/dbms/src/Encryption/RateLimiter.cpp index 68999926ed8..5c5721d45b4 100644 --- a/dbms/src/Encryption/RateLimiter.cpp +++ b/dbms/src/Encryption/RateLimiter.cpp @@ -527,6 +527,7 @@ void IORateLimiter::updateReadLimiter(Int64 bg_bytes, Int64 fg_bytes) { bg_read_limiter->updateMaxBytesPerSec(bg_bytes); } + GET_METRIC(tiflash_storage_io_limiter_curr, type_bg_read_bytes).Set(bg_bytes); if (fg_bytes == 0) { @@ -540,6 +541,7 @@ void IORateLimiter::updateReadLimiter(Int64 bg_bytes, Int64 fg_bytes) { fg_read_limiter->updateMaxBytesPerSec(fg_bytes); } + GET_METRIC(tiflash_storage_io_limiter_curr, type_fg_read_bytes).Set(fg_bytes); } void IORateLimiter::updateWriteLimiter(Int64 bg_bytes, Int64 fg_bytes) @@ -557,6 +559,7 @@ void IORateLimiter::updateWriteLimiter(Int64 bg_bytes, Int64 fg_bytes) { bg_write_limiter->updateMaxBytesPerSec(bg_bytes); } + GET_METRIC(tiflash_storage_io_limiter_curr, type_bg_write_bytes).Set(bg_bytes); if (fg_bytes == 0) { @@ -570,6 +573,7 @@ void IORateLimiter::updateWriteLimiter(Int64 bg_bytes, Int64 fg_bytes) { fg_write_limiter->updateMaxBytesPerSec(fg_bytes); } + GET_METRIC(tiflash_storage_io_limiter_curr, type_fg_write_bytes).Set(fg_bytes); } void IORateLimiter::setBackgroundThreadIds(std::vector thread_ids) 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/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index d15e5887519..97bff2e4abe 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -68,7 +68,7 @@ class InterpreterCreateQuery : public IInterpreter ASTPtr query_ptr; Context & context; - std::string_view log_suffix; + std::string log_suffix; /// Using while loading database. ThreadPool * thread_pool = nullptr; diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h index 4c9282566bf..35fb473f15c 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile.h @@ -65,9 +65,9 @@ class ColumnFile : id(++MAX_COLUMN_FILE_ID) {} +public: virtual ~ColumnFile() = default; -public: enum Type : UInt32 { DELETE_RANGE = 1, @@ -96,8 +96,9 @@ class ColumnFile UInt64 getId() const { return id; } virtual size_t getRows() const { return 0; } - 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; } virtual Type getType() const = 0; @@ -132,14 +133,19 @@ class ColumnFile virtual ColumnFileReaderPtr getReader( const DMContext & context, const IColumnFileDataProviderPtr & data_provider, - const ColumnDefinesPtr & col_defs) const - = 0; + const ColumnDefinesPtr & col_defs) const = 0; /// Note: Only ColumnFileInMemory can be appendable. Other ColumnFiles (i.e. ColumnFilePersisted) have /// 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..61a7281f480 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileInMemory.h @@ -60,7 +60,8 @@ class ColumnFileInMemory : public ColumnFile Type getType() const override { return Type::INMEMORY_FILE; } size_t getRows() const override { return rows; } - size_t getBytes() const override { return bytes; }; + size_t getBytes() const override { return bytes; } + size_t getAllocateBytes() const override { return cache->block.allocatedBytes(); } CachePtr getCache() { return cache; } @@ -75,9 +76,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..0329199e730 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp @@ -22,10 +22,79 @@ #include #include -namespace DB + +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) { -namespace DM + 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); +} + void MemTableSet::appendColumnFileInner(const ColumnFilePtr & column_file) { if (!column_files.empty()) @@ -38,11 +107,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 +252,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) @@ -244,9 +320,9 @@ ColumnFileSetSnapshotPtr MemTableSet::createSnapshot( column_files.back()->disableAppend(); auto snap = std::make_shared(data_provider); - snap->rows = rows; - snap->bytes = bytes; - snap->deletes = deletes; + snap->rows = stat.rows; + snap->bytes = stat.bytes; + snap->deletes = stat.deletes; snap->column_files.reserve(column_files.size()); size_t total_rows = 0; @@ -273,11 +349,11 @@ 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()); return snap; } @@ -312,7 +388,7 @@ ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( cur_rows_offset += column_file->getRows(); cur_deletes_offset += column_file->getDeletes(); } - if (unlikely(flush_task->getFlushRows() != rows || flush_task->getFlushDeletes() != deletes)) + if (unlikely(flush_task->getFlushRows() != stat.rows || flush_task->getFlushDeletes() != stat.deletes)) { LOG_ERROR( log, @@ -320,8 +396,8 @@ ColumnFileFlushTaskPtr MemTableSet::buildFlushTask( "Files: {}", flush_task->getFlushRows(), flush_task->getFlushDeletes(), - rows.load(), - deletes.load(), + stat.rows.load(), + stat.deletes.load(), columnFilesToString(column_files)); throw Exception("Rows and deletes check failed.", ErrorCodes::LOGICAL_ERROR); } @@ -345,6 +421,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,15 +429,17 @@ 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); } -} // namespace DM -} // namespace DB +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.h index bd8cf9f3ced..1588ffe4e2a 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; @@ -35,14 +33,55 @@ class MemTableSet , private boost::noncopyable { private: + struct Statistic + { + // TODO: check the proper memory_order when use this atomic variable + std::atomic column_files_count = 0; + std::atomic rows = 0; + std::atomic bytes = 0; + std::atomic allocated_bytes = 0; + std::atomic deletes = 0; + + CurrentMetrics::Increment holder_bytes; + CurrentMetrics::Increment holder_allocated_bytes; + + Statistic(); + + String info() const + { + return fmt::format( + "MemTableSet: {} column files, {} rows, {} bytes, {} deletes", + column_files_count.load(), + rows.load(), + bytes.load(), + deletes.load()); + } + + void append( + size_t rows_added, + size_t bytes_added, + size_t allocated_bytes_added, + size_t deletes_added, + size_t files_added); + + void resetTo( + size_t new_column_files_count, + size_t new_rows, + size_t new_bytes, + size_t new_allocated_bytes, + size_t new_deletes); + }; + +private: + // Keep track of the number of mem-table in memory. + CurrentMetrics::Increment holder_counter; + // Note that we must update `column_files_count` for outer thread-safe after `column_files` changed ColumnFiles column_files; - // TODO: check the proper memory_order when use this atomic variable - std::atomic column_files_count; - std::atomic rows = 0; - std::atomic bytes = 0; - std::atomic deletes = 0; + // In order to avoid data-race and make it lightweight for accessing the statistic + // of mem-table, we use atomic variables to track the state of this MemTableSet. + Statistic stat; LoggerPtr log; @@ -50,18 +89,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 +99,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 +170,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..ac12cdb1aa9 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -95,6 +95,7 @@ struct SegmentStats 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; @@ -111,6 +112,7 @@ using SegmentsStats = std::vector; struct StoreStats { + UInt64 column_count = 0; UInt64 segment_count = 0; UInt64 total_rows = 0; @@ -122,6 +124,7 @@ struct StoreStats Float64 delta_placed_rate = 0; UInt64 delta_cache_size = 0; + UInt64 delta_cache_alloc_size = 0; Float64 delta_cache_rate = 0; Float64 delta_cache_wasted_rate = 0; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp index 411d09a9f22..0e7936330eb 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -17,9 +17,7 @@ #include #include -namespace DB -{ -namespace DM +namespace DB::DM { StoreStats DeltaMergeStore::getStoreStats() @@ -31,11 +29,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 +64,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 +88,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 +155,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 +164,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 +202,4 @@ SegmentsStats DeltaMergeStore::getSegmentsStats() } -} // namespace DM -} // namespace DB +} // 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/NotIn.h b/dbms/src/Storages/DeltaMerge/Filter/NotIn.h index ece4aa7225e..41bc8b453a9 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/NotIn.h +++ b/dbms/src/Storages/DeltaMerge/Filter/NotIn.h @@ -46,6 +46,20 @@ class NotIn : 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 { RSResults res(pack_count, RSResult::Some); diff --git a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h index 97cdcc6d570..a049df435f2 100644 --- a/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h +++ b/dbms/src/Storages/DeltaMerge/Filter/PushDownFilter.h @@ -14,6 +14,11 @@ #pragma once +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#pragma GCC diagnostic pop + #include #include @@ -48,6 +53,16 @@ class PushDownFilter : public std::enable_shared_from_this : rs_operator(rs_operator_) {} + Poco::JSON::Object::Ptr toJSONObject() const + { + Poco::JSON::Object::Ptr json = new Poco::JSON::Object(); + if (rs_operator) + { + json->set("rs_operator", rs_operator->toJSONObject()); + } + return json; + } + // Rough set operator RSOperatorPtr rs_operator; // Filter expression actions and the name of the tmp filter column @@ -55,7 +70,7 @@ class PushDownFilter : public std::enable_shared_from_this const ExpressionActionsPtr before_where; // The projection after the filter, used to remove the tmp filter column // Used to construct the ExpressionBlockInputStream - // Note: ususally we will remove the tmp filter column in the LateMaterializationBlockInputStream, this only used for unexpected cases + // Note: usually we will remove the tmp filter column in the LateMaterializationBlockInputStream, this only used for unexpected cases const ExpressionActionsPtr project_after_where; const String filter_column_name; // The columns needed by the filter expression diff --git a/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h b/dbms/src/Storages/DeltaMerge/Filter/RSOperator.h index 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/ScanContext.cpp b/dbms/src/Storages/DeltaMerge/ScanContext.cpp index 8dae573e5d8..e0aba165af6 100644 --- a/dbms/src/Storages/DeltaMerge/ScanContext.cpp +++ b/dbms/src/Storages/DeltaMerge/ScanContext.cpp @@ -17,7 +17,9 @@ #pragma GCC diagnostic ignored "-Wunused-parameter" #include #pragma GCC diagnostic pop + #include +#include #include #include @@ -155,6 +157,11 @@ String ScanContext::toJson() const }; json->set("region_num_of_instance", to_json_array(region_num_of_instance)); + if (pushdown_executor) + { + json->set("pushdown", pushdown_executor->toJSONObject()); + } + std::stringstream buf; json->stringify(buf); return buf.str(); diff --git a/dbms/src/Storages/DeltaMerge/ScanContext.h b/dbms/src/Storages/DeltaMerge/ScanContext.h index 78321dd8737..8fd44641874 100644 --- a/dbms/src/Storages/DeltaMerge/ScanContext.h +++ b/dbms/src/Storages/DeltaMerge/ScanContext.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -28,6 +29,8 @@ namespace DB::DM { +class PushDownFilter; +using PushDownFilterPtr = std::shared_ptr; /// ScanContext is used to record statistical information in table scan for current query. /// For each table scan(one executor id), there is only one ScanContext. /// ScanContext helps to collect the statistical information of the table scan to show in `EXPLAIN ANALYZE`. @@ -85,6 +88,7 @@ class ScanContext std::atomic build_bitmap_time_ns{0}; const String resource_group_name; + PushDownFilterPtr 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/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..a4bd7098bae 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -48,6 +48,7 @@ #include #include #include +#include #include #include #include @@ -62,6 +63,10 @@ #include #include +namespace CurrentMetrics +{ +extern const Metric DT_NumStorageDeltaMerge; +} // namespace CurrentMetrics namespace DB { @@ -89,6 +94,7 @@ StorageDeltaMerge::StorageDeltaMerge( Context & global_context_) : IManageableStorage{columns_, tombstone} , data_path_contains_database_name(db_engine != "TiFlash") + , holder_counter(CurrentMetrics::DT_NumStorageDeltaMerge, 1) , store_inited(false) , max_column_id_used(0) , global_context(global_context_.getGlobalContext()) @@ -977,6 +983,7 @@ BlockInputStreams StorageDeltaMerge::read( auto runtime_filter_list = parseRuntimeFilterList(query_info, context); const auto & scan_context = mvcc_query_info.scan_context; + scan_context->pushdown_executor = filter; auto streams = store->read( context, @@ -1071,6 +1078,7 @@ void StorageDeltaMerge::read( auto runtime_filter_list = parseRuntimeFilterList(query_info, context); const auto & scan_context = mvcc_query_info.scan_context; + scan_context->pushdown_executor = filter; store->read( exec_context_, diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index dcb2d9e08e5..d130705646d 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. @@ -317,6 +320,7 @@ class StorageDeltaMerge std::atomic next_version = 1; //TODO: remove this!!! + Context & global_context; LoggerPtr log; diff --git a/dbms/src/Storages/System/StorageSystemDTSegments.cpp b/dbms/src/Storages/System/StorageSystemDTSegments.cpp index c10165141c2..221ee6fa85d 100644 --- a/dbms/src/Storages/System/StorageSystemDTSegments.cpp +++ b/dbms/src/Storages/System/StorageSystemDTSegments.cpp @@ -57,6 +57,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 +141,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..8361a325000 100644 --- a/dbms/src/TestUtils/gtests_dbms_main.cpp +++ b/dbms/src/TestUtils/gtests_dbms_main.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include @@ -66,7 +67,8 @@ int main(int argc, char ** argv) install_fault_signal_handlers({SIGSEGV, SIGILL, SIGFPE, SIGABRT, SIGTERM}); bool enable_colors = isatty(STDERR_FILENO) && isatty(STDOUT_FILENO); - DB::tests::TiFlashTestEnv::setupLogger("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..2c46bd1ae82 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -79,6 +79,7 @@ #include #include #include +#include #include #include #include @@ -622,7 +623,8 @@ static void terminate_handler() log << "what(): " << e.what() << std::endl; } catch (...) - {} + { + } log << "Stack trace:\n\n" << StackTrace().toString() << std::endl; } @@ -672,21 +674,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 +748,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_proxy_details.json b/metrics/grafana/tiflash_proxy_details.json index f33fdb0c0df..72bc556b24a 100644 --- a/metrics/grafana/tiflash_proxy_details.json +++ b/metrics/grafana/tiflash_proxy_details.json @@ -14,7 +14,7 @@ "type": "grafana", "id": "grafana", "name": "Grafana", - "version": "7.5.11" + "version": "7.5.17" }, { "type": "panel", @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1670499325053, + "iteration": 1742544258476, "links": [], "panels": [ { @@ -113,7 +113,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -220,7 +220,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -327,7 +327,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -434,7 +434,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -541,7 +541,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -662,7 +662,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -725,38 +725,1641 @@ { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The memory usage of raft entry cache per TiFlash instance", + "description": null, + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 25 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4560, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "((\n tiflash_proxy_tikv_server_mem_trace_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",name=~\"raftstore-.*\"}\n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{name}}", + "metric": "", + "query": "((\n tikv_server_mem_trace_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",name=~\"raftstore-.*\"}\n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "((\n tiflash_proxy_raft_engine_memory_usage\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-raft-engine", + "metric": "", + "query": "((\n raft_engine_memory_usage\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) ", + "refId": "B", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Memory trace", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + } + ], + "repeat": null, + "title": "Cluster", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 1 + }, + "id": 4558, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The count of operations per second", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 2 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4538, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_apply_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "write {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_apply_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_read_entry_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "read_entry {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_read_entry_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_read_message_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "read_message {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_read_message_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "C", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Operation", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time used in write operation", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 2 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4540, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [ + { + "alias": "/^count/", + "bars": false, + "dashLength": 1, + "dashes": true, + "fill": 2, + "fillBelowTo": null, + "lines": true, + "spaceLength": 1, + "transform": "negative-Y", + "yaxis": 2, + "zindex": -3 + }, + { + "alias": "/^avg/", + "bars": false, + "fill": 7, + "fillBelowTo": null, + "lines": true, + "yaxis": 1, + "zindex": 0 + } + ], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.9999,(\n sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99.99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.9999,(\n sum(rate(\n raft_engine_write_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "(sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg {{$additional_groupby}}", + "metric": "", + "query": "(sum(rate(\n raft_engine_write_duration_seconds_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n raft_engine_write_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "refId": "C", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": true, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "count {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "D", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Duration", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The I/O flow rate", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 9 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4554, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_size_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "write {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_size_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_background_rewrite_bytes_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "rewrite-{{type}} {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_background_rewrite_bytes_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "refId": "B", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Flow", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "99% duration breakdown of write operation", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 9 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4556, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_preprocess_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "wait {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_preprocess_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_leader_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "wal {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_leader_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_apply_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "apply {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_apply_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "C", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Duration Breakdown (99%)", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The bytes per write", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 16 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4550, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [ + { + "alias": "/^count/", + "bars": false, + "dashLength": 1, + "dashes": true, + "fill": 2, + "fillBelowTo": null, + "lines": true, + "spaceLength": 1, + "transform": "negative-Y", + "yaxis": 2, + "zindex": -3 + }, + { + "alias": "/^avg/", + "bars": false, + "fill": 7, + "fillBelowTo": null, + "lines": true, + "yaxis": 1, + "zindex": 0 + } + ], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.9999,(\n sum(rate(\n tiflash_proxy_raft_engine_write_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99.99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.9999,(\n sum(rate(\n raft_engine_write_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "(sum(rate(\n tiflash_proxy_raft_engine_write_size_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n tiflash_proxy_raft_engine_write_size_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg {{$additional_groupby}}", + "metric": "", + "query": "(sum(rate(\n raft_engine_write_size_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n raft_engine_write_size_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "refId": "C", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_size_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by () ", + "format": "time_series", + "hide": true, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "count {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_size_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "D", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Bytes / Written", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "999% duration breakdown of WAL write operation", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 16 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4552, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_write_leader_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_write_leader_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_sync_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "sync {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_sync_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_allocate_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "allocate {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_allocate_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "C", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_rotate_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "rotate {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_rotate_log_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "D", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "WAL Duration Breakdown (999%)", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The average number of files", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 23 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4546, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "avg((\n tiflash_proxy_raft_engine_log_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "avg((\n raft_engine_log_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "avg((\n tiflash_proxy_raft_engine_swap_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "swap {{$additional_groupby}}", + "metric": "", + "query": "avg((\n raft_engine_swap_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by ($additional_groupby) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "avg((\n tiflash_proxy_raft_engine_recycled_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}-recycle {{$additional_groupby}}", + "metric": "", + "query": "avg((\n raft_engine_recycled_file_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "refId": "C", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "File Count", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The 99% duration of operations other than write", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 23 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4548, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_read_entry_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "read_entry {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_read_entry_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_read_message_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "read_message {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_read_message_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.999,(\n sum(rate(\n tiflash_proxy_raft_engine_purge_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "purge {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.999,(\n sum(rate(\n raft_engine_purge_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "C", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Other Durations (99%)", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The average number of log entries", "editable": true, "error": false, "fieldConfig": { - "defaults": {}, + "defaults": { + "links": [] + }, "overrides": [] }, - "fill": 0, - "fillGradient": 0, - "grid": {}, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 25 + "y": 30 }, + "height": null, "hiddenSeries": false, - "id": 4536, + "hideTimeOverride": false, + "id": 4542, + "interval": null, + "isNew": true, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", + "sort": "max", "sortDesc": true, "total": false, "values": true @@ -764,42 +2367,54 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", + "repeat": null, + "repeatDirection": null, "seriesOverrides": [], "spaceLength": 10, + "span": null, "stack": false, "steppedLine": false, "targets": [ { + "datasource": "${DS_TEST-CLUSTER}", "exemplar": true, - "expr": "tiflash_proxy_tikv_server_mem_trace_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"raftstore-.*\"}", + "expr": "avg((\n tiflash_proxy_raft_engine_log_entry_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", "format": "time_series", + "hide": false, + "instant": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "{{name}} {{instance}}", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "avg((\n raft_engine_log_entry_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", "refId": "A", - "step": 10 + "step": 10, + "target": "" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Entry Cache", + "title": "Entry Count", "tooltip": { - "msResolution": false, + "msResolution": true, "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, + "transformations": [], "type": "graph", "xaxis": { "buckets": null, @@ -810,14 +2425,16 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": null, + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { + "decimals": null, "format": "short", "label": null, "logBase": 1, @@ -828,12 +2445,218 @@ ], "yaxis": { "align": false, - "alignLevel": null + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The compression ratio per write", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 30 + }, + "height": null, + "hiddenSeries": false, + "hideTimeOverride": false, + "id": 4544, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [ + { + "alias": "/^count/", + "bars": false, + "dashLength": 1, + "dashes": true, + "fill": 2, + "fillBelowTo": null, + "lines": true, + "spaceLength": 1, + "transform": "negative-Y", + "yaxis": 2, + "zindex": -3 + }, + { + "alias": "/^avg/", + "bars": false, + "fill": 7, + "fillBelowTo": null, + "lines": true, + "yaxis": 1, + "zindex": 0 + } + ], + "spaceLength": 10, + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.9999,(\n sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99.99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.9999,(\n sum(rate(\n raft_engine_write_compression_ratio_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "A", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "99% {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n raft_engine_write_compression_ratio_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "B", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "(sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg {{$additional_groupby}}", + "metric": "", + "query": "(sum(rate(\n raft_engine_write_compression_ratio_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) / sum(rate(\n raft_engine_write_compression_ratio_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) )", + "refId": "C", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "exemplar": true, + "expr": "sum(rate(\n tiflash_proxy_raft_engine_write_compression_ratio_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "count {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n raft_engine_write_compression_ratio_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by ($additional_groupby) ", + "refId": "D", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Compression Ratio", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 } } ], - "repeat": null, - "title": "Cluster", + "title": "Raft Engine", "type": "row" }, { @@ -843,7 +2666,7 @@ "h": 1, "w": 24, "x": 0, - "y": 1 + "y": 2 }, "id": 2743, "panels": [ @@ -1864,7 +3687,7 @@ "h": 1, "w": 24, "x": 0, - "y": 2 + "y": 3 }, "id": 2744, "panels": [ @@ -3102,7 +4925,7 @@ "h": 1, "w": 24, "x": 0, - "y": 3 + "y": 4 }, "id": 2745, "panels": [ @@ -3715,7 +5538,7 @@ "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 5 }, "id": 2746, "panels": [ @@ -5275,7 +7098,7 @@ "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 6 }, "id": 2747, "panels": [ @@ -5287,13 +7110,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of requests that TiKV sends to PD", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 6 + "y": 7 }, + "hiddenSeries": false, "id": 1069, "legend": { "alignAsTable": true, @@ -5313,7 +7142,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5380,13 +7213,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed by requests that TiKV sends to PD", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 6 + "y": 7 }, + "hiddenSeries": false, "id": 1070, "legend": { "alignAsTable": true, @@ -5406,7 +7245,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5473,13 +7316,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": " \tThe total number of PD heartbeat messages", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 14 + "y": 15 }, + "hiddenSeries": false, "id": 1215, "legend": { "alignAsTable": true, @@ -5499,7 +7348,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5566,13 +7419,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The total number of peers validated by the PD worker", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 14 + "y": 15 }, + "hiddenSeries": false, "id": 1396, "legend": { "alignAsTable": true, @@ -5592,7 +7451,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5664,7 +7527,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 7 }, "id": 2748, "panels": [ @@ -5677,14 +7540,20 @@ "description": "The time consumed when Raft applies log", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 7 + "y": 8 }, + "hiddenSeries": false, "id": 31, "legend": { "alignAsTable": true, @@ -5703,7 +7572,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5789,14 +7662,20 @@ "description": "The time consumed for Raft to apply logs per TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 7 + "y": 8 }, + "hiddenSeries": false, "id": 32, "legend": { "alignAsTable": true, @@ -5815,7 +7694,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5884,14 +7767,20 @@ "description": "The time consumed when Raft appends log", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 14 + "y": 15 }, + "hiddenSeries": false, "id": 39, "legend": { "alignAsTable": true, @@ -5910,7 +7799,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -5996,14 +7889,20 @@ "description": "The time consumed when Raft appends log on each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 14 + "y": 15 }, + "hiddenSeries": false, "id": 40, "legend": { "alignAsTable": true, @@ -6022,7 +7921,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6089,13 +7992,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The time consumed when Raft commits log", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 22 }, + "hiddenSeries": false, "id": 3690, "legend": { "alignAsTable": true, @@ -6112,7 +8021,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -6191,13 +8104,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The time consumed when Raft commits log on each TiKV instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 22 }, + "hiddenSeries": false, "id": 3688, "legend": { "alignAsTable": true, @@ -6214,7 +8133,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -6284,7 +8207,7 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 8 }, "id": 2749, "panels": [ @@ -6298,14 +8221,20 @@ "description": "The count of different ready type of Raft", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 8 + "y": 9 }, + "hiddenSeries": false, "id": 5, "legend": { "alignAsTable": true, @@ -6325,7 +8254,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6404,14 +8337,20 @@ "description": "The time consumed for peer processes to be ready in Raft", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 8 + "y": 9 }, + "hiddenSeries": false, "id": 118, "legend": { "alignAsTable": true, @@ -6431,7 +8370,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6509,14 +8452,20 @@ "description": "The time consumed by raftstore events (P99).99", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 17 }, + "hiddenSeries": false, "id": 123, "legend": { "alignAsTable": true, @@ -6536,7 +8485,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6616,7 +8569,7 @@ "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 9 }, "id": 2750, "panels": [ @@ -6630,14 +8583,20 @@ "description": "The number of Raft messages sent by each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 47 + "y": 10 }, + "hiddenSeries": false, "id": 1615, "legend": { "alignAsTable": true, @@ -6657,7 +8616,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6727,14 +8690,20 @@ "description": "The number of Raft messages flushed by each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 47 + "y": 10 }, + "hiddenSeries": false, "id": 1616, "legend": { "alignAsTable": true, @@ -6754,7 +8723,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6823,13 +8796,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The number of Raft messages received by each TiKV instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 54 + "y": 17 }, + "hiddenSeries": false, "id": 106, "legend": { "alignAsTable": true, @@ -6849,7 +8828,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -6918,14 +8901,20 @@ "description": "The number of different types of Raft messages that are sent", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 54 + "y": 17 }, + "hiddenSeries": false, "id": 11, "legend": { "alignAsTable": true, @@ -6945,7 +8934,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7015,14 +9008,20 @@ "description": "The total number of vote messages that are sent in Raft", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 61 + "y": 24 }, + "hiddenSeries": false, "id": 25, "legend": { "alignAsTable": true, @@ -7042,7 +9041,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7112,14 +9115,20 @@ "description": "The number of dropped Raft messages per type", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 61 + "y": 24 }, + "hiddenSeries": false, "id": 1309, "legend": { "alignAsTable": true, @@ -7139,7 +9148,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7211,7 +9224,7 @@ "h": 1, "w": 24, "x": 0, - "y": 9 + "y": 10 }, "id": 2751, "panels": [ @@ -7223,13 +9236,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The proposal count of all Regions in a mio tick", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 48 + "y": 11 }, + "hiddenSeries": false, "id": 108, "legend": { "alignAsTable": true, @@ -7249,7 +9268,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7319,14 +9342,20 @@ "description": "The number of proposals per type", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 48 + "y": 11 }, + "hiddenSeries": false, "id": 7, "legend": { "alignAsTable": true, @@ -7346,7 +9375,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7417,14 +9450,20 @@ "description": "The number of read proposals which are made by each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 55 + "y": 18 }, + "hiddenSeries": false, "id": 119, "legend": { "alignAsTable": true, @@ -7444,7 +9483,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7515,14 +9558,20 @@ "description": "The number of write proposals which are made by each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 55 + "y": 18 }, + "hiddenSeries": false, "id": 120, "legend": { "alignAsTable": true, @@ -7542,7 +9591,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7612,14 +9665,20 @@ "description": "The wait time of each proposal", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 62 + "y": 25 }, + "hiddenSeries": false, "id": 41, "legend": { "alignAsTable": true, @@ -7638,7 +9697,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7724,14 +9787,20 @@ "description": "The wait time of each proposal in each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 62 + "y": 25 }, + "hiddenSeries": false, "id": 42, "legend": { "alignAsTable": true, @@ -7750,7 +9819,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7818,14 +9891,20 @@ "datasource": "${DS_TEST-CLUSTER}", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 69 + "y": 32 }, + "hiddenSeries": false, "id": 2535, "legend": { "alignAsTable": true, @@ -7844,7 +9923,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -7929,14 +10012,20 @@ "datasource": "${DS_TEST-CLUSTER}", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 69 + "y": 32 }, + "hiddenSeries": false, "id": 2536, "legend": { "alignAsTable": true, @@ -7955,7 +10044,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8022,13 +10115,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The rate at which peers propose logs", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 76 + "y": 39 }, + "hiddenSeries": false, "id": 1975, "legend": { "alignAsTable": true, @@ -8047,7 +10146,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8114,14 +10217,20 @@ "datasource": "${DS_TEST-CLUSTER}", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 76 + "y": 39 }, + "hiddenSeries": false, "id": 4375, "legend": { "alignAsTable": true, @@ -8140,7 +10249,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8221,7 +10334,7 @@ "h": 1, "w": 24, "x": 0, - "y": 10 + "y": 11 }, "id": 2752, "panels": [ @@ -8235,14 +10348,20 @@ "description": "The number of admin proposals", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 11 + "y": 12 }, + "hiddenSeries": false, "id": 76, "legend": { "alignAsTable": true, @@ -8262,7 +10381,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8333,14 +10456,20 @@ "description": "The number of the processed apply command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 11 + "y": 12 }, + "hiddenSeries": false, "id": 77, "legend": { "alignAsTable": true, @@ -8360,7 +10489,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8431,14 +10564,20 @@ "description": "The number of raftstore split checksss", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 18 + "y": 19 }, + "hiddenSeries": false, "id": 70, "legend": { "alignAsTable": true, @@ -8458,7 +10597,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8529,14 +10672,20 @@ "description": "The time consumed when running split check in .9999", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 18 + "y": 19 }, + "hiddenSeries": false, "id": 71, "legend": { "alignAsTable": true, @@ -8557,7 +10706,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -8630,7 +10783,7 @@ "h": 1, "w": 24, "x": 0, - "y": 11 + "y": 12 }, "id": 4200, "panels": [ @@ -8641,13 +10794,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The time used by each level in the unified read pool per second. Level 0 refers to small queries.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, "y": 13 }, + "hiddenSeries": false, "id": 4194, "legend": { "alignAsTable": true, @@ -8665,7 +10824,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -8730,13 +10893,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The chance that level 0 (small) tasks are scheduled in the unified read pool.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, "y": 13 }, + "hiddenSeries": false, "id": 4196, "legend": { "alignAsTable": true, @@ -8753,7 +10922,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -8818,13 +10991,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The number of concurrently running tasks in the unified read pool.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, "y": 21 }, + "hiddenSeries": false, "id": 4198, "legend": { "alignAsTable": true, @@ -8841,7 +11020,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": false, "renderer": "flot", @@ -8911,7 +11094,7 @@ "h": 1, "w": 24, "x": 0, - "y": 12 + "y": 13 }, "id": 2754, "panels": [ @@ -8925,7 +11108,12 @@ "description": "The total count of different kinds of commands received", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -8933,6 +11121,7 @@ "x": 0, "y": 14 }, + "hiddenSeries": false, "id": 2, "legend": { "alignAsTable": true, @@ -8954,7 +11143,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9024,7 +11217,12 @@ "description": "The total number of engine asynchronous request errors", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9032,6 +11230,7 @@ "x": 12, "y": 14 }, + "hiddenSeries": false, "id": 8, "legend": { "alignAsTable": true, @@ -9053,7 +11252,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9124,7 +11327,12 @@ "description": "The time consumed by processing asynchronous snapshot requests", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9132,6 +11340,7 @@ "x": 0, "y": 22 }, + "hiddenSeries": false, "id": 15, "legend": { "alignAsTable": true, @@ -9152,7 +11361,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9238,7 +11451,12 @@ "description": "The time consumed by processing asynchronous write requests", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9246,6 +11464,7 @@ "x": 12, "y": 22 }, + "hiddenSeries": false, "id": 109, "legend": { "alignAsTable": true, @@ -9266,7 +11485,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9354,7 +11577,7 @@ "h": 1, "w": 24, "x": 0, - "y": 13 + "y": 14 }, "id": 2755, "panels": [ @@ -9366,7 +11589,12 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The total number of commands on each stage", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 10, "w": 12, @@ -9374,6 +11602,7 @@ "y": 15 }, "height": "400", + "hiddenSeries": false, "id": 167, "legend": { "alignAsTable": true, @@ -9394,7 +11623,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9469,7 +11702,12 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The total writing bytes of commands on each stage", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 10, "w": 12, @@ -9477,6 +11715,7 @@ "y": 15 }, "height": "400", + "hiddenSeries": false, "id": 3834, "legend": { "alignAsTable": true, @@ -9497,7 +11736,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9566,7 +11809,12 @@ "description": "The count of different priority commands", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9575,6 +11823,7 @@ "y": 25 }, "height": "", + "hiddenSeries": false, "id": 1, "legend": { "alignAsTable": true, @@ -9595,7 +11844,11 @@ "links": [], "maxPerRow": 2, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9700,7 +11953,12 @@ "description": "The count of pending commands per TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9709,6 +11967,7 @@ "y": 25 }, "height": "", + "hiddenSeries": false, "id": 193, "legend": { "alignAsTable": true, @@ -9729,7 +11988,11 @@ "links": [], "maxPerRow": 2, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9754,7 +12017,8 @@ "fill": true, "line": true, "op": "gt", - "value": 300 + "value": 300, + "visible": true } ], "timeFrom": null, @@ -9810,7 +12074,7 @@ "h": 1, "w": 24, "x": 0, - "y": 14 + "y": 15 }, "id": 2756, "panels": [ @@ -9822,7 +12086,12 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The total number of commands on each stage in commit command", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 10, "w": 24, @@ -9830,6 +12099,7 @@ "y": 16 }, "height": "400", + "hiddenSeries": false, "id": 168, "legend": { "alignAsTable": true, @@ -9850,7 +12120,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -9934,7 +12208,12 @@ "description": "The time consumed when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -9942,6 +12221,7 @@ "x": 0, "y": 26 }, + "hiddenSeries": false, "id": 3, "legend": { "alignAsTable": true, @@ -9963,7 +12243,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10059,7 +12343,12 @@ "description": "The time which is caused by latch wait in commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -10067,6 +12356,7 @@ "x": 12, "y": 26 }, + "hiddenSeries": false, "id": 194, "legend": { "alignAsTable": true, @@ -10088,7 +12378,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10184,7 +12478,12 @@ "description": "The count of keys read by a commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -10192,6 +12491,7 @@ "x": 0, "y": 34 }, + "hiddenSeries": false, "id": 195, "legend": { "alignAsTable": true, @@ -10213,7 +12513,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10309,7 +12613,12 @@ "description": "The count of keys written by a commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -10317,6 +12626,7 @@ "x": 12, "y": 34 }, + "hiddenSeries": false, "id": 373, "legend": { "alignAsTable": true, @@ -10338,7 +12648,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10434,6 +12748,10 @@ "description": "The keys scan details of each CF when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "grid": {}, "gridPos": { @@ -10464,6 +12782,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10541,6 +12860,10 @@ "description": "The keys scan details of lock CF when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "grid": {}, "gridPos": { @@ -10571,6 +12894,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10648,6 +12972,10 @@ "description": "The keys scan details of write CF when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "grid": {}, "gridPos": { @@ -10678,6 +13006,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10755,6 +13084,10 @@ "description": "The keys scan details of default CF when executing commit command", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "grid": {}, "gridPos": { @@ -10785,6 +13118,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10864,7 +13198,7 @@ "h": 1, "w": 24, "x": 0, - "y": 15 + "y": 16 }, "id": 2759, "panels": [ @@ -10877,14 +13211,20 @@ "description": "The rate of Raft snapshot messages sent", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 8, "x": 0, - "y": 16 + "y": 17 }, + "hiddenSeries": false, "id": 35, "legend": { "alignAsTable": true, @@ -10904,7 +13244,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -10973,14 +13317,20 @@ "description": "The time consumed when handling snapshots", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 8, "x": 8, - "y": 16 + "y": 17 }, + "hiddenSeries": false, "id": 36, "legend": { "alignAsTable": true, @@ -11000,7 +13350,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11085,14 +13439,20 @@ "description": "The number of snapshots in different states", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 8, "x": 16, - "y": 16 + "y": 17 }, + "hiddenSeries": false, "id": 38, "legend": { "alignAsTable": true, @@ -11112,7 +13472,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11182,14 +13546,20 @@ "description": "The snapshot size (P99.99).9999", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 23 + "y": 24 }, + "hiddenSeries": false, "id": 44, "legend": { "alignAsTable": true, @@ -11209,7 +13579,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11279,14 +13653,20 @@ "description": "The number of KV within a snapshot in .9999", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 23 + "y": 24 }, + "hiddenSeries": false, "id": 43, "legend": { "alignAsTable": true, @@ -11306,7 +13686,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11379,7 +13763,7 @@ "h": 1, "w": 24, "x": 0, - "y": 16 + "y": 17 }, "id": 2760, "panels": [ @@ -11393,14 +13777,20 @@ "description": "The number of tasks handled by worker", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 21 + "y": 18 }, + "hiddenSeries": false, "id": 59, "legend": { "alignAsTable": true, @@ -11422,7 +13812,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11492,14 +13886,20 @@ "description": " \tCurrent pending and running tasks of worker", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 21 + "y": 18 }, + "hiddenSeries": false, "id": 1395, "legend": { "alignAsTable": true, @@ -11521,7 +13921,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11591,14 +13995,20 @@ "description": "The number of tasks handled by future_pool", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 29 + "y": 26 }, + "hiddenSeries": false, "id": 1876, "legend": { "alignAsTable": true, @@ -11620,7 +14030,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11690,14 +14104,20 @@ "description": "Current pending and running tasks of future_pool", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 29 + "y": 26 }, + "hiddenSeries": false, "id": 1877, "legend": { "alignAsTable": true, @@ -11719,7 +14139,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -11791,7 +14215,7 @@ "h": 1, "w": 24, "x": 0, - "y": 17 + "y": 18 }, "id": 2761, "panels": [ @@ -11802,13 +14226,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 24 + "y": 19 }, + "hiddenSeries": false, "id": 2108, "legend": { "alignAsTable": true, @@ -11828,7 +14258,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": true, "renderer": "flot", @@ -11901,13 +14335,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 24 + "y": 19 }, + "hiddenSeries": false, "id": 2258, "legend": { "alignAsTable": true, @@ -11927,7 +14367,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": true, "renderer": "flot", @@ -11995,13 +14439,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 31 + "y": 26 }, + "hiddenSeries": false, "id": 2660, "legend": { "alignAsTable": true, @@ -12021,7 +14471,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": true, "renderer": "flot", @@ -12089,13 +14543,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 31 + "y": 26 }, + "hiddenSeries": false, "id": 2661, "legend": { "alignAsTable": true, @@ -12115,7 +14575,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 2, "points": true, "renderer": "flot", @@ -12188,7 +14652,7 @@ "h": 1, "w": 24, "x": 0, - "y": 18 + "y": 19 }, "id": 2762, "panels": [ @@ -12200,13 +14664,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of get operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 25 + "y": 20 }, + "hiddenSeries": false, "id": 138, "legend": { "alignAsTable": true, @@ -12226,7 +14696,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12334,13 +14808,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing get operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 25 + "y": 20 }, + "hiddenSeries": false, "id": 82, "legend": { "alignAsTable": true, @@ -12360,7 +14840,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12458,13 +14942,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of seek operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 33 + "y": 28 }, + "hiddenSeries": false, "id": 129, "legend": { "alignAsTable": true, @@ -12484,7 +14974,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12604,13 +15098,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing seek operation", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 33 + "y": 28 }, + "hiddenSeries": false, "id": 125, "legend": { "alignAsTable": true, @@ -12630,7 +15130,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12728,14 +15232,146 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of write operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 36 + }, + "hiddenSeries": false, + "id": 139, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "scopedVars": { + "db": { + "selected": false, + "text": "kv", + "value": "kv" + } + }, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=~\"write_done_by_self|write_done_by_other\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "done", + "refId": "A", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_timeout\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "timeout", + "refId": "B", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_with_wal\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "with_wal", + "refId": "C", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write operations", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The time consumed when executing write operation", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 41 + "x": 12, + "y": 36 }, - "id": 139, + "hiddenSeries": false, + "id": 126, "legend": { "alignAsTable": true, "avg": false, @@ -12754,7 +15390,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12771,35 +15411,43 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=~\"write_done_by_self|write_done_by_other\"}[1m]))", + "expr": "max(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_max\"})", "format": "time_series", "intervalFactor": 2, - "legendFormat": "done", + "legendFormat": "max", "refId": "A", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_timeout\"}[1m]))", + "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_percentile99\"})", "format": "time_series", "intervalFactor": 2, - "legendFormat": "timeout", + "legendFormat": "99%", "refId": "B", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_write_served{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_with_wal\"}[1m]))", + "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_percentile95\"})", "format": "time_series", "intervalFactor": 2, - "legendFormat": "with_wal", + "legendFormat": "95%", "refId": "C", "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_average\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "avg", + "refId": "D", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write operations", + "title": "Write duration", "tooltip": { "shared": true, "sort": 0, @@ -12815,9 +15463,9 @@ }, "yaxes": [ { - "format": "ops", + "format": "µs", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -12843,15 +15491,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The time consumed when executing write operation", - "fill": 0, + "description": " \tThe count of WAL sync operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 41 + "x": 0, + "y": 44 }, - "id": 126, + "id": 137, "legend": { "alignAsTable": true, "avg": false, @@ -12871,6 +15523,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -12887,43 +15540,20 @@ "steppedLine": false, "targets": [ { - "expr": "max(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_max\"})", + "expr": "sum(rate(tiflash_proxy_tikv_engine_wal_file_synced{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m]))", "format": "time_series", "intervalFactor": 2, - "legendFormat": "max", + "legendFormat": "sync", + "metric": "", "refId": "A", "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_percentile99\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99%", - "refId": "B", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_percentile95\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "95%", - "refId": "C", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_micro_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\",type=\"write_average\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "avg", - "refId": "D", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write duration", + "title": "WAL sync operations", "tooltip": { "shared": true, "sort": 0, @@ -12939,9 +15569,9 @@ }, "yaxes": [ { - "format": "µs", + "format": "ops", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -12968,12 +15598,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing write wal operation", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 41 + "y": 44 }, "id": 130, "legend": { @@ -12995,6 +15629,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13091,15 +15726,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": " \tThe count of WAL sync operations", + "description": "The count of compaction and flush operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 49 + "y": 52 }, - "id": 137, + "id": 128, "legend": { "alignAsTable": true, "avg": false, @@ -13119,6 +15758,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13135,12 +15775,12 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_wal_file_synced{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_event_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "sync", - "metric": "", - "refId": "A", + "legendFormat": "{{type}}", + "metric": "tiflash_proxy_tikv_engine_event_total", + "refId": "B", "step": 10 } ], @@ -13148,7 +15788,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "WAL sync operations", + "title": "Compaction operations", "tooltip": { "shared": true, "sort": 0, @@ -13193,12 +15833,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing WAL sync operation", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 49 + "y": 52 }, "id": 135, "legend": { @@ -13221,6 +15865,7 @@ "maxPerRow": 2, "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13317,15 +15962,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The count of compaction and flush operations", + "description": "The time consumed when reading SST files", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 57 + "y": 60 }, - "id": 128, + "id": 140, "legend": { "alignAsTable": true, "avg": false, @@ -13345,6 +15994,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13361,20 +16011,47 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_event_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (type)", + "expr": "max(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_max\"})", "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{type}}", - "metric": "tiflash_proxy_tikv_engine_event_total", + "legendFormat": "max", + "metric": "", + "refId": "A", + "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_percentile99\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99%", + "metric": "", "refId": "B", "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_percentile95\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95%", + "metric": "", + "refId": "C", + "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_average\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "avg", + "metric": "", + "refId": "D", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compaction operations", + "title": "SST read duration", "tooltip": { "shared": true, "sort": 0, @@ -13390,9 +16067,9 @@ }, "yaxes": [ { - "format": "ops", + "format": "µs", "label": null, - "logBase": 1, + "logBase": 10, "max": null, "min": null, "show": true @@ -13419,12 +16096,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The time consumed when executing the compaction and flush operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 57 + "y": 60 }, "id": 136, "legend": { @@ -13446,6 +16127,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13543,15 +16225,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The time consumed when reading SST files", + "description": "The block cache size. Broken down by column family if shared block cache is disabled.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 65 + "y": 68 }, - "id": 140, + "id": 102, "legend": { "alignAsTable": true, "avg": false, @@ -13571,6 +16257,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13587,47 +16274,19 @@ "steppedLine": false, "targets": [ { - "expr": "max(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_max\"})", + "expr": "topk(20, avg(tiflash_proxy_tikv_engine_block_cache_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by(cf, instance))", "format": "time_series", "intervalFactor": 2, - "legendFormat": "max", - "metric": "", + "legendFormat": "{{instance}}-{{cf}}", "refId": "A", "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_percentile99\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99%", - "metric": "", - "refId": "B", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_percentile95\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "95%", - "metric": "", - "refId": "C", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_sst_read_micros{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"sst_read_micros_average\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "avg", - "metric": "", - "refId": "D", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "SST read duration", + "title": "Block cache size", "tooltip": { "shared": true, "sort": 0, @@ -13643,9 +16302,9 @@ }, "yaxes": [ { - "format": "µs", + "format": "bytes", "label": null, - "logBase": 10, + "logBase": 1, "max": null, "min": null, "show": true @@ -13671,15 +16330,18 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The time which is caused by write stall", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 153 + "y": 68 }, - "id": 87, + "id": 2451, "legend": { "alignAsTable": true, "avg": false, @@ -13699,55 +16361,30 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", - "scopedVars": { - "db": { - "selected": false, - "text": "kv", - "value": "kv" - } - }, - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_max\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "max", - "metric": "", - "refId": "A", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_percentile99\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99%", - "metric": "", - "refId": "B", - "step": 10 - }, - { - "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_percentile95\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "95%", - "metric": "", - "refId": "C", - "step": 10 - }, + "scopedVars": { + "db": { + "selected": false, + "text": "kv", + "value": "kv" + } + }, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_average\"})", + "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (cf, reason)", "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "avg", + "legendFormat": "{{cf}} - {{reason}}", "metric": "", - "refId": "D", + "refId": "A", "step": 10 } ], @@ -13755,7 +16392,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write stall duration", + "title": "Compaction reason", "tooltip": { "shared": true, "sort": 0, @@ -13771,11 +16408,11 @@ }, "yaxes": [ { - "format": "µs", + "format": "short", "label": null, - "logBase": 10, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -13783,7 +16420,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -13799,15 +16436,20 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The memtable size of each column family", - "fill": 1, + "description": "The flow of different kinds of block cache operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 153 + "x": 0, + "y": 76 }, - "id": 103, + "height": "", + "id": 467, "legend": { "alignAsTable": true, "avg": false, @@ -13827,6 +16469,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13843,19 +16486,86 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_proxy_tikv_engine_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"mem-tables\"}) by (cf)", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_byte_read\"}[1m]))", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 2, - "legendFormat": "{{cf}}", + "legendFormat": "total_read", "refId": "A", "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_byte_write\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "total_written", + "refId": "C", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_data_bytes_insert\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "data_insert", + "metric": "", + "refId": "D", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_filter_bytes_insert\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "filter_insert", + "metric": "", + "refId": "B", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_filter_bytes_evict\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "filter_evict", + "metric": "", + "refId": "E", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_index_bytes_insert\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "index_insert", + "metric": "", + "refId": "F", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_index_bytes_evict\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "index_evict", + "metric": "", + "refId": "G", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memtable size", + "title": "Block cache flow", "tooltip": { "shared": true, "sort": 0, @@ -13871,19 +16581,19 @@ }, "yaxes": [ { - "format": "bytes", + "format": "Bps", "label": null, - "logBase": 1, + "logBase": 10, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -13900,12 +16610,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The hit rate of memtable", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 73 + "y": 76 }, "id": 88, "legend": { @@ -13927,6 +16641,7 @@ "links": [], "nullPointMode": "connected", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -13999,15 +16714,20 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The block cache size. Broken down by column family if shared block cache is disabled.", - "fill": 1, + "description": "The flow of different kinds of operations on keys", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 81 + "y": 84 }, - "id": 102, + "height": "", + "id": 132, "legend": { "alignAsTable": true, "avg": false, @@ -14027,6 +16747,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14043,10 +16764,33 @@ "steppedLine": false, "targets": [ { - "expr": "topk(20, avg(tiflash_proxy_tikv_engine_block_cache_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by(cf, instance))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"keys_read\"}[1m]))", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 2, - "legendFormat": "{{instance}}-{{cf}}", + "legendFormat": "read", + "refId": "B", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"keys_written\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "written", + "refId": "C", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_num_corrupt_keys{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "corrupt", + "metric": "", "refId": "A", "step": 10 } @@ -14055,7 +16799,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Block cache size", + "title": "Keys flow", "tooltip": { "shared": true, "sort": 0, @@ -14071,11 +16815,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "ops", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -14083,7 +16827,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -14100,12 +16844,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The hit rate of block cache", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 81 + "y": 84 }, "id": 80, "legend": { @@ -14128,6 +16876,7 @@ "maxPerRow": 2, "nullPointMode": "connected", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14237,16 +16986,20 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow of different kinds of block cache operations", + "description": "The flow rate of read operations per type", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 89 + "y": 92 }, "height": "", - "id": 467, + "id": 85, "legend": { "alignAsTable": true, "avg": false, @@ -14266,6 +17019,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14282,86 +17036,31 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_byte_read\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_read\"}[1m]))", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "total_read", + "legendFormat": "get", "refId": "A", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_byte_write\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"iter_bytes_read\"}[1m]))", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "total_written", + "legendFormat": "scan", "refId": "C", "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_data_bytes_insert\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "data_insert", - "metric": "", - "refId": "D", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_filter_bytes_insert\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "filter_insert", - "metric": "", - "refId": "B", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_filter_bytes_evict\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "filter_evict", - "metric": "", - "refId": "E", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_index_bytes_insert\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "index_insert", - "metric": "", - "refId": "F", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_cache_efficiency{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"block_cache_index_bytes_evict\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "index_evict", - "metric": "", - "refId": "G", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Block cache flow", + "title": "Read flow", "tooltip": { "shared": true, "sort": 0, @@ -14379,13 +17078,13 @@ { "format": "Bps", "label": null, - "logBase": 10, + "logBase": 1, "max": null, "min": "0", "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -14406,12 +17105,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of different kinds of block cache operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 89 + "y": 92 }, "id": 468, "legend": { @@ -14433,6 +17136,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14542,16 +17246,20 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow of different kinds of operations on keys", - "fill": 0, + "description": "The flow of different kinds of write operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 97 + "y": 100 }, "height": "", - "id": 132, + "id": 86, "legend": { "alignAsTable": true, "avg": false, @@ -14571,6 +17279,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14587,33 +17296,20 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"keys_read\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "read", - "refId": "B", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"keys_written\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"wal_file_bytes\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "written", + "legendFormat": "wal", "refId": "C", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_num_corrupt_keys{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_written\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "corrupt", - "metric": "", + "legendFormat": "write", "refId": "A", "step": 10 } @@ -14622,7 +17318,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Keys flow", + "title": "Write flow", "tooltip": { "shared": true, "sort": 0, @@ -14638,7 +17334,7 @@ }, "yaxes": [ { - "format": "ops", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -14667,12 +17363,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The count of keys in each column family", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 97 + "y": 100 }, "id": 131, "legend": { @@ -14694,6 +17394,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14768,16 +17469,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow rate of read operations per type", - "fill": 0, + "description": "The flow rate of compaction operations per type", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 105 + "y": 108 }, - "height": "", - "id": 85, + "id": 90, "legend": { "alignAsTable": true, "avg": false, @@ -14797,6 +17501,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -14813,31 +17518,38 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_read\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_read\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "get", + "legendFormat": "read", "refId": "A", "step": 10 }, { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"iter_bytes_read\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_written\"}[1m]))", "format": "time_series", "hide": false, - "interval": "", "intervalFactor": 2, - "legendFormat": "scan", + "legendFormat": "written", "refId": "C", "step": 10 + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"flush_write_bytes\"}[1m]))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "flushed", + "refId": "B", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read flow", + "title": "Compaction flow", "tooltip": { "shared": true, "sort": 0, @@ -14861,7 +17573,7 @@ "show": true }, { - "format": "short", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -14882,12 +17594,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The bytes per read", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 105 + "y": 108 }, "id": 133, "legend": { @@ -14910,6 +17626,7 @@ "maxPerRow": 2, "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15006,16 +17723,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow of different kinds of write operations", + "description": "The read amplification per TiKV instance \t", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 113 + "y": 116 }, - "height": "", - "id": 86, + "id": 518, "legend": { "alignAsTable": true, "avg": false, @@ -15035,6 +17755,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15051,20 +17772,12 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"wal_file_bytes\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "wal", - "refId": "C", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_written\"}[1m]))", + "expr": "sum(rate(tiflash_proxy_tikv_engine_read_amp_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"read_amp_total_read_bytes\"}[1m])) by (instance) / sum(rate(tiflash_proxy_tikv_engine_read_amp_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", db=\"$db\", type=\"read_amp_estimate_useful_bytes\"}[1m])) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "write", + "legendFormat": "{{instance}}", + "metric": "", "refId": "A", "step": 10 } @@ -15073,7 +17786,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write flow", + "title": "Read amplication", "tooltip": { "shared": true, "sort": 0, @@ -15089,7 +17802,7 @@ }, "yaxes": [ { - "format": "Bps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -15118,12 +17831,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The bytes per write", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 113 + "y": 116 }, "id": 134, "legend": { @@ -15146,6 +17863,7 @@ "maxPerRow": 2, "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15242,15 +17960,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The flow rate of compaction operations per type", + "description": "The number of snapshot of each TiKV instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 121 + "y": 124 }, - "id": 90, + "id": 516, "legend": { "alignAsTable": true, "avg": false, @@ -15270,6 +17992,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15286,38 +18009,21 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_read\"}[1m]))", + "expr": "tiflash_proxy_tikv_engine_num_snapshots{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "read", + "legendFormat": "{{instance}}", + "metric": "", "refId": "A", "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"bytes_written\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "written", - "refId": "C", - "step": 10 - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"flush_write_bytes\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "flushed", - "refId": "B", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compaction flow", + "title": "Number of snapshots", "tooltip": { "shared": true, "sort": 0, @@ -15333,7 +18039,7 @@ }, "yaxes": [ { - "format": "Bps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -15341,7 +18047,7 @@ "show": true }, { - "format": "Bps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -15362,12 +18068,16 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The pending bytes to be compacted", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 121 + "y": 124 }, "id": 127, "legend": { @@ -15389,6 +18099,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15443,109 +18154,7 @@ "show": true }, { - "format": "Bps", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The read amplification per TiKV instance \t", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 129 - }, - "id": 518, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "scopedVars": { - "db": { - "selected": false, - "text": "kv", - "value": "kv" - } - }, - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tiflash_proxy_tikv_engine_read_amp_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"read_amp_total_read_bytes\"}[1m])) by (instance) / sum(rate(tiflash_proxy_tikv_engine_read_amp_flow_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", db=\"$db\", type=\"read_amp_estimate_useful_bytes\"}[1m])) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "metric": "", - "refId": "A", - "step": 10 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Read amplication", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -15564,25 +18173,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The compression ratio of each level", + "description": "The number of SST files for different column families in each level", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 129 + "x": 0, + "y": 132 }, - "id": 863, + "id": 2002, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, - "min": false, + "min": true, "rightSide": true, "show": true, - "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -15593,6 +18204,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15609,21 +18221,18 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_proxy_tikv_engine_compression_ratio{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by (level)", + "expr": "avg(tiflash_proxy_tikv_engine_num_files_at_level{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by (cf, level)", "format": "time_series", - "hide": false, "intervalFactor": 2, - "legendFormat": "level - {{level}}", - "metric": "", - "refId": "A", - "step": 10 + "legendFormat": "cf-{{cf}}, level-{{level}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compression ratio", + "title": "Number files at each level", "tooltip": { "shared": true, "sort": 0, @@ -15643,7 +18252,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -15651,7 +18260,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -15667,15 +18276,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The number of snapshot of each TiKV instance", + "description": "The compression ratio of each level", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 137 + "x": 12, + "y": 132 }, - "id": 516, + "id": 863, "legend": { "alignAsTable": true, "avg": false, @@ -15695,6 +18308,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15711,11 +18325,11 @@ "steppedLine": false, "targets": [ { - "expr": "tiflash_proxy_tikv_engine_num_snapshots{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", + "expr": "avg(tiflash_proxy_tikv_engine_compression_ratio{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by (level)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}", + "legendFormat": "level - {{level}}", "metric": "", "refId": "A", "step": 10 @@ -15725,7 +18339,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Number of snapshots", + "title": "Compression ratio", "tooltip": { "shared": true, "sort": 0, @@ -15768,25 +18382,28 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The time that the oldest unreleased snapshot survivals", + "description": "Stall conditions changed of each column family", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 137 + "x": 0, + "y": 140 }, - "id": 517, + "id": 2381, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideZero": true, "max": true, - "min": false, + "min": true, "rightSide": true, "show": true, - "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -15797,6 +18414,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15813,21 +18431,18 @@ "steppedLine": false, "targets": [ { - "expr": "tiflash_proxy_tikv_engine_oldest_snapshot_duration{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", + "expr": "tiflash_proxy_tikv_engine_stall_conditions_changed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", "format": "time_series", - "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}", - "metric": "tiflash_proxy_tikv_engine_oldest_snapshot_duration", - "refId": "A", - "step": 10 + "legendFormat": "{{instance}}-{{cf}}-{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Oldest snapshots duration", + "title": "Stall conditions changed of each CF", "tooltip": { "shared": true, "sort": 0, @@ -15843,11 +18458,11 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -15855,7 +18470,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -15870,23 +18485,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of SST files for different column families in each level", + "decimals": 1, + "description": "The time that the oldest unreleased snapshot survivals", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 145 + "x": 12, + "y": 140 }, - "id": 2002, + "id": 517, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, - "min": true, + "min": false, "rightSide": true, "show": true, + "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -15897,6 +18518,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -15913,18 +18535,21 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_proxy_tikv_engine_num_files_at_level{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}) by (cf, level)", + "expr": "tiflash_proxy_tikv_engine_oldest_snapshot_duration{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "cf-{{cf}}, level-{{level}}", - "refId": "A" + "legendFormat": "{{instance}}", + "metric": "tiflash_proxy_tikv_engine_oldest_snapshot_duration", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Number files at each level", + "title": "Oldest snapshots duration", "tooltip": { "shared": true, "sort": 0, @@ -15940,11 +18565,11 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -15952,7 +18577,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -15967,23 +18592,28 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The time consumed when ingesting SST files", + "decimals": 1, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 145 + "x": 0, + "y": 148 }, - "id": 2003, + "id": 2452, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, - "min": true, + "min": false, "rightSide": true, "show": true, + "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, @@ -15994,6 +18624,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -16010,25 +18641,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99%", - "refId": "A" - }, - { - "expr": "sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) / sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "expr": "sum(increase(tiflash_proxy_tikv_engine_write_stall_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (type)", "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "average", - "refId": "B" + "legendFormat": "{{type}}", + "metric": "", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Ingest SST duration seconds", + "title": "Write Stall Reason", "tooltip": { "shared": true, "sort": 0, @@ -16044,11 +18671,11 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -16056,7 +18683,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -16071,20 +18698,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Stall conditions changed of each column family", + "description": "The time consumed when ingesting SST files", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 153 + "x": 12, + "y": 148 }, - "id": 2381, + "id": 2003, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideZero": true, "max": true, "min": true, "rightSide": true, @@ -16099,6 +18729,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -16115,10 +18746,17 @@ "steppedLine": false, "targets": [ { - "expr": "tiflash_proxy_tikv_engine_stall_conditions_changed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}", + "expr": "histogram_quantile(0.99, sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{instance}}-{{cf}}-{{type}}", + "legendFormat": "99%", + "refId": "A" + }, + { + "expr": "sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) / sum(rate(tiflash_proxy_tikv_snapshot_ingest_sst_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "average", "refId": "B" } ], @@ -16126,7 +18764,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Stall conditions changed of each CF", + "title": "Ingest SST duration seconds", "tooltip": { "shared": true, "sort": 0, @@ -16142,7 +18780,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -16170,14 +18808,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "description": "The time which is caused by write stall", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 161 + "x": 12, + "y": 156 }, - "id": 2452, + "id": 87, "legend": { "alignAsTable": true, "avg": false, @@ -16197,6 +18840,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -16213,21 +18857,47 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tiflash_proxy_tikv_engine_write_stall_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (type)", + "expr": "max(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_max\"})", "format": "time_series", - "hide": false, "intervalFactor": 2, - "legendFormat": "{{type}}", + "legendFormat": "max", "metric": "", "refId": "A", "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_percentile99\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99%", + "metric": "", + "refId": "B", + "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_percentile95\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95%", + "metric": "", + "refId": "C", + "step": 10 + }, + { + "expr": "avg(tiflash_proxy_tikv_engine_write_stall{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"write_stall_average\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "avg", + "metric": "", + "refId": "D", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Stall Reason", + "title": "Write stall duration", "tooltip": { "shared": true, "sort": 0, @@ -16243,11 +18913,11 @@ }, "yaxes": [ { - "format": "short", + "format": "µs", "label": null, - "logBase": 1, + "logBase": 10, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -16255,7 +18925,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -16271,14 +18941,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "description": "The memtable size of each column family", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 65 + "y": 164 }, - "id": 2451, + "id": 103, "legend": { "alignAsTable": true, "avg": false, @@ -16298,6 +18973,7 @@ "links": [], "nullPointMode": "null", "percentage": false, + "pluginVersion": "7.5.17", "pointradius": 5, "points": false, "renderer": "flot", @@ -16314,12 +18990,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_proxy_tikv_engine_compaction_reason{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (cf, reason)", + "expr": "avg(tiflash_proxy_tikv_engine_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\", type=\"mem-tables\"}) by (cf)", "format": "time_series", - "hide": false, "intervalFactor": 2, - "legendFormat": "{{cf}} - {{reason}}", - "metric": "", + "legendFormat": "{{cf}}", "refId": "A", "step": 10 } @@ -16328,7 +19002,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Compaction reason", + "title": "Memtable size", "tooltip": { "shared": true, "sort": 0, @@ -16344,11 +19018,11 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -16356,7 +19030,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -16845,7 +19519,7 @@ } ], "refresh": "1m", - "schemaVersion": 18, + "schemaVersion": 27, "style": "dark", "tags": [], "templating": { @@ -16984,6 +19658,37 @@ "tagsQuery": "", "type": "query", "useTags": false + }, + { + "allValue": null, + "current": { + "selected": true, + "text": "none", + "value": "none" + }, + "description": null, + "error": null, + "hide": 0, + "includeAll": false, + "label": "additional_groupby", + "multi": false, + "name": "additional_groupby", + "options": [ + { + "selected": true, + "text": "none", + "value": "none" + }, + { + "selected": false, + "text": "instance", + "value": "instance" + } + ], + "query": "none,instance", + "queryValue": "", + "skipUrlSync": false, + "type": "custom" } ] }, diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index e11c62ac3fb..0249e3e3e77 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -2882,7 +2882,7 @@ "x": 0, "y": 2 }, - "id": 6, + "id": 294, "panels": [ { "aliasColors": {}, @@ -2890,20 +2890,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": 0, - "y": 3 + "y": 1 }, "hiddenSeries": false, - "id": 9, + "id": 296, "legend": { "alignAsTable": true, "avg": false, @@ -2912,20 +2913,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, @@ -2933,18 +2937,30 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_coprocessor_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "exemplar": true, + "expr": "sum(tiflash_proxy_threads_state{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (instance, state)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" + "legendFormat": "{{instance}}-{{state}}", + "refId": "A", + "step": 4 + }, + { + "exemplar": true, + "expr": "sum(tiflash_proxy_threads_state{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}-total", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request QPS", + "title": "Threads state", "tooltip": { "shared": true, "sort": 0, @@ -2960,16 +2976,15 @@ }, "yaxes": [ { - "decimals": null, "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -2988,20 +3003,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, - "y": 3 + "y": 1 }, "hiddenSeries": false, - "id": 2, + "id": 298, "legend": { "alignAsTable": true, "avg": false, @@ -3010,20 +3026,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, @@ -3031,18 +3050,22 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_coprocessor_executor_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "exemplar": true, + "expr": "sum(rate(tiflash_proxy_threads_io_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name, io) > 1024", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "A" + "legendFormat": "{{name}}-{{io}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Executor QPS", + "title": "Threads IO", "tooltip": { "shared": true, "sort": 0, @@ -3058,15 +3081,15 @@ }, "yaxes": [ { - "format": "none", + "format": "Bps", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3085,6 +3108,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, "fieldConfig": { "defaults": {}, "overrides": [] @@ -3095,32 +3119,35 @@ "h": 7, "w": 12, "x": 0, - "y": 10 + "y": 8 }, "hiddenSeries": false, - "id": 11, + "id": 300, "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 +3156,21 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_proxy_thread_voluntary_context_switches{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (instance, name) > 200", "format": "time_series", + "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "999-{{type}}", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "99-{{type}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "95-{{type}}", - "refId": "C" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" + "legendFormat": "{{instance}} - {{name}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request Duration", + "title": "Thread Voluntary Context Switches", "tooltip": { "shared": true, "sort": 0, @@ -3181,11 +3186,11 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -3208,20 +3213,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, - "y": 10 + "y": 8 }, "hiddenSeries": false, - "id": 12, + "id": 302, "legend": { "alignAsTable": true, "avg": false, @@ -3230,20 +3236,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 +3260,22 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_coprocessor_request_error{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (reason)", + "exemplar": true, + "expr": "sum(rate(tiflash_proxy_thread_nonvoluntary_context_switches{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (instance, name) > 50", "format": "time_series", + "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "{{reason}}", - "refId": "A" + "legendFormat": "{{instance}} - {{name}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Error QPS", + "title": "Thread Nonvoluntary Context Switches", "tooltip": { "shared": true, "sort": 0, @@ -3282,11 +3295,11 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3298,7 +3311,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, @@ -3309,26 +3337,26 @@ "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 17 + "y": 3 }, "hiddenSeries": false, - "id": 13, + "id": 9, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -3348,39 +3376,18 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_coprocessor_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "999-{{type}}", + "legendFormat": "{{type}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "99-{{type}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "95-{{type}}", - "refId": "C" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Request Handle Duration", + "title": "Request QPS", "tooltip": { "shared": true, "sort": 0, @@ -3396,7 +3403,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": null, + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3404,7 +3412,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3433,10 +3441,10 @@ "h": 7, "w": 12, "x": 12, - "y": 17 + "y": 3 }, "hiddenSeries": false, - "id": 14, + "id": 2, "legend": { "alignAsTable": true, "avg": false, @@ -3466,10 +3474,8 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_coprocessor_response_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_coprocessor_executor_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "interval": "", "intervalFactor": 1, "legendFormat": "{{type}}", "refId": "A" @@ -3479,7 +3485,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Response Bytes/Seconds", + "title": "Executor QPS", "tooltip": { "shared": true, "sort": 0, @@ -3495,7 +3501,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3503,7 +3509,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3532,15 +3538,17 @@ "h": 7, "w": 12, "x": 0, - "y": 24 + "y": 10 }, "hiddenSeries": false, - "id": 63, + "id": 11, "legend": { + "alignAsTable": false, "avg": false, "current": false, "max": false, "min": false, + "rightSide": false, "show": true, "total": false, "values": false @@ -3554,7 +3562,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3563,29 +3571,34 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "interval": "", "intervalFactor": 1, "legendFormat": "999-{{type}}", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "hide": true, "intervalFactor": 1, "legendFormat": "99-{{type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "hide": true, "intervalFactor": 1, "legendFormat": "95-{{type}}", "refId": "C" }, { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", + "hide": true, "intervalFactor": 1, "legendFormat": "80-{{type}}", "refId": "D" @@ -3595,7 +3608,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Cop task memory usage", + "title": "Request Duration", "tooltip": { "shared": true, "sort": 0, @@ -3611,7 +3624,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -3648,10 +3661,10 @@ "h": 7, "w": 12, "x": 12, - "y": 24 + "y": 10 }, "hiddenSeries": false, - "id": 165, + "id": 12, "legend": { "alignAsTable": true, "avg": false, @@ -3681,12 +3694,10 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_exchange_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_coprocessor_request_error{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (reason)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{reason}}", "refId": "A" } ], @@ -3694,7 +3705,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Exchange Bytes/Seconds", + "title": "Error QPS", "tooltip": { "shared": true, "sort": 0, @@ -3710,7 +3721,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3718,7 +3729,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3741,26 +3752,26 @@ "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 31 + "y": 17 }, "hiddenSeries": false, - "id": 100, + "id": 13, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -3780,20 +3791,39 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type!~\".*max\"}) by (instance, type)", + "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "999-{{type}}", "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_handle_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Threads of Rpc", + "title": "Request Handle Duration", "tooltip": { "shared": true, "sort": 0, @@ -3809,7 +3839,7 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -3846,10 +3876,10 @@ "h": 7, "w": 12, "x": 12, - "y": 31 + "y": 17 }, "hiddenSeries": false, - "id": 77, + "id": 14, "legend": { "alignAsTable": true, "avg": false, @@ -3879,8 +3909,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tiflash_coprocessor_handling_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", + "exemplar": true, + "expr": "sum(rate(tiflash_coprocessor_response_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", + "interval": "", "intervalFactor": 1, "legendFormat": "{{type}}", "refId": "A" @@ -3890,7 +3922,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Handling Request Number", + "title": "Response Bytes/Seconds", "tooltip": { "shared": true, "sort": 0, @@ -3906,8 +3938,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -3915,7 +3946,7 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3938,26 +3969,24 @@ "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 38 + "y": 24 }, "hiddenSeries": false, - "id": 102, + "id": 63, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -3968,7 +3997,7 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3977,20 +4006,39 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\".*max\", type!~\"rpc.*\"}) by (instance, type)", + "expr": "histogram_quantile(0.999, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "999-{{type}}", "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_coprocessor_request_memory_usage_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Threads", + "title": "Cop task memory usage", "tooltip": { "shared": true, "sort": 0, @@ -4006,7 +4054,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4043,10 +4091,10 @@ "h": 7, "w": 12, "x": 12, - "y": 38 + "y": 24 }, "hiddenSeries": false, - "id": 101, + "id": 165, "legend": { "alignAsTable": true, "avg": false, @@ -4077,11 +4125,11 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type=~\".*max\"}) by (instance, type)", + "expr": "sum(rate(tiflash_exchange_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{type}}", "refId": "A" } ], @@ -4089,7 +4137,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Max Threads of Rpc", + "title": "Exchange Bytes/Seconds", "tooltip": { "shared": true, "sort": 0, @@ -4105,7 +4153,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4132,7 +4180,6 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The MPP query count in TiFlash", "fieldConfig": { "defaults": {}, "overrides": [] @@ -4143,10 +4190,10 @@ "h": 7, "w": 12, "x": 0, - "y": 45 + "y": 31 }, "hiddenSeries": false, - "id": 157, + "id": 100, "legend": { "alignAsTable": true, "avg": false, @@ -4177,7 +4224,7 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_mpp_task_manager{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance, type)", + "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type!~\".*max\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -4189,7 +4236,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MPP Query count", + "title": "Threads of Rpc", "tooltip": { "shared": true, "sort": 0, @@ -4242,10 +4289,10 @@ "h": 7, "w": 12, "x": 12, - "y": 45 + "y": 31 }, "hiddenSeries": false, - "id": 103, + "id": 77, "legend": { "alignAsTable": true, "avg": false, @@ -4275,12 +4322,10 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\".*max\", type!~\"rpc.*\"}) by (instance, type)", + "expr": "sum(tiflash_coprocessor_handling_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{type}}", "refId": "A" } ], @@ -4288,7 +4333,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Max Threads", + "title": "Handling Request Number", "tooltip": { "shared": true, "sort": 0, @@ -4304,6 +4349,7 @@ }, "yaxes": [ { + "decimals": null, "format": "none", "label": null, "logBase": 1, @@ -4312,7 +4358,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4341,10 +4387,10 @@ "h": 7, "w": 12, "x": 0, - "y": 52 + "y": 38 }, "hiddenSeries": false, - "id": 199, + "id": 102, "legend": { "alignAsTable": true, "avg": false, @@ -4375,11 +4421,11 @@ "targets": [ { "exemplar": true, - "expr": "tiflash_mpp_task_monitor{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\".*max\", type!~\"rpc.*\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4387,7 +4433,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Time of the Longest Live MPP Task", + "title": "Threads", "tooltip": { "shared": true, "sort": 0, @@ -4403,7 +4449,7 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4440,10 +4486,10 @@ "h": 7, "w": 12, "x": 12, - "y": 52 + "y": 38 }, "hiddenSeries": false, - "id": 166, + "id": 101, "legend": { "alignAsTable": true, "avg": false, @@ -4474,11 +4520,11 @@ "targets": [ { "exemplar": true, - "expr": "tiflash_exchange_queueing_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "max(tiflash_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"rpc.*\", type=~\".*max\"}) by (instance, type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4486,7 +4532,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Data size in send and receive queue", + "title": "Max Threads of Rpc", "tooltip": { "shared": true, "sort": 0, @@ -4502,7 +4548,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4522,66 +4568,50 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Coprocessor", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 3 - }, - "id": 105, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the min_tso of each instance", + "description": "The MPP query count in TiFlash", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 4 + "y": 45 }, "hiddenSeries": false, - "id": 107, + "id": 157, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, "total": false, "values": true }, - "lines": false, + "lines": true, "linewidth": 1, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { - "alertThreshold": false + "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 1, - "points": true, + "pointradius": 5, + "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -4590,10 +4620,11 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"min_tso\"}) by (instance, resource_group)", + "expr": "max(tiflash_mpp_task_manager{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance, type)", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4601,7 +4632,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Min TSO", + "title": "MPP Query count", "tooltip": { "shared": true, "sort": 0, @@ -4617,13 +4648,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 +4675,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "estimated thread usage in min-tso scheduler, and the sort/hard limit of estimated thread in scheduler.", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 4 + "y": 45 }, "hiddenSeries": false, - "id": 109, + "id": 103, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -4676,13 +4702,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 +4719,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_thread_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\".*max\", type!~\"rpc.*\"}) by (instance, type)", + "format": "time_series", "interval": "", + "intervalFactor": 1, "legendFormat": "{{instance}}-{{type}}", - "queryType": "randomWalk", - "refId": "E" + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Estimated Thread Usage and Limit", + "title": "Max Threads", "tooltip": { "shared": true, "sort": 0, @@ -4755,12 +4747,11 @@ }, "yaxes": [ { - "decimals": null, "format": "none", - "label": "Threads", - "logBase": 10, + "label": null, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -4783,29 +4774,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the count of active/ waiting queries", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 12 + "y": 52 }, "hiddenSeries": false, - "id": 111, + "id": 199, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -4814,13 +4801,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 +4818,19 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_queries_count\"}) by (instance, type, resource_group)", + "expr": "tiflash_mpp_task_monitor{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_queries_count\"}) by (instance, type, resource_group)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Active and Waiting Queries Count", + "title": "Time of the Longest Live MPP Task", "tooltip": { "shared": true, "sort": 0, @@ -4866,12 +4846,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "Queries", + "format": "s", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -4894,29 +4873,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "the count of active/ waiting tasks", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 12 + "y": 52 }, "hiddenSeries": false, - "id": 113, + "id": 166, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -4925,13 +4900,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 +4917,19 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_tasks_count\"}) by (instance, type, resource_group)", + "expr": "tiflash_exchange_queueing_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{type}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_tasks_count\"}) by (instance, type, resource_group)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", - "queryType": "randomWalk", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Active and Waiting Tasks Count", + "title": "Data size in send and receive queue", "tooltip": { "shared": true, "sort": 0, @@ -4977,12 +4945,11 @@ }, "yaxes": [ { - "decimals": null, - "format": "none", - "label": "Tasks", + "format": "bytes", + "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -4998,7 +4965,23 @@ "align": false, "alignLevel": null } - }, + } + ], + "repeat": null, + "title": "Coprocessor", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 105, + "panels": [ { "aliasColors": {}, "bars": false, @@ -5006,23 +4989,21 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": null, - "description": "the usage of estimated threads exceeded the hard limit where errors occur.", + "description": "the min_tso of each instance", "fieldConfig": { - "defaults": { - "unit": "none" - }, + "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 20 + "y": 4 }, "hiddenSeries": false, - "id": 117, + "id": 107, "legend": { "alignAsTable": false, "avg": false, @@ -5034,16 +5015,16 @@ "total": false, "values": true }, - "lines": true, + "lines": false, "linewidth": 1, - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { - "alertThreshold": true + "alertThreshold": false }, "percentage": false, "pluginVersion": "7.5.11", "pointradius": 1, - "points": false, + "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -5052,19 +5033,18 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"hard_limit_exceeded_count\"}) by (instance, type, resource_group)", - "hide": false, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"min_tso\"}) by (instance, resource_group)", "interval": "", "legendFormat": "{{instance}}-{{resource_group}}", "queryType": "randomWalk", - "refId": "B" + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Hard Limit Exceeded Count", + "title": "Min TSO", "tooltip": { "shared": true, "sort": 0, @@ -5082,11 +5062,11 @@ { "decimals": null, "format": "none", - "label": "", + "label": "TSO", "logBase": 1, "max": null, "min": null, - "show": true + "show": false }, { "format": "short", @@ -5109,9 +5089,11 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": null, - "description": "the time of waiting for schedule", + "description": "estimated thread usage in min-tso scheduler, and the sort/hard limit of estimated thread in scheduler.", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, @@ -5120,10 +5102,10 @@ "h": 8, "w": 12, "x": 12, - "y": 20 + "y": 4 }, "hiddenSeries": false, - "id": 115, + "id": 109, "legend": { "alignAsTable": false, "avg": false, @@ -5153,37 +5135,54 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.80, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", - "hide": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_soft_limit\"}) by (instance, type, resource_group)", "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}-80", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "A" }, { "exemplar": true, - "expr": "histogram_quantile(0.90, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", - "hide": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"estimated_thread_usage\"}) by (instance, type, resource_group)", + "hide": false, "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}-90", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "B" }, { "exemplar": true, - "expr": "histogram_quantile(1.00, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"thread_hard_limit\"}) by (instance, type, resource_group)", "hide": false, "interval": "", - "legendFormat": "{{instance}}-{{resource_group}}-100", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", "queryType": "randomWalk", "refId": "C" + }, + { + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"global_estimated_thread_usage\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", + "refId": "D" + }, + { + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"group_entry_count\"}) by (instance, type)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "queryType": "randomWalk", + "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Task Waiting Duration", + "title": "Estimated Thread Usage and Limit", "tooltip": { "shared": true, "sort": 0, @@ -5200,9 +5199,9 @@ "yaxes": [ { "decimals": null, - "format": "s", - "label": "Time", - "logBase": 1, + "format": "none", + "label": "Threads", + "logBase": 10, "max": null, "min": null, "show": true @@ -5220,65 +5219,51 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Task Scheduler", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 4 - }, - "id": 16, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Executed DDL jobs per minute", + "decimals": null, + "description": "the count of active/ waiting queries", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 5 + "y": 12 }, "hiddenSeries": false, - "id": 19, + "id": 111, "legend": { "alignAsTable": false, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -5287,41 +5272,28 @@ "steppedLine": false, "targets": [ { - "expr": "avg(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{type}}", + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_queries_count\"}) by (instance, type, resource_group)", + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "A" }, { - "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "total", + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_queries_count\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "B" - }, - { - "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type,instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{type}}-{{instance}}", - "refId": "C" - }, - { - "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "total-{{instance}}", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Internal DDL OPM", + "title": "Active and Waiting Queries Count", "tooltip": { "shared": true, "sort": 0, @@ -5338,15 +5310,15 @@ "yaxes": [ { "decimals": null, - "format": "opm", - "label": null, + "format": "none", + "label": "Queries", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5365,42 +5337,44 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Executed DDL apply jobs per minute", + "decimals": null, + "description": "the count of active/ waiting tasks", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 5 + "y": 12 }, "hiddenSeries": false, - "id": 18, + "id": 113, "legend": { "alignAsTable": false, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -5409,18 +5383,28 @@ "steppedLine": false, "targets": [ { - "expr": "avg(increase(tiflash_schema_trigger_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "triggle-by-{{type}}", + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"waiting_tasks_count\"}) by (instance, type, resource_group)", + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "A" + }, + { + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"active_tasks_count\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{type}}-{{resource_group}}", + "queryType": "randomWalk", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Apply OPM", + "title": "Active and Waiting Tasks Count", "tooltip": { "shared": true, "sort": 0, @@ -5437,15 +5421,15 @@ "yaxes": [ { "decimals": null, - "format": "opm", - "label": null, + "format": "none", + "label": "Tasks", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5464,97 +5448,66 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": null, + "description": "the usage of estimated threads exceeded the hard limit where errors occur.", "fieldConfig": { - "defaults": {}, + "defaults": { + "unit": "none" + }, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 12 + "y": 20 }, "hiddenSeries": false, - "id": 20, + "id": 117, "legend": { "alignAsTable": false, "avg": false, - "current": false, - "hideZero": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/^applying/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "999-{{type}}", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "99-{{type}}", + "exemplar": true, + "expr": "max(tiflash_task_scheduler{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"hard_limit_exceeded_count\"}) by (instance, type, resource_group)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}", + "queryType": "randomWalk", "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "95-{{type}}", - "refId": "C" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", keyspace!=\"\"}[1m])) by (le, type, keyspace))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" - }, - { - "expr": "sum(tiflash_sync_schema_applying{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"$type\"}) by (instance)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "applying-{{instance}}", - "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema Apply Duration", + "title": "Hard Limit Exceeded Count", "tooltip": { "shared": true, "sort": 0, @@ -5570,19 +5523,20 @@ }, "yaxes": [ { - "format": "s", - "label": null, + "decimals": null, + "format": "none", + "label": "", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { "format": "short", "label": null, "logBase": 1, - "max": "2", - "min": "0", + "max": null, + "min": null, "show": true } ], @@ -5590,30 +5544,15 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "DDL", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 5 - }, - "id": 25, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total count of different kinds of commands received", + "decimals": null, + "description": "the time of waiting for schedule", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5623,66 +5562,71 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 6 + "x": 12, + "y": 20 }, "hiddenSeries": false, - "id": 41, + "id": 115, "legend": { "alignAsTable": false, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/delete_range|ingest/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "{{type}}", + "exemplar": true, + "expr": "histogram_quantile(0.80, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", + "hide": true, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}-80", + "queryType": "randomWalk", "refId": "A" }, { - "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "write block", + "exemplar": true, + "expr": "histogram_quantile(0.90, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", + "hide": true, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}-90", + "queryType": "randomWalk", "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(1.00, max(rate(tiflash_task_scheduler_waiting_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,le,resource_group))", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-{{resource_group}}-100", + "queryType": "randomWalk", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Command OPS", + "title": "Task Waiting Duration", "tooltip": { "shared": true, "sort": 0, @@ -5699,19 +5643,19 @@ "yaxes": [ { "decimals": null, - "format": "ops", - "label": null, + "format": "s", + "label": "Time", "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -5719,14 +5663,30 @@ "align": false, "alignLevel": null } - }, + } + ], + "repeat": null, + "title": "Task Scheduler", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 16, + "panels": [ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Executed DDL jobs per minute", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5734,28 +5694,28 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 6 + "x": 0, + "y": 5 }, "hiddenSeries": false, - "id": 38, + "id": 19, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, + "current": false, "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -5764,88 +5724,52 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/fs|write/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum by (instance) (\ntiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} +\ntiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} +\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)", + "expr": "avg(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "hide": false, - "interval": "", "intervalFactor": 1, - "legendFormat": "amp-total-{{instance}}", + "legendFormat": "{{type}}", "refId": "A" }, { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\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)", + "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", - "hide": false, - "interval": "", "intervalFactor": 1, - "legendFormat": "amp-5min-{{instance}}", + "legendFormat": "total", "refId": "B" }, { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[10m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[10m]) +\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)", + "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type,instance)", "format": "time_series", "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "amp-10min-{{instance}}", + "legendFormat": "{{type}}-{{instance}}", "refId": "C" }, { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30m]) +\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)", + "expr": "sum(increase(tiflash_schema_internal_ddl_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "amp-30min-{{instance}}", + "legendFormat": "total-{{instance}}", "refId": "D" - }, - { - "exemplar": true, - "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\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": "Schema Internal DDL OPM", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, - "transformations": [], "type": "graph", "xaxis": { "buckets": null, @@ -5857,15 +5781,15 @@ "yaxes": [ { "decimals": null, - "format": "short", + "format": "opm", "label": null, "logBase": 1, - "max": "20", + "max": null, "min": "0", "show": true }, { - "format": "binBps", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -5884,7 +5808,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage engine read tasks", + "description": "Executed DDL apply jobs per minute", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5892,23 +5816,23 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, - "w": 24, - "x": 0, - "y": 14 + "h": 7, + "w": 12, + "x": 12, + "y": 5 }, "hiddenSeries": false, - "id": 40, + "id": 18, "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -5928,11 +5852,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_read_tasks_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "avg(increase(tiflash_schema_trigger_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}", + "intervalFactor": 1, + "legendFormat": "triggle-by-{{type}}", "refId": "A" } ], @@ -5940,7 +5863,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read Tasks OPS", + "title": "Schema Apply OPM", "tooltip": { "shared": true, "sort": 0, @@ -5957,8 +5880,8 @@ "yaxes": [ { "decimals": null, - "format": "ops", - "label": "", + "format": "opm", + "label": null, "logBase": 1, "max": null, "min": "0", @@ -5984,8 +5907,1737 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of (maybe foreground) tasks of storage in bytes", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 12 + }, + "hiddenSeries": false, + "id": 20, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideZero": 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": [ + { + "alias": "/^applying/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "999-{{type}}", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_schema_apply_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", keyspace!=\"\"}[1m])) by (le, type, keyspace))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "refId": "D" + }, + { + "expr": "sum(tiflash_sync_schema_applying{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"$type\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "applying-{{instance}}", + "refId": "E" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Schema Apply Duration", + "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": "2", + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "repeat": null, + "title": "DDL", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 6 + }, + "id": 304, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiFlash CPU usage calculated with process CPU running seconds.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 1 + }, + "hiddenSeries": false, + "id": 306, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "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": "{{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": "CPU Usage (irate)", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 1, + "format": "percentunit", + "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": null, + "description": "", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 1 + }, + "hiddenSeries": false, + "id": 308, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "Limit", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2, + "nullPointMode": "connected" + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "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]))", + "format": "time_series", + "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": "Segment Reader", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 1, + "format": "percentunit", + "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}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 9 + }, + "hiddenSeries": false, + "id": 310, + "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": "sum(rate(tiflash_coprocessor_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type, instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Request QPS 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": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The flow of different kinds of read operations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 9 + }, + "height": "", + "hiddenSeries": false, + "id": 312, + "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(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "File Descriptor-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_PSMReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "Page-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "PageBackGround-{{instance}}", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Read Throughput by instance", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total count of different kinds of commands received", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 17 + }, + "hiddenSeries": false, + "id": 314, + "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 + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The throughput of write by instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 17 + }, + "height": "", + "hiddenSeries": false, + "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", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_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": "Write Throughput By Instance", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Imbalance read/write", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 7 + }, + "id": 318, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 8 + }, + "hiddenSeries": false, + "id": 320, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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": "keyspace-{{instance}}", + "refId": "I" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Number of Keyspaces", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 8 + }, + "hiddenSeries": false, + "id": 322, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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": "tables-{{instance}}", + "refId": "I" + }, + { + "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": "tables-all-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Number of Physical Tables", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 16 + }, + "hiddenSeries": false, + "id": 324, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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": "segments-{{instance}}", + "refId": "I" + }, + { + "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": "mem_table-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Number of Segments", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 16 + }, + "hiddenSeries": false, + "id": 326, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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": "bytes-{{instance}}", + "refId": "I" + }, + { + "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": "bytes-allocated-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Bytes of MemTables", + "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": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The memory usage of mark cache and minmax index cache", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 24 + }, + "hiddenSeries": false, + "id": 328, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/limit/", + "color": "#F2495C", + "hideTooltip": true, + "legend": false, + "linewidth": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_MarkCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "hide": false, + "interval": "", + "legendFormat": "mark_cache_{{instance}}", + "refId": "L" + }, + { + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_MinMaxIndexFiles{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "hide": false, + "interval": "", + "legendFormat": "minmax_index_cache_{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_RNMVCCIndexCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "hide": false, + "interval": "", + "legendFormat": "rn_mvcc_index_cache_{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Mark Cache and Minmax Index Cache Memory Usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "cache misses or cache hits of mark_cache.\nBased on this infactor, we can check whether mark_cache is large enough", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 24 + }, + "hiddenSeries": false, + "id": 330, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "max(tiflash_system_profile_event_MarkCacheMisses{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "interval": "", + "legendFormat": "mark cache misses", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "max(tiflash_system_profile_event_MarkCacheHits{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "mark cache hits", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Effectiveness of Mark Cache", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [ + { + "id": "calculateField", + "options": { + "alias": "mark cache count total", + "binary": { + "left": "mark cache misses", + "operator": "+", + "reducer": "sum", + "right": "mark cache hits" + }, + "mode": "binary", + "reduce": { + "reducer": "sum" + } + } + }, + { + "id": "calculateField", + "options": { + "alias": "mark cache effectiveness", + "binary": { + "left": "mark cache hits", + "operator": "/", + "reducer": "sum", + "right": "mark cache count total" + }, + "mode": "binary", + "reduce": { + "reducer": "sum" + } + } + }, + { + "id": "filterFieldsByName", + "options": { + "include": { + "names": [ + "Time", + "mark cache effectiveness" + ] + } + } + } + ], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "percent", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Information about schema of column file, to learn the memory usage of schema", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 32 + }, + "hiddenSeries": false, + "id": 332, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"current_size\"}) by (instance)", + "interval": "", + "legendFormat": "current_size-{{instance}}", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"hit_count\"}[1m])) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "hit_count_ops-{{instance}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"still_used_when_evict\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "still_used_when_evict-{{instance}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"miss_count\"}[1m])) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "miss_count_ops-{{instance}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Schema of Column File", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Memory trace", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 8 + }, + "id": 25, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total count of different kinds of commands received", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5996,31 +7648,25 @@ "h": 8, "w": 12, "x": 0, - "y": 22 + "y": 9 }, - "height": "", "hiddenSeries": false, - "id": 253, + "id": 41, "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 }, @@ -6029,10 +7675,9 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/total/", + "alias": "/delete_range|ingest/", "yaxis": 2 } ], @@ -6041,25 +7686,30 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_subtask_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(increase(tiflash_storage_command_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "hide": false, - "interval": "", - "intervalFactor": 1, + "intervalFactor": 2, "legendFormat": "{{type}}", - "refId": "A", - "step": 10 + "refId": "A" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "write block", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "SubTasks Write Throughput (bytes)", + "title": "Write Command OPS", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6072,7 +7722,8 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -6080,12 +7731,12 @@ "show": true }, { - "format": "bytes", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, - "show": false + "min": "0", + "show": true } ], "yaxis": { @@ -6096,11 +7747,10 @@ { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The throughput of (maybe foreground) tasks of storage in rows", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6111,24 +7761,18 @@ "h": 8, "w": 12, "x": 12, - "y": 22 + "y": 9 }, - "height": "", "hiddenSeries": false, - "id": 255, + "id": 38, "legend": { "alignAsTable": true, "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 }, @@ -6144,10 +7788,9 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/total/", + "alias": "/fs|write/", "yaxis": 2 } ], @@ -6157,26 +7800,76 @@ "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": "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": 1, - "legendFormat": "{{type}}", - "refId": "A", - "step": 10 + "legendFormat": "amp-total-{{instance}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum by (instance) (\nrate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\nrate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) +\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": "", + "intervalFactor": 1, + "legendFormat": "amp-5min-{{instance}}", + "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": "SubTasks Write Throughput (rows)", + "title": "Write Amplification", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, + "transformations": [], "type": "graph", "xaxis": { "buckets": null, @@ -6187,20 +7880,21 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": null, + "format": "short", "label": null, "logBase": 1, - "max": null, + "max": "20", "min": "0", "show": true }, { - "format": "bytes", + "format": "binBps", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -6214,7 +7908,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage's internal sub tasks", + "description": "Total number of storage engine read tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6222,24 +7916,23 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 5, - "w": 12, + "h": 8, + "w": 24, "x": 0, - "y": 30 + "y": 17 }, "hiddenSeries": false, - "id": 39, + "id": 40, "legend": { "alignAsTable": true, - "avg": false, - "current": false, - "max": false, + "avg": true, + "current": true, + "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -6256,16 +7949,14 @@ "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": true, + "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", + "expr": "sum(rate(tiflash_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" } ], @@ -6273,7 +7964,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Small Internal Tasks OPS", + "title": "Read Tasks OPS", "tooltip": { "shared": true, "sort": 0, @@ -6289,21 +7980,21 @@ }, "yaxes": [ { - "decimals": 1, + "decimals": null, "format": "ops", - "label": null, + "label": "", "logBase": 1, "max": null, "min": "0", "show": true }, { - "format": "opm", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", - "show": false + "min": null, + "show": true } ], "yaxis": { @@ -6317,7 +8008,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of storage's internal sub tasks", + "decimals": 1, + "description": "The throughput of (maybe foreground) tasks of storage in bytes", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6325,30 +8017,34 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 8, "w": 12, - "x": 12, - "y": 30 + "x": 0, + "y": 25 }, + "height": "", "hiddenSeries": false, - "id": 42, + "id": 253, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": false, "max": true, "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 }, @@ -6357,30 +8053,37 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "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)", + "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": 2, - "legendFormat": "max-{{type}}", - "refId": "A" + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Small Internal Tasks Duration", + "title": "SubTasks Write Throughput (bytes)", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -6393,8 +8096,7 @@ }, "yaxes": [ { - "decimals": 1, - "format": "s", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -6402,11 +8104,11 @@ "show": true }, { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": false } ], @@ -6421,7 +8123,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of storage's internal sub tasks", + "decimals": 1, + "description": "The throughput of (maybe foreground) tasks of storage in rows", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6429,29 +8132,34 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 5, + "h": 8, "w": 12, - "x": 0, - "y": 35 + "x": 12, + "y": 25 }, + "height": "", "hiddenSeries": false, - "id": 130, + "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": 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,30 +8168,37 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, - "steppedLine": true, + "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", + "expr": "sum(rate(tiflash_storage_subtask_throughput_rows{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "hide": false, "interval": "", - "intervalFactor": 2, + "intervalFactor": 1, "legendFormat": "{{type}}", - "refId": "A" + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Large Internal Tasks OPS", + "title": "SubTasks Write Throughput (rows)", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -6496,8 +8211,7 @@ }, "yaxes": [ { - "decimals": 1, - "format": "ops", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -6505,11 +8219,11 @@ "show": true }, { - "format": "opm", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": false } ], @@ -6524,7 +8238,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of storage's internal sub tasks", + "description": "Total number of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6534,23 +8248,22 @@ "gridPos": { "h": 5, "w": 12, - "x": 12, - "y": 35 + "x": 0, + "y": 33 }, "hiddenSeries": false, - "id": 131, + "id": 39, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, "rightSide": true, "show": true, - "sort": null, - "sortDesc": null, + "sideWidth": null, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -6567,16 +8280,16 @@ "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": false, + "steppedLine": true, "targets": [ { "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval]))) by (le,type) / 1000000000)", + "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "max-{{type}}", + "legendFormat": "{{type}}", "refId": "A" } ], @@ -6584,7 +8297,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Large Internal Tasks Duration", + "title": "Small Internal Tasks OPS", "tooltip": { "shared": true, "sort": 0, @@ -6601,7 +8314,7 @@ "yaxes": [ { "decimals": 1, - "format": "s", + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -6609,7 +8322,7 @@ "show": true }, { - "format": "s", + "format": "opm", "label": null, "logBase": 1, "max": null, @@ -6628,7 +8341,7 @@ "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)", + "description": "Duration of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6636,33 +8349,30 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 40 + "h": 5, + "w": 12, + "x": 12, + "y": 33 }, "hiddenSeries": false, - "id": 50, + "id": 42, "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, + "sort": null, + "sortDesc": null, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -6677,49 +8387,21 @@ "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)", + "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": "RW-{{instance}}", - "refId": "D" + "intervalFactor": 2, + "legendFormat": "max-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Opened File Count", + "title": "Small Internal Tasks Duration", "tooltip": { "shared": true, "sort": 0, @@ -6735,7 +8417,8 @@ }, "yaxes": [ { - "format": "none", + "decimals": 1, + "format": "s", "label": null, "logBase": 1, "max": null, @@ -6743,11 +8426,11 @@ "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": false } ], @@ -6762,7 +8445,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of open file descriptors action.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", + "description": "Total number of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6770,33 +8453,29 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 40 + "h": 5, + "w": 12, + "x": 0, + "y": 38 }, "hiddenSeries": false, - "id": 22, + "id": 130, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": 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 }, @@ -6808,32 +8487,24 @@ "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": false, + "steppedLine": true, "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_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", "format": "time_series", + "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "Open-{{instance}}", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_FileOpenFailed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "OpenFail-{{instance}}", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "File Open OPS", + "title": "Large Internal Tasks OPS", "tooltip": { "shared": true, "sort": 0, @@ -6849,6 +8520,7 @@ }, "yaxes": [ { + "decimals": 1, "format": "ops", "label": null, "logBase": 1, @@ -6857,11 +8529,11 @@ "show": true }, { - "format": "short", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": false } ], @@ -6876,7 +8548,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": "Duration of storage's internal sub tasks", "fieldConfig": { "defaults": {}, "overrides": [] @@ -6884,13 +8556,13 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 40 + "h": 5, + "w": 12, + "x": 12, + "y": 38 }, "hiddenSeries": false, - "id": 52, + "id": 131, "legend": { "alignAsTable": true, "avg": false, @@ -6899,9 +8571,8 @@ "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -6917,39 +8588,27 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/max-fsync/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_FileFSync{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "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", - "interval": "", - "intervalFactor": 1, - "legendFormat": "ops-fsync-{{instance}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_system_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"fsync\"}[$__rate_interval]))) by (le, instance) / 1000000000)", "hide": false, "interval": "", - "legendFormat": "max-fsync-{{instance}}", - "refId": "B" + "intervalFactor": 2, + "legendFormat": "max-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "FSync Status", + "title": "Large Internal Tasks Duration", "tooltip": { "shared": true, "sort": 0, @@ -6965,8 +8624,8 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "decimals": 1, + "format": "s", "label": null, "logBase": 1, "max": null, @@ -6978,8 +8637,8 @@ "label": null, "logBase": 1, "max": null, - "min": null, - "show": true + "min": "0", + "show": false } ], "yaxis": { @@ -6993,7 +8652,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of different kinds of read operations", + "description": "The number of currently opened file descriptors.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7002,29 +8661,32 @@ "fillGradient": 0, "gridPos": { "h": 7, - "w": 12, + "w": 8, "x": 0, - "y": 47 + "y": 43 }, "hiddenSeries": false, - "id": 46, + "id": 50, "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 }, @@ -7039,43 +8701,52 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_PSMWriteIOCalls{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "tiflash_proxy_process_open_fds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tiflash\"}", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "Page", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{instance}}", "refId": "A" }, { - "expr": "sum(rate(tiflash_system_profile_event_PSMWritePages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "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": true, - "intervalFactor": 2, - "legendFormat": "PageFile", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "R-{{instance}}", "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(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": "File Descriptor", + "legendFormat": "RW-{{instance}}", "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": "Disk Write OPS", + "title": "Opened File Count", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -7088,8 +8759,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -7097,12 +8767,12 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7116,7 +8786,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of different kinds of read operations", + "description": "The number of open file descriptors action.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7125,29 +8795,32 @@ "fillGradient": 0, "gridPos": { "h": 7, - "w": 12, - "x": 12, - "y": 47 + "w": 8, + "x": 8, + "y": 43 }, "hiddenSeries": false, - "id": 47, + "id": 22, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, + "hideEmpty": true, "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": 250, + "sort": "max", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -7162,43 +8835,32 @@ "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": "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": "File Descriptor", - "refId": "D" + "legendFormat": "Open-{{instance}}", + "refId": "A" }, { - "expr": "sum(rate(tiflash_system_profile_event_ReadBufferAIORead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "exemplar": true, + "expr": "sum(rate(tiflash_system_profile_event_FileOpenFailed{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "AIO", - "refId": "F" + "legendFormat": "OpenFail-{{instance}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Disk Read OPS", + "title": "File Open OPS", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -7211,7 +8873,6 @@ }, "yaxes": [ { - "decimals": null, "format": "ops", "label": null, "logBase": 1, @@ -7220,12 +8881,12 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7239,35 +8900,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of different kinds of write operations", + "description": "OPS and duration of fsync operations.\n(Only counting storage engine of TiFlash by now. Not including TiFlash-Proxy)", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 54 + "h": 7, + "w": 8, + "x": 16, + "y": 43 }, - "height": "", "hiddenSeries": false, - "id": 60, + "id": 52, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", + "sideWidth": 250, + "sort": "max", "sortDesc": true, "total": false, "values": true @@ -7275,7 +8932,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -7284,52 +8941,42 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/max-fsync/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "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": "sum(rate(tiflash_system_profile_event_FileFSync{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "PageBackGround", - "refId": "C" + "legendFormat": "ops-fsync-{{instance}}", + "refId": "A" }, { - "expr": "sum(rate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", + "exemplar": true, + "expr": "histogram_quantile(1.00, sum(round(1000000000*rate(tiflash_system_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"fsync\"}[$__rate_interval]))) by (le, instance) / 1000000000)", + "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "AIO", - "refId": "D" + "legendFormat": "max-fsync-{{instance}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write flow", + "title": "FSync Status", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -7342,7 +8989,8 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -7350,11 +8998,11 @@ "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -7369,23 +9017,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of different kinds of read operations", + "description": "The number of different kinds of read operations", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 54 + "x": 0, + "y": 50 }, - "height": "", "hiddenSeries": false, - "id": 59, + "id": 46, "legend": { "alignAsTable": true, "avg": false, @@ -7396,16 +9042,13 @@ "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 }, @@ -7414,48 +9057,46 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "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": 2, - "legendFormat": "File Descriptor", - "refId": "A", - "step": 10 + "legendFormat": "Page", + "refId": "A" }, { - "expr": "sum(rate(tiflash_system_profile_event_PSMReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "expr": "sum(rate(tiflash_system_profile_event_PSMWritePages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "Page", - "refId": "B" + "hide": true, + "intervalFactor": 2, + "legendFormat": "PageFile", + "refId": "C" }, { - "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWrite{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "PageBackGround", - "refId": "C" + "legendFormat": "File Descriptor", + "refId": "D" }, { - "expr": "sum(rate(tiflash_system_profile_event_ReadBufferAIOReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "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": "D" + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Read flow", + "title": "Disk Write OPS", "tooltip": { "shared": true, "sort": 2, @@ -7471,7 +9112,8 @@ }, "yaxes": [ { - "format": "binBps", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -7479,11 +9121,11 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -7495,11 +9137,10 @@ { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The current processing number of segments' background management", + "description": "The number of different kinds of read operations", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7507,18 +9148,20 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 0, - "y": 62 + "x": 12, + "y": 50 }, "hiddenSeries": false, - "id": 67, + "id": 47, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "hideEmpty": false, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, @@ -7543,36 +9186,43 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tiflash_system_current_metric_DT_DeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_system_profile_event_PSMReadIOCalls{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "delta_merge-{{instance}}", + "intervalFactor": 2, + "legendFormat": "Page", "refId": "A" }, { - "expr": "avg(tiflash_system_current_metric_DT_SegmentSplit{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "expr": "sum(rate(tiflash_system_profile_event_PSMReadPages{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "PageFile", + "refId": "C" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorRead{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "seg_split-{{instance}}", - "refId": "B" + "legendFormat": "File Descriptor", + "refId": "D" }, { - "expr": "avg(tiflash_system_current_metric_DT_SegmentMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "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": "seg_merge-{{instance}}", - "refId": "C" + "legendFormat": "AIO", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Current Data Management Tasks", + "title": "Disk Read OPS", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -7585,8 +9235,8 @@ }, "yaxes": [ { - "decimals": 0, - "format": "short", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -7613,36 +9263,43 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Errors of DeltaIndex", + "decimals": 1, + "description": "The flow of different kinds of write operations", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 62 + "x": 0, + "y": 57 }, + "height": "", "hiddenSeries": false, - "id": 237, + "id": 60, "legend": { "alignAsTable": true, "avg": false, - "current": 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 as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -7651,30 +9308,52 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_DTDeltaIndexError{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "sum(rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", "hide": false, + "intervalFactor": 2, + "legendFormat": "File Descriptor", + "refId": "A", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_system_profile_event_PSMWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "Page", + "refId": "B" + }, + { + "expr": "sum(rate(tiflash_system_profile_event_PSMBackgroundWriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "PageBackGround", + "refId": "C" + }, + { + "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": "DeltaIndexError-{{instance}}", - "refId": "A" + "legendFormat": "AIO", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "DeltaIndexError", + "title": "Write flow", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -7687,8 +9366,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "cps", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -7696,12 +9374,12 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, "min": "0", - "show": false + "show": true } ], "yaxis": { @@ -7715,7 +9393,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The storage I/O limiter metrics.", + "decimals": 1, + "description": "The flow of different kinds of read operations", "fieldConfig": { "defaults": {}, "overrides": [] @@ -7725,56 +9404,85 @@ "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 70 + "x": 12, + "y": 57 }, + "height": "", "hiddenSeries": false, - "id": 84, + "id": 59, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "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, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_io_limiter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_system_profile_event_ReadBufferFromFileDescriptorReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", - "instant": false, + "hide": false, "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A" + "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" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Throughput", + "title": "Read flow", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -7787,7 +9495,6 @@ }, "yaxes": [ { - "decimals": 0, "format": "binBps", "label": null, "logBase": 1, @@ -7800,7 +9507,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -7812,29 +9519,30 @@ { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The storage I/O limiter metrics.", + "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": 70 + "x": 0, + "y": 65 }, "hiddenSeries": false, - "id": 266, + "id": 67, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -7850,36 +9558,42 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, + "pointradius": 5, "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)", + "expr": "avg(tiflash_system_current_metric_DT_DeltaMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", - "instant": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{type}}", + "hide": false, + "intervalFactor": 1, + "legendFormat": "delta_merge-{{instance}}", "refId": "A" + }, + { + "expr": "avg(tiflash_system_current_metric_DT_SegmentSplit{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "seg_split-{{instance}}", + "refId": "B" + }, + { + "expr": "avg(tiflash_system_current_metric_DT_SegmentMerge{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "seg_merge-{{instance}}", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Pending Rate and Duration", + "title": "Current Data Management Tasks", "tooltip": { "shared": true, "sort": 0, @@ -7895,9 +9609,8 @@ }, "yaxes": [ { - "$$hashKey": "object:230", "decimals": 0, - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -7905,8 +9618,7 @@ "show": true }, { - "$$hashKey": "object:231", - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -7925,117 +9637,65 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "I/O Limiter current pending count.", + "description": "Errors of DeltaIndex", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 70 + "y": 65 }, "hiddenSeries": false, - "id": 86, + "id": 237, "legend": { "alignAsTable": true, "avg": false, "current": false, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 2, - "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", - "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" - }, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_io_limiter_pending_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_system_profile_event_DTDeltaIndexError{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "format": "time_series", "hide": false, "interval": "", - "legendFormat": "{{type}}-pending-P99", - "refId": "G" + "intervalFactor": 1, + "legendFormat": "DeltaIndexError-{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "I/O Limiter Current Pending Count", + "title": "DeltaIndexError", "tooltip": { "shared": true, "sort": 0, @@ -8051,8 +9711,8 @@ }, "yaxes": [ { - "decimals": 0, - "format": "short", + "decimals": null, + "format": "cps", "label": null, "logBase": 1, "max": null, @@ -8060,12 +9720,12 @@ "show": true }, { - "format": "s", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, - "show": true + "min": "0", + "show": false } ], "yaxis": { @@ -8090,7 +9750,7 @@ "h": 8, "w": 12, "x": 0, - "y": 78 + "y": 73 }, "hiddenSeries": false, "id": 132, @@ -8222,7 +9882,7 @@ "h": 8, "w": 12, "x": 12, - "y": 78 + "y": 73 }, "hiddenSeries": false, "id": 269, @@ -8251,7 +9911,6 @@ "renderer": "flot", "seriesOverrides": [ { - "$$hashKey": "object:308", "alias": "/push_block/", "yaxis": 2 } @@ -8291,7 +9950,6 @@ }, "yaxes": [ { - "$$hashKey": "object:321", "decimals": null, "format": "ops", "label": null, @@ -8301,7 +9959,6 @@ "show": true }, { - "$$hashKey": "object:322", "format": "binBps", "label": null, "logBase": 1, @@ -8315,6 +9972,132 @@ "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": 81 + }, + "hiddenSeries": false, + "id": 292, + "legend": { + "alignAsTable": false, + "avg": true, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "999-{{type}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "99-{{type}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "95-{{type}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.80, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "80-{{type}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Read Thread Internal Duration", + "tooltip": { + "shared": true, + "sort": 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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "aliasColors": {}, "bars": false, @@ -8332,7 +10115,7 @@ "h": 8, "w": 12, "x": 12, - "y": 78 + "y": 81 }, "hiddenSeries": false, "id": 88, @@ -8513,14 +10296,30 @@ "align": false, "alignLevel": null } - }, + } + ], + "repeat": null, + "title": "Storage", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 9 + }, + "id": 334, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "cache misses or cache hits of mark_cache.\nBased on this infactor, we can check whether mark_cache is large enough", + "description": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8531,107 +10330,57 @@ "h": 8, "w": 12, "x": 0, - "y": 86 + "y": 10 }, "hiddenSeries": false, - "id": 169, + "id": 84, "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, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "max(tiflash_system_profile_event_MarkCacheMisses{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "interval": "", - "legendFormat": "mark cache misses", - "queryType": "randomWalk", - "refId": "A" - }, - { - "exemplar": true, - "expr": "max(tiflash_system_profile_event_MarkCacheHits{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "mark cache hits", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Effectiveness of Mark Cache", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "transformations": [ - { - "id": "calculateField", - "options": { - "alias": "mark cache count total", - "binary": { - "left": "mark cache misses", - "operator": "+", - "reducer": "sum", - "right": "mark cache hits" - }, - "mode": "binary", - "reduce": { - "reducer": "sum" - } - } - }, - { - "id": "calculateField", - "options": { - "alias": "mark cache effectiveness", - "binary": { - "left": "mark cache hits", - "operator": "/", - "reducer": "sum", - "right": "mark cache count total" - }, - "mode": "binary", - "reduce": { - "reducer": "sum" - } - } - }, + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "id": "filterFieldsByName", - "options": { - "include": { - "names": [ - "Time", - "mark cache effectiveness" - ] - } - } + "expr": "sum(rate(tiflash_storage_io_limiter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" } ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "I/O Limiter Throughput", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, "type": "graph", "xaxis": { "buckets": null, @@ -8642,20 +10391,21 @@ }, "yaxes": [ { - "format": "percentunit", + "decimals": 0, + "format": "binBps", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "percent", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -8669,7 +10419,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Information about schema of column file, to learn the memory usage of schema", + "description": "Current limit bytes per second of Storage I/O limiter", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8680,22 +10430,27 @@ "h": 8, "w": 12, "x": 12, - "y": 86 + "y": 10 }, "hiddenSeries": false, - "id": 168, + "id": 336, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": true, "show": true, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, - "nullPointMode": "null", + "links": [], + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -8706,47 +10461,25 @@ "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": false, + "stack": true, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"current_size\"}) by (instance)", + "expr": "sum(tiflash_storage_io_limiter_curr{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", + "format": "time_series", + "instant": false, "interval": "", - "legendFormat": "current_size", - "queryType": "randomWalk", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"hit_count\"}[1m])) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "hit_count_ops", - "refId": "B" - }, - { - "exemplar": true, - "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"still_used_when_evict\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "still_used_when_evict", - "refId": "C" - }, - { - "exemplar": true, - "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"miss_count\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "miss_count", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Schema of Column File", + "title": "I/O Limiter Threshold", "tooltip": { "shared": true, "sort": 0, @@ -8762,11 +10495,12 @@ }, "yaxes": [ { - "format": "short", + "decimals": 0, + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -8789,6 +10523,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The storage I/O limiter metrics.", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8799,15 +10534,15 @@ "h": 8, "w": 12, "x": 0, - "y": 165 + "y": 18 }, "hiddenSeries": false, - "id": 292, + "id": 266, "legend": { - "alignAsTable": false, - "avg": true, - "current": false, - "max": false, + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, "min": false, "rightSide": true, "show": true, @@ -8822,57 +10557,36 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.17", - "pointradius": 5, + "pluginVersion": "7.5.11", + "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/-/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.999, sum(rate(tiflash_read_thread_internal_us_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "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": 1, - "legendFormat": "999-{{type}}", + "intervalFactor": 2, + "legendFormat": "{{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": "I/O Limiter Pending Rate and Duration", "tooltip": { "shared": true, "sort": 0, @@ -8888,8 +10602,8 @@ }, "yaxes": [ { - "$$hashKey": "object:222", - "format": "µs", + "decimals": 0, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -8897,13 +10611,12 @@ "show": true }, { - "$$hashKey": "object:223", - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -8917,34 +10630,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The memory usage of mark cache and minmax index cache", + "description": "I/O Limiter current pending count.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 94 + "x": 12, + "y": 18 }, "hiddenSeries": false, - "id": 238, + "id": 86, "legend": { "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -8955,52 +10665,82 @@ }, "percentage": false, "pluginVersion": "7.5.11", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/limit/", - "color": "#F2495C", - "hideTooltip": true, - "legend": false, - "linewidth": 2 + "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": "tiflash_system_asynchronous_metric_MarkCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "mark_cache_{{instance}}", - "refId": "L" + "legendFormat": "bgwrite-current-{{instance}}", + "refId": "B" }, { "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_MinMaxIndexFiles{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingFgWriteReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "minmax_index_cache_{{instance}}", - "refId": "A" + "legendFormat": "fgwrite-current-{{instance}}", + "refId": "C" }, { "exemplar": true, - "expr": "tiflash_system_asynchronous_metric_RNDeltaIndexCacheBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "avg(tiflash_system_current_metric_IOLimiterPendingBgReadReq{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "rn_delta_index_cache_{{instance}}", - "refId": "B" + "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": "Mark Cache and Minmax Index Cache Memory Usage", + "title": "I/O Limiter Current Pending Count", "tooltip": { "shared": true, "sort": 0, @@ -9016,7 +10756,8 @@ }, "yaxes": [ { - "format": "bytes", + "decimals": 0, + "format": "short", "label": null, "logBase": 1, "max": null, @@ -9024,12 +10765,12 @@ "show": true }, { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -9038,8 +10779,7 @@ } } ], - "repeat": null, - "title": "Storage", + "title": "Rate Limiter", "type": "row" }, { @@ -9049,7 +10789,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 10 }, "id": 64, "panels": [ @@ -9654,7 +11394,7 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 11 }, "id": 119, "panels": [ @@ -10917,7 +12657,7 @@ "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 12 }, "id": 34, "panels": [ @@ -13473,7 +15213,7 @@ "h": 1, "w": 24, "x": 0, - "y": 9 + "y": 13 }, "id": 95, "panels": [ @@ -13703,7 +15443,7 @@ "h": 1, "w": 24, "x": 0, - "y": 10 + "y": 14 }, "id": 171, "panels": [ @@ -15337,7 +17077,7 @@ "h": 1, "w": 24, "x": 0, - "y": 11 + "y": 15 }, "id": 172, "panels": [ @@ -16243,7 +17983,7 @@ "h": 1, "w": 24, "x": 0, - "y": 12 + "y": 16 }, "id": 202, "panels": [ @@ -17283,7 +19023,7 @@ "h": 1, "w": 24, "x": 0, - "y": 13 + "y": 17 }, "id": 248, "panels": [