From c1f8bcac3cf8bb339dd2f10b6291c14fa8b23f54 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Thu, 13 Jun 2024 19:18:26 +0800 Subject: [PATCH] Storages: Track memory for PageDirectory (#9134) close pingcap/tiflash#8835 Signed-off-by: CalvinNeo Co-authored-by: JaySon-Huang --- dbms/src/Common/TiFlashMetrics.h | 5 + dbms/src/Interpreters/AsynchronousMetrics.cpp | 2 + .../KVStore/FFI/JointThreadAllocInfo.cpp | 11 + .../KVStore/FFI/JointThreadAllocInfo.h | 2 + dbms/src/Storages/KVStore/FFI/SSTReader.h | 46 ++-- .../KVStore/MultiRaft/PrehandleSnapshot.cpp | 25 ++- .../KVStore/tests/gtest_new_kvstore.cpp | 5 + .../KVStore/tests/region_kvstore_test.h | 2 +- .../Storages/Page/PageStorageMemorySummary.h | 28 +++ dbms/src/Storages/Page/V3/PageDirectory.h | 48 +++-- .../Page/V3/PageDirectory/PageIdTrait.h | 3 + dbms/src/Storages/Page/V3/PageEntry.h | 1 + .../Page/V3/Universal/UniversalPageId.h | 31 ++- metrics/grafana/tiflash_summary.json | 201 ++++++++++++++---- 14 files changed, 311 insertions(+), 99 deletions(-) create mode 100644 dbms/src/Storages/Page/PageStorageMemorySummary.h diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index a679ed86bdf..ce3849bf7db 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -212,6 +212,11 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva "The freshness of tiflash data with tikv data", \ Histogram, \ F(type_syncing_data_freshness, {{"type", "data_freshness"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_memory_usage_by_class, \ + "TiFlash memory consumes by class", \ + Gauge, \ + F(type_uni_page_ids, {"type", "uni_page_ids"}), \ + F(type_versioned_entries, {"type", "versioned_entries"})) \ M(tiflash_storage_read_tasks_count, "Total number of storage engine read tasks", Counter) \ M(tiflash_storage_command_count, \ "Total number of storage's command, such as delete range / shutdown /startup", \ diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index 90d20a1f381..b0fd613b170 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -300,6 +301,7 @@ void AsynchronousMetrics::update() set("LogNums", usage.total_log_file_num); set("LogDiskBytes", usage.total_log_disk_size); set("PagesInMem", usage.num_pages); + set("VersionedEntries", DB::PS::PageStorageMemorySummary::versioned_entry_or_delete_count.load()); } if (context.getSharedContextDisagg()->isDisaggregatedStorageMode()) diff --git a/dbms/src/Storages/KVStore/FFI/JointThreadAllocInfo.cpp b/dbms/src/Storages/KVStore/FFI/JointThreadAllocInfo.cpp index b1fd26eb16b..2bef3a5a173 100644 --- a/dbms/src/Storages/KVStore/FFI/JointThreadAllocInfo.cpp +++ b/dbms/src/Storages/KVStore/FFI/JointThreadAllocInfo.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -47,6 +48,7 @@ void JointThreadInfoJeallocMap::recordThreadAllocInfo() { recordThreadAllocInfoForProxy(); recordThreadAllocInfoForStorage(); + recordClassdAlloc(); } JointThreadInfoJeallocMap::~JointThreadInfoJeallocMap() @@ -269,4 +271,13 @@ void JointThreadInfoJeallocMap::accessStorageMap(std::function findSplitKeys(uint64_t splits_count) const override; void seek(BaseBuffView && view) const override; @@ -147,20 +147,27 @@ class MultiSSTReader : public SSTReader } size_t getSplitId() const override { return split_id; } - // Switch to next mono reader if current is drained, + // Switch to next mono reader if current SST is drained, // and we have a next sst file to read. - void maybeNextReader() const + void maybeNextReader() { - if (!mono->remained()) + if (likely(mono->remained())) + return; + + sst_idx++; + if (sst_idx < args.size()) { - current++; - if (current < args.size()) - { - // We don't drop if mono is the last instance for safety, - // and it will be dropped as MultiSSTReader is dropped. - LOG_INFO(log, "Open sst file {}", buffToStrView(args[current].path)); - mono = initer(proxy_helper, args[current], range, split_id); - } + // We don't drop if mono is the last instance for safety, + // and it will be dropped as MultiSSTReader is dropped. + LOG_INFO( + log, + "Open sst file {}, range={} sst_idx={} sst_tot={} split_id={}", + buffToStrView(args[sst_idx].path), + range->toDebugString(), + sst_idx, + args.size(), + split_id); + mono = initer(proxy_helper, args[sst_idx], range, split_id); } } @@ -177,18 +184,19 @@ class MultiSSTReader : public SSTReader , type(type_) , initer(initer_) , args(args_) - , current(0) + , sst_idx(0) , range(range_) , split_id(split_id_) { assert(args.size() > 0); LOG_INFO( log, - "Open sst file first {} range {} split_id={}", - buffToStrView(args[current].path), + "Open sst file first {}, range={} sst_tot={} split_id={}", + buffToStrView(args[sst_idx].path), range->toDebugString(), + args.size(), split_id); - mono = initer(proxy_helper, args[current], range, split_id); + mono = initer(proxy_helper, args[sst_idx], range, split_id); } ~MultiSSTReader() override @@ -202,12 +210,12 @@ class MultiSSTReader : public SSTReader /// The instance is ill-formed if the size of `args` is zero. mutable std::unique_ptr mono; const TiFlashRaftProxyHelper * proxy_helper; - ColumnFamilyType type; + const ColumnFamilyType type; Initer initer; std::vector args; - mutable size_t current; + size_t sst_idx; RegionRangeFilter range; - size_t split_id; + const size_t split_id; }; } // namespace DB diff --git a/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp b/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp index db41fb67407..f832b19866c 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp @@ -84,7 +84,7 @@ void PreHandlingTrace::waitForSubtaskResources(uint64_t region_id, size_t parall { LOG_DEBUG( log, - "Prehandle resource meet, limit={}, current={}, region_id={}", + "Prehandle resource meet, limit={} current={} region_id={}", parallel_subtask_limit, ongoing_prehandle_subtask_count.load(), region_id); @@ -398,8 +398,7 @@ static inline std::pair, size_t> getSplitKey( LOG_INFO( log, "getSplitKey result {}, total_concurrency={} ongoing={} total_split_parts={} split_keys={} " - "region_range={} approx_bytes={} " - "region_id={}", + "region_range={} approx_bytes={} region_id={}", fmt_buf.toString(), total_concurrency, ongoing_count, @@ -448,7 +447,7 @@ static void runInParallel( = executeTransform(log, prehandle_ctx, part_new_region, part_sst_stream); LOG_INFO( log, - "Finished extra parallel prehandle task limit {} write_cf={} lock_cf={} default_cf={} dmfiles={} error={}, " + "Finished extra parallel prehandle task limit {} write_cf={} lock_cf={} default_cf={} dmfiles={} error={} " "split_id={} region_id={}", limit_tag, part_prehandle_result.stats.write_cf_keys, @@ -476,8 +475,7 @@ static void runInParallel( LOG_INFO( log, "Parallel prehandling error {}" - " write_cf_off={}" - " split_id={} region_id={}", + " write_cf_off={} split_id={} region_id={}", e.message(), processed_keys.write_cf, extra_id, @@ -509,10 +507,11 @@ void executeParallelTransform( split_key_count); LOG_INFO( log, - "Parallel prehandling for single big region, range={}, split keys={}, region_id={}", + "Parallel prehandling for single big region, range={} split_keys={} region_id={} snaps={}", new_region->getRange()->toDebugString(), split_key_count, - new_region->id()); + new_region->id(), + snaps.len); Stopwatch watch; // Make sure the queue is bigger than `split_key_count`, otherwise `addTask` may fail. auto async_tasks = SingleSnapshotAsyncTasks(split_key_count, split_key_count, split_key_count + 5); @@ -550,9 +549,8 @@ void executeParallelTransform( auto [head_result, head_prehandle_result] = executeTransform(log, prehandle_ctx, new_region, sst_stream); LOG_INFO( log, - "Finished extra parallel prehandle task limit={} write_cf {} lock_cf={} default_cf={} dmfiles={} " - "error={}, split_id={}, " - "region_id={}", + "Finished extra parallel prehandle task, limit={} write_cf={} lock_cf={} default_cf={} dmfiles={} " + "error={} split_id={} region_id={}", sst_stream->getSoftLimit()->toDebugString(), head_prehandle_result.stats.write_cf_keys, head_prehandle_result.stats.lock_cf_keys, @@ -714,9 +712,10 @@ PrehandleResult KVStore::preHandleSSTsToDTFiles( { LOG_INFO( log, - "Single threaded prehandling for single region, range={} region_id={}", + "Single threaded prehandling for single region, range={} region_id={} snaps={}", new_region->getRange()->toDebugString(), - new_region->id()); + new_region->id(), + snaps.len); std::tie(result, prehandle_result) = executeTransform(log, prehandle_ctx, new_region, sst_stream); } else diff --git a/dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp index 2afa8ead3cd..e3ddb37da1b 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp @@ -20,6 +20,11 @@ #include #include #include +#include +#include +#include +#include +#include #include #include #include diff --git a/dbms/src/Storages/KVStore/tests/region_kvstore_test.h b/dbms/src/Storages/KVStore/tests/region_kvstore_test.h index 9f49ef724e9..67342b033a3 100644 --- a/dbms/src/Storages/KVStore/tests/region_kvstore_test.h +++ b/dbms/src/Storages/KVStore/tests/region_kvstore_test.h @@ -62,7 +62,7 @@ inline void validateSSTGeneration( size_t split_id) -> std::unique_ptr { auto parsed_kind = MockSSTGenerator::parseSSTViewKind(buffToStrView(snap.path)); auto reader = std::make_unique(proxy_helper, snap, range, split_id); - assert(reader->sst_format_kind() == parsed_kind); + assert(reader->sstFormatKind() == parsed_kind); return reader; }; MultiSSTReader reader{ diff --git a/dbms/src/Storages/Page/PageStorageMemorySummary.h b/dbms/src/Storages/Page/PageStorageMemorySummary.h new file mode 100644 index 00000000000..9c1717fed48 --- /dev/null +++ b/dbms/src/Storages/Page/PageStorageMemorySummary.h @@ -0,0 +1,28 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +namespace DB::PS +{ +struct PageStorageMemorySummary +{ + static inline std::atomic_int64_t uni_page_id_bytes{0}; + static inline std::atomic_int64_t versioned_entry_or_delete_bytes{0}; + static inline std::atomic_int64_t versioned_entry_or_delete_count{0}; +}; + +} // namespace DB::PS \ No newline at end of file diff --git a/dbms/src/Storages/Page/V3/PageDirectory.h b/dbms/src/Storages/Page/V3/PageDirectory.h index f6fb9fb8907..5103e327ddc 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.h +++ b/dbms/src/Storages/Page/V3/PageDirectory.h @@ -222,31 +222,47 @@ struct EntryOrDelete MultiVersionRefCount being_ref_count; std::optional entry; - static EntryOrDelete newDelete() + EntryOrDelete(const EntryOrDelete & other) + : being_ref_count(other.being_ref_count) + , entry(other.entry) { - return EntryOrDelete{ - .entry = std::nullopt, - }; - }; - static EntryOrDelete newNormalEntry(const PageEntryV3 & entry) + PageStorageMemorySummary::versioned_entry_or_delete_count.fetch_add(1); + if (entry) + PageStorageMemorySummary::versioned_entry_or_delete_bytes.fetch_add(sizeof(PageEntryV3)); + } + EntryOrDelete() { PageStorageMemorySummary::versioned_entry_or_delete_count.fetch_add(1); } + EntryOrDelete(std::optional entry_) + : entry(std::move(entry_)) + { + PageStorageMemorySummary::versioned_entry_or_delete_count.fetch_add(1); + if (entry) + PageStorageMemorySummary::versioned_entry_or_delete_bytes.fetch_add(sizeof(PageEntryV3)); + } + EntryOrDelete(MultiVersionRefCount being_ref_count_, std::optional entry_) + : being_ref_count(being_ref_count_) + , entry(std::move(entry_)) { - return EntryOrDelete{ - .entry = entry, - }; + PageStorageMemorySummary::versioned_entry_or_delete_count.fetch_add(1); + if (entry) + PageStorageMemorySummary::versioned_entry_or_delete_bytes.fetch_add(sizeof(PageEntryV3)); } + ~EntryOrDelete() + { + PageStorageMemorySummary::versioned_entry_or_delete_count.fetch_sub(1); + if (entry) + PageStorageMemorySummary::versioned_entry_or_delete_bytes.fetch_sub(sizeof(PageEntryV3)); + } + + static EntryOrDelete newDelete() { return EntryOrDelete(std::nullopt); }; + static EntryOrDelete newNormalEntry(const PageEntryV3 & entry) { return EntryOrDelete(entry); } static EntryOrDelete newReplacingEntry(const EntryOrDelete & ori_entry, const PageEntryV3 & entry) { - return EntryOrDelete{ - .being_ref_count = ori_entry.being_ref_count, - .entry = entry, - }; + return EntryOrDelete(ori_entry.being_ref_count, entry); } static EntryOrDelete newFromRestored(PageEntryV3 entry, const PageVersion & ver, Int64 being_ref_count) { - auto result = EntryOrDelete{ - .entry = entry, - }; + auto result = EntryOrDelete(std::move(entry)); result.being_ref_count.restoreFrom(ver, being_ref_count); return result; } diff --git a/dbms/src/Storages/Page/V3/PageDirectory/PageIdTrait.h b/dbms/src/Storages/Page/V3/PageDirectory/PageIdTrait.h index aec9717a15f..54251f0a795 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory/PageIdTrait.h +++ b/dbms/src/Storages/Page/V3/PageDirectory/PageIdTrait.h @@ -29,6 +29,7 @@ struct PageIdTrait static inline PageIdU64 getU64ID(const PageId & page_id) { return page_id.low; } static inline Prefix getPrefix(const PageId & page_id) { return page_id.high; } static inline PageIdU64 getPageMapKey(const PageId & page_id) { return page_id.low; } + static inline size_t getPageIDSize(const PageId & page_id) { return sizeof(page_id); } }; } // namespace u128 namespace universal @@ -45,6 +46,8 @@ struct PageIdTrait static Prefix getPrefix(const PageId & page_id); static inline PageId getPageMapKey(const PageId & page_id) { return page_id; } + + static inline size_t getPageIDSize(const PageId & page_id) { return page_id.size(); } }; } // namespace universal } // namespace DB::PS::V3 diff --git a/dbms/src/Storages/Page/V3/PageEntry.h b/dbms/src/Storages/Page/V3/PageEntry.h index c3fad616f70..c2c530f5203 100644 --- a/dbms/src/Storages/Page/V3/PageEntry.h +++ b/dbms/src/Storages/Page/V3/PageEntry.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageId.h b/dbms/src/Storages/Page/V3/Universal/UniversalPageId.h index 9e757732805..0d995dd9d01 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageId.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageId.h @@ -18,26 +18,49 @@ #include #include #include +#include namespace DB { class UniversalPageId final { public: - UniversalPageId() = default; + UniversalPageId() { PS::PageStorageMemorySummary::uni_page_id_bytes.fetch_add(id.size()); } + UniversalPageId(const UniversalPageId & other) + : id(other.id) + { + PS::PageStorageMemorySummary::uni_page_id_bytes.fetch_add(id.size()); + } UniversalPageId(String id_) // NOLINT(google-explicit-constructor) : id(std::move(id_)) - {} + { + PS::PageStorageMemorySummary::uni_page_id_bytes.fetch_add(id.size()); + } UniversalPageId(const char * id_) // NOLINT(google-explicit-constructor) : id(id_) - {} + { + PS::PageStorageMemorySummary::uni_page_id_bytes.fetch_add(id.size()); + } UniversalPageId(const char * id_, size_t sz_) : id(id_, sz_) - {} + { + PS::PageStorageMemorySummary::uni_page_id_bytes.fetch_add(id.size()); + } + + ~UniversalPageId() { PS::PageStorageMemorySummary::uni_page_id_bytes.fetch_sub(id.size()); } UniversalPageId & operator=(String && id_) noexcept { + if (id.size() == id_.size()) {} + else if (id.size() > id_.size()) + { + PS::PageStorageMemorySummary::uni_page_id_bytes.fetch_sub(id.size() - id_.size()); + } + else + { + PS::PageStorageMemorySummary::uni_page_id_bytes.fetch_add(id_.size() - id.size()); + } id.swap(id_); return *this; } diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 919b29d292c..067e2b6a752 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1717744619239, + "iteration": 1718272201438, "links": [], "panels": [ { @@ -8890,7 +8890,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Information about schema of column file, to learn the memory usage of schema", + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -8898,13 +8898,13 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, + "x": 0, "y": 102 }, "hiddenSeries": false, - "id": 168, + "id": 291, "legend": { "avg": false, "current": false, @@ -8916,6 +8916,7 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "alertThreshold": true @@ -8932,42 +8933,19 @@ "targets": [ { "exemplar": true, - "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"current_size\"}) by (instance)", - "interval": "", - "legendFormat": "current_size", - "queryType": "randomWalk", - "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"hit_count\"}[1m])) by (instance)", - "hide": false, + "expr": "tiflash_memory_usage_by_class{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", "interval": "", - "legendFormat": "hit_count_ops", + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", "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": "Memory by class", "tooltip": { "shared": true, "sort": 0, @@ -8983,7 +8961,7 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -9021,7 +8999,7 @@ "h": 7, "w": 12, "x": 12, - "y": 110 + "y": 102 }, "hiddenSeries": false, "id": 289, @@ -9111,6 +9089,126 @@ "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": 12, + "y": 109 + }, + "hiddenSeries": false, + "id": 168, + "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_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"current_size\"}) by (instance)", + "interval": "", + "legendFormat": "current_size", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"hit_count\"}[1m])) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "hit_count_ops", + "refId": "B" + }, + { + "exemplar": true, + "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"still_used_when_evict\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "still_used_when_evict", + "refId": "C" + }, + { + "exemplar": true, + "expr": "max(tiflash_shared_block_schemas{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"miss_count\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "miss_count", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Schema of Column File", + "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": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, @@ -9754,7 +9852,7 @@ "h": 8, "w": 12, "x": 0, - "y": 104 + "y": 8 }, "hiddenSeries": false, "id": 128, @@ -9897,7 +9995,7 @@ "h": 8, "w": 12, "x": 12, - "y": 104 + "y": 8 }, "hiddenSeries": false, "id": 129, @@ -10014,7 +10112,7 @@ "h": 8, "w": 12, "x": 0, - "y": 112 + "y": 16 }, "heatmap": {}, "hideZeroBuckets": true, @@ -10076,7 +10174,7 @@ "h": 8, "w": 12, "x": 12, - "y": 112 + "y": 16 }, "hiddenSeries": false, "id": 158, @@ -10212,7 +10310,7 @@ "h": 8, "w": 12, "x": 0, - "y": 120 + "y": 24 }, "hiddenSeries": false, "id": 163, @@ -10317,7 +10415,7 @@ "h": 8, "w": 12, "x": 12, - "y": 120 + "y": 24 }, "hiddenSeries": false, "id": 162, @@ -10437,7 +10535,7 @@ "h": 8, "w": 12, "x": 0, - "y": 128 + "y": 32 }, "hiddenSeries": false, "id": 164, @@ -10481,6 +10579,17 @@ "legendFormat": "num_pages-{{instance}}", "refId": "A", "step": 10 + }, + { + "exemplar": true, + "expr": "tiflash_system_asynchronous_metric_VersionedEntries{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "num_entries-{{instance}}", + "refId": "B", + "step": 10 } ], "thresholds": [], @@ -10546,7 +10655,7 @@ "h": 8, "w": 12, "x": 12, - "y": 128 + "y": 32 }, "hiddenSeries": false, "id": 123, @@ -10677,7 +10786,7 @@ "h": 8, "w": 12, "x": 0, - "y": 136 + "y": 40 }, "height": "", "hiddenSeries": false, @@ -10786,7 +10895,7 @@ "h": 8, "w": 12, "x": 12, - "y": 136 + "y": 40 }, "hiddenSeries": false, "id": 231, @@ -10892,7 +11001,7 @@ "h": 9, "w": 24, "x": 0, - "y": 144 + "y": 48 }, "hiddenSeries": false, "id": 232,