diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index 605e119f8a8..036f4c4f544 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -22,6 +22,7 @@ M(OpenFileForReadWrite) \ M(MemoryTracking) \ M(MemoryTrackingInBackgroundProcessingPool) \ + M(MemoryTrackingKVStore) \ M(LogicalCPUCores) \ M(MemoryCapacity) \ M(PSMVCCNumSnapshots) \ diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index 4d9cf4b283c..b5969e8b4c8 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -28,6 +28,7 @@ namespace CurrentMetrics extern const Metric MemoryTrackingQueryStorageTask; extern const Metric MemoryTrackingFetchPages; extern const Metric MemoryTrackingSharedColumnData; +extern const Metric MemoryTrackingKVStore; } // namespace CurrentMetrics std::atomic real_rss{0}, proc_num_threads{1}, baseline_of_query_mem_tracker{0}; @@ -76,6 +77,7 @@ static String storageMemoryUsageDetail() { return fmt::format( "non-query: peak={}, amount={}; " + "kvstore: peak={}, amount={}; " "query-storage-task: peak={}, amount={}; " "fetch-pages: peak={}, amount={}; " "shared-column-data: peak={}, amount={}.", @@ -83,6 +85,9 @@ static String storageMemoryUsageDetail() : "0", root_of_non_query_mem_trackers ? formatReadableSizeWithBinarySuffix(root_of_non_query_mem_trackers->get()) : "0", + root_of_kvstore_mem_trackers ? formatReadableSizeWithBinarySuffix(root_of_kvstore_mem_trackers->getPeak()) + : "0", + root_of_kvstore_mem_trackers ? formatReadableSizeWithBinarySuffix(root_of_kvstore_mem_trackers->get()) : "0", sub_root_of_query_storage_task_mem_trackers ? formatReadableSizeWithBinarySuffix(sub_root_of_query_storage_task_mem_trackers->getPeak()) : "0", @@ -293,6 +298,7 @@ thread_local MemoryTracker * current_memory_tracker = nullptr; std::shared_ptr root_of_non_query_mem_trackers = MemoryTracker::createGlobalRoot(); std::shared_ptr root_of_query_mem_trackers = MemoryTracker::createGlobalRoot(); +std::shared_ptr root_of_kvstore_mem_trackers = MemoryTracker::createGlobalRoot(); std::shared_ptr sub_root_of_query_storage_task_mem_trackers; std::shared_ptr fetch_pages_mem_tracker; @@ -323,6 +329,8 @@ void initStorageMemoryTracker(Int64 limit, Int64 larger_than_limit) shared_column_data_mem_tracker = MemoryTracker::create(0, sub_root_of_query_storage_task_mem_trackers.get(), log_in_destructor); shared_column_data_mem_tracker->setAmountMetric(CurrentMetrics::MemoryTrackingSharedColumnData); + + root_of_kvstore_mem_trackers->setAmountMetric(CurrentMetrics::MemoryTrackingKVStore); } namespace CurrentMemoryTracker diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index 62563abf0e5..8e8b171b91c 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -163,6 +163,7 @@ extern thread_local MemoryTracker * current_memory_tracker; extern std::shared_ptr root_of_non_query_mem_trackers; extern std::shared_ptr root_of_query_mem_trackers; +extern std::shared_ptr root_of_kvstore_mem_trackers; // Initialize in `initStorageMemoryTracker`. // If a memory tracker of storage tasks is driven by query, it should inherit `sub_root_of_query_storage_task_mem_trackers`. diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 4c7e12c2e04..bb3eb9746f8 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -34,6 +34,9 @@ namespace DB { +constexpr size_t RAFT_REGION_BIG_WRITE_THRES = 2 * 1024; +constexpr size_t RAFT_REGION_BIG_WRITE_MAX = 4 * 1024 * 1024; // raft-entry-max-size = 8MiB +static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Invalid RAFT_REGION_BIG_WRITE_THRES"); /// Central place to define metrics across all subsystems. /// Refer to gtest_tiflash_metrics.cpp for more sample defines. /// Usage: @@ -485,12 +488,13 @@ namespace DB M(tiflash_raft_raft_frequent_events_count, \ "Raft frequent event counter", \ Counter, \ + F(type_write_commit, {{"type", "write_commit"}}), \ F(type_write, {{"type", "write"}})) \ M(tiflash_raft_region_flush_bytes, \ "Bucketed histogram of region flushed bytes", \ Histogram, \ - F(type_flushed, {{"type", "flushed"}}, ExpBuckets{32, 2, 21}), \ - F(type_unflushed, {{"type", "unflushed"}}, ExpBuckets{32, 2, 21})) \ + F(type_flushed, {{"type", "flushed"}}, ExpBucketsWithRange{32, 4, 32 * 1024 * 1024}), \ + F(type_unflushed, {{"type", "unflushed"}}, ExpBucketsWithRange{32, 4, 32 * 1024 * 1024})) \ M(tiflash_raft_entry_size, \ "Bucketed histogram entry size", \ Histogram, \ @@ -501,6 +505,20 @@ namespace DB F(type_raft_snapshot, {{"type", "raft_snapshot"}}), \ F(type_dt_on_disk, {{"type", "dt_on_disk"}}), \ F(type_dt_total, {{"type", "dt_total"}})) \ + M(tiflash_raft_throughput_bytes, \ + "Raft handled bytes in global", \ + Counter, \ + F(type_write, {{"type", "write"}}), \ + F(type_write_committed, {{"type", "write_committed"}})) \ + M(tiflash_raft_write_flow_bytes, \ + "Bucketed histogram of bytes for each write", \ + Histogram, \ + F(type_ingest_uncommitted, {{"type", "ingest_uncommitted"}}, ExpBucketsWithRange{16, 4, 64 * 1024}), \ + F(type_snapshot_uncommitted, {{"type", "snapshot_uncommitted"}}, ExpBucketsWithRange{16, 4, 1024 * 1024}), \ + F(type_write_committed, {{"type", "write_committed"}}, ExpBucketsWithRange{16, 2, 1024 * 1024}), \ + F(type_big_write_to_region, \ + {{"type", "big_write_to_region"}}, \ + ExpBucketsWithRange{RAFT_REGION_BIG_WRITE_THRES, 4, RAFT_REGION_BIG_WRITE_MAX})) \ M(tiflash_raft_snapshot_total_bytes, \ "Bucketed snapshot total size", \ Histogram, \ diff --git a/dbms/src/Storages/KVStore/Decode/DecodedTiKVKeyValue.cpp b/dbms/src/Storages/KVStore/Decode/DecodedTiKVKeyValue.cpp index 63ce48a0345..08b106673fb 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodedTiKVKeyValue.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodedTiKVKeyValue.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { @@ -37,4 +38,10 @@ std::string DecodedTiKVKey::makeKeyspacePrefix(KeyspaceID keyspace_id) { return TiKVKeyspaceID::makeKeyspacePrefix(keyspace_id); } + +HandleID RawTiDBPK::getHandleID() const +{ + const auto & pk = *this; + return RecordKVFormat::decodeInt64(RecordKVFormat::read(pk->data())); +} } // namespace DB diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 7f99a71d76a..f32a4ddb5b1 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -388,9 +388,16 @@ DM::WriteResult RegionTable::writeBlockByRegion( reportUpstreamLatency(*data_list_read); auto write_result = writeRegionDataToStorage(context, region, *data_list_read, log); - + auto prev_region_size = region->dataSize(); RemoveRegionCommitCache(region, *data_list_read, lock_region); - + auto new_region_size = region->dataSize(); + if likely (new_region_size <= prev_region_size) + { + auto committed_bytes = prev_region_size - new_region_size; + GET_METRIC(tiflash_raft_write_flow_bytes, type_write_committed).Observe(committed_bytes); + GET_METRIC(tiflash_raft_throughput_bytes, type_write_committed).Increment(committed_bytes); + GET_METRIC(tiflash_raft_raft_frequent_events_count, type_write_commit).Increment(1); + } /// Save removed data to outer. data_list_to_remove = std::move(*data_list_read); return write_result; diff --git a/dbms/src/Storages/KVStore/KVStore.cpp b/dbms/src/Storages/KVStore/KVStore.cpp index 3d4f9f7f34d..0d38b823099 100644 --- a/dbms/src/Storages/KVStore/KVStore.cpp +++ b/dbms/src/Storages/KVStore/KVStore.cpp @@ -660,6 +660,7 @@ void KVStore::StoreMeta::update(Base && base_) KVStore::~KVStore() { + LOG_INFO(log, "Destroy KVStore"); releaseReadIndexWorkers(); } diff --git a/dbms/src/Storages/KVStore/MultiRaft/ApplySnapshot.cpp b/dbms/src/Storages/KVStore/MultiRaft/ApplySnapshot.cpp index 844a1d7a553..86c5d5e6ead 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/ApplySnapshot.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/ApplySnapshot.cpp @@ -302,6 +302,7 @@ void KVStore::onSnapshot( manage_lock.index.add(new_region); } + GET_METRIC(tiflash_raft_write_flow_bytes, type_snapshot_uncommitted).Observe(new_region->dataSize()); persistRegion(*new_region, ®ion_lock, PersistRegionReason::ApplySnapshotCurRegion, ""); tmt.getRegionTable().shrinkRegionRange(*new_region); diff --git a/dbms/src/Storages/KVStore/MultiRaft/IngestSST.cpp b/dbms/src/Storages/KVStore/MultiRaft/IngestSST.cpp index 61a3fb68115..90913bc5bf0 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/IngestSST.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/IngestSST.cpp @@ -180,6 +180,8 @@ void Region::finishIngestSSTByDTFile(RegionPtr && temp_region, UInt64 index, UIn { std::unique_lock lock(mutex); + auto uncommitted_ingest = temp_region->dataSize(); + GET_METRIC(tiflash_raft_write_flow_bytes, type_ingest_uncommitted).Observe(uncommitted_ingest); if (temp_region) { // Merge the uncommitted data from `temp_region`. diff --git a/dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp b/dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp index 34a1b5072bb..73dbaeb6ef7 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp @@ -314,6 +314,10 @@ std::pair Region::handleWriteRaftCmd( size_t put_key_count = 0; size_t del_key_count = 0; + // How many bytes has been written to KVStore(and maybe then been moved to underlying DeltaTree). + // We don't count DEL because it is only used to delete LOCK, which is small and not count in doInsert. + size_t write_size = 0; + size_t prev_size = dataSize(); SCOPE_EXIT({ GET_METRIC(tiflash_raft_apply_write_command_duration_seconds, type_write).Observe(watch.elapsedSeconds()); @@ -322,6 +326,10 @@ std::pair Region::handleWriteRaftCmd( GET_METRIC(tiflash_raft_raft_frequent_events_count, type_write).Increment(1); GET_METRIC(tiflash_raft_process_keys, type_write_put).Increment(put_key_count); GET_METRIC(tiflash_raft_process_keys, type_write_del).Increment(del_key_count); + auto after_size = dataSize(); + if (after_size > prev_size + RAFT_REGION_BIG_WRITE_THRES) + GET_METRIC(tiflash_raft_write_flow_bytes, type_big_write_to_region).Observe(after_size - prev_size); + GET_METRIC(tiflash_raft_throughput_bytes, type_write).Increment(write_size); }); if (cmds.len) @@ -349,11 +357,11 @@ std::pair Region::handleWriteRaftCmd( if (is_v2) { // There may be orphan default key in a snapshot. - doInsert(cf, std::move(tikv_key), std::move(tikv_value), DupCheck::AllowSame); + write_size += doInsert(cf, std::move(tikv_key), std::move(tikv_value), DupCheck::AllowSame); } else { - doInsert(cf, std::move(tikv_key), std::move(tikv_value), DupCheck::Deny); + write_size += doInsert(cf, std::move(tikv_key), std::move(tikv_value), DupCheck::Deny); } } catch (Exception & e) diff --git a/dbms/src/Storages/KVStore/MultiRaft/RegionCFDataBase.cpp b/dbms/src/Storages/KVStore/MultiRaft/RegionCFDataBase.cpp index 9bf07b060b9..b648bab5e54 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/RegionCFDataBase.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/RegionCFDataBase.cpp @@ -44,7 +44,6 @@ RegionDataRes RegionCFDataBase::insert(TiKVKey && key, TiKVValue && value { const auto & raw_key = RecordKVFormat::decodeTiKVKey(key); auto kv_pair = Trait::genKVPair(std::move(key), raw_key, std::move(value)); - if (!kv_pair) return 0; @@ -58,6 +57,7 @@ RegionDataRes RegionCFDataBase::insert(TiKVKey && key, Ti Pair kv_pair = RegionLockCFDataTrait::genKVPair(std::move(key), std::move(value)); // according to the process of pessimistic lock, just overwrite. data.insert_or_assign(std::move(kv_pair.first), std::move(kv_pair.second)); + // lock cf is not count into the size of RegionData, always return 0 return 0; } @@ -94,6 +94,8 @@ RegionDataRes RegionCFDataBase::insert(std::pair && kv_pair, + " new_val: " + prev_value.toDebugString(), ErrorCodes::LOGICAL_ERROR); } + // duplicated key is ignored + return 0; } else { @@ -122,7 +124,9 @@ size_t RegionCFDataBase::calcTiKVKeyValueSize(const TiKVKey & key, const return 0; } else + { return key.dataSize() + value.dataSize(); + } } diff --git a/dbms/src/Storages/KVStore/MultiRaft/RegionCFDataBase.h b/dbms/src/Storages/KVStore/MultiRaft/RegionCFDataBase.h index 66c841e2827..308842f4fe9 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/RegionCFDataBase.h +++ b/dbms/src/Storages/KVStore/MultiRaft/RegionCFDataBase.h @@ -60,7 +60,7 @@ struct RegionCFDataBase size_t getSize() const; - RegionCFDataBase() {} + RegionCFDataBase() = default; RegionCFDataBase(RegionCFDataBase && region); RegionCFDataBase & operator=(RegionCFDataBase && region); diff --git a/dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp b/dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp index d7e84e6d70c..a63d1ec1733 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp @@ -26,30 +26,51 @@ extern const int LOGICAL_ERROR; extern const int ILLFORMAT_RAFT_ROW; } // namespace ErrorCodes -HandleID RawTiDBPK::getHandleID() const +void RegionData::reportAlloc(size_t delta) { - const auto & pk = *this; - return RecordKVFormat::decodeInt64(RecordKVFormat::read(pk->data())); + root_of_kvstore_mem_trackers->alloc(delta, false); } -void RegionData::insert(ColumnFamilyType cf, TiKVKey && key, TiKVValue && value, DupCheck mode) +void RegionData::reportDealloc(size_t delta) +{ + root_of_kvstore_mem_trackers->free(delta); +} + +void RegionData::reportDelta(size_t prev, size_t current) +{ + if (current >= prev) + { + root_of_kvstore_mem_trackers->alloc(current - prev, false); + } + else + { + root_of_kvstore_mem_trackers->free(prev - current); + } +} + +size_t RegionData::insert(ColumnFamilyType cf, TiKVKey && key, TiKVValue && value, DupCheck mode) { switch (cf) { case ColumnFamilyType::Write: { - cf_data_size += write_cf.insert(std::move(key), std::move(value), mode); - return; + auto delta = write_cf.insert(std::move(key), std::move(value), mode); + cf_data_size += delta; + reportAlloc(delta); + return delta; } case ColumnFamilyType::Default: { - cf_data_size += default_cf.insert(std::move(key), std::move(value), mode); - return; + auto delta = default_cf.insert(std::move(key), std::move(value), mode); + cf_data_size += delta; + reportAlloc(delta); + return delta; } case ColumnFamilyType::Lock: { + // lock cf is not count into the size of RegionData lock_cf.insert(std::move(key), std::move(value), mode); - return; + return 0; } } } @@ -64,7 +85,9 @@ void RegionData::remove(ColumnFamilyType cf, const TiKVKey & key) auto pk = RecordKVFormat::getRawTiDBPK(raw_key); Timestamp ts = RecordKVFormat::getTs(key); // removed by gc, may not exist. - cf_data_size -= write_cf.remove(RegionWriteCFData::Key{pk, ts}, true); + auto delta = write_cf.remove(RegionWriteCFData::Key{pk, ts}, true); + cf_data_size -= delta; + reportDealloc(delta); return; } case ColumnFamilyType::Default: @@ -73,7 +96,9 @@ void RegionData::remove(ColumnFamilyType cf, const TiKVKey & key) auto pk = RecordKVFormat::getRawTiDBPK(raw_key); Timestamp ts = RecordKVFormat::getTs(key); // removed by gc, may not exist. - cf_data_size -= default_cf.remove(RegionDefaultCFData::Key{pk, ts}, true); + auto delta = default_cf.remove(RegionDefaultCFData::Key{pk, ts}, true); + cf_data_size -= delta; + reportDealloc(delta); return; } case ColumnFamilyType::Lock: @@ -99,12 +124,16 @@ RegionData::WriteCFIter RegionData::removeDataByWriteIt(const WriteCFIter & writ if (auto data_it = map.find({pk, decoded_val.prewrite_ts}); data_it != map.end()) { - cf_data_size -= RegionDefaultCFData::calcTiKVKeyValueSize(data_it->second); + auto delta = RegionDefaultCFData::calcTiKVKeyValueSize(data_it->second); + cf_data_size -= delta; map.erase(data_it); + reportDealloc(delta); } } - cf_data_size -= RegionWriteCFData::calcTiKVKeyValueSize(write_it->second); + auto delta = RegionWriteCFData::calcTiKVKeyValueSize(write_it->second); + cf_data_size -= delta; + reportDealloc(delta); return write_cf.getDataMut().erase(write_it); } @@ -238,6 +267,7 @@ void RegionData::splitInto(const RegionRange & range, RegionData & new_region_da size_t size_changed = 0; size_changed += default_cf.splitInto(range, new_region_data.default_cf); size_changed += write_cf.splitInto(range, new_region_data.write_cf); + // reportAlloc: Remember to track memory here if we have a region-wise metrics later. size_changed += lock_cf.splitInto(range, new_region_data.lock_cf); cf_data_size -= size_changed; new_region_data.cf_data_size += size_changed; @@ -248,6 +278,7 @@ void RegionData::mergeFrom(const RegionData & ori_region_data) size_t size_changed = 0; size_changed += default_cf.mergeFrom(ori_region_data.default_cf); size_changed += write_cf.mergeFrom(ori_region_data.write_cf); + // reportAlloc: Remember to track memory here if we have a region-wise metrics later. size_changed += lock_cf.mergeFrom(ori_region_data.lock_cf); cf_data_size += size_changed; } @@ -328,6 +359,7 @@ RegionData & RegionData::operator=(RegionData && rhs) write_cf = std::move(rhs.write_cf); default_cf = std::move(rhs.default_cf); lock_cf = std::move(rhs.lock_cf); + reportDelta(cf_data_size, rhs.cf_data_size.load()); cf_data_size = rhs.cf_data_size.load(); return *this; } @@ -360,7 +392,6 @@ uint64_t RegionData::OrphanKeysInfo::remainedKeyCount() const return remained_keys.size(); } - void RegionData::OrphanKeysInfo::mergeFrom(const RegionData::OrphanKeysInfo & other) { // TODO support move. diff --git a/dbms/src/Storages/KVStore/MultiRaft/RegionData.h b/dbms/src/Storages/KVStore/MultiRaft/RegionData.h index b59925fca58..fb6a0918948 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/RegionData.h +++ b/dbms/src/Storages/KVStore/MultiRaft/RegionData.h @@ -37,7 +37,11 @@ class RegionData using WriteCFIter = RegionWriteCFData::Map::iterator; using ConstWriteCFIter = RegionWriteCFData::Map::const_iterator; - void insert(ColumnFamilyType cf, TiKVKey && key, TiKVValue && value, DupCheck mode = DupCheck::Deny); + static void reportAlloc(size_t delta); + static void reportDealloc(size_t delta); + static void reportDelta(size_t prev, size_t current); + + size_t insert(ColumnFamilyType cf, TiKVKey && key, TiKVValue && value, DupCheck mode = DupCheck::Deny); void remove(ColumnFamilyType cf, const TiKVKey & key); WriteCFIter removeDataByWriteIt(const WriteCFIter & write_it); diff --git a/dbms/src/Storages/KVStore/Region.cpp b/dbms/src/Storages/KVStore/Region.cpp index 17a3a8f7475..16167c94460 100644 --- a/dbms/src/Storages/KVStore/Region.cpp +++ b/dbms/src/Storages/KVStore/Region.cpp @@ -76,16 +76,16 @@ DecodedLockCFValuePtr Region::getLockInfo(const RegionLockReadQuery & query) con void Region::insert(const std::string & cf, TiKVKey && key, TiKVValue && value, DupCheck mode) { - return insert(NameToCF(cf), std::move(key), std::move(value), mode); + insert(NameToCF(cf), std::move(key), std::move(value), mode); } void Region::insert(ColumnFamilyType type, TiKVKey && key, TiKVValue && value, DupCheck mode) { std::unique_lock lock(mutex); - return doInsert(type, std::move(key), std::move(value), mode); + doInsert(type, std::move(key), std::move(value), mode); } -void Region::doInsert(ColumnFamilyType type, TiKVKey && key, TiKVValue && value, DupCheck mode) +size_t Region::doInsert(ColumnFamilyType type, TiKVKey && key, TiKVValue && value, DupCheck mode) { if (getClusterRaftstoreVer() == RaftstoreVer::V2) { @@ -95,11 +95,12 @@ void Region::doInsert(ColumnFamilyType type, TiKVKey && key, TiKVValue && value, { // We can't assert the key exists in write_cf here, // since it may be already written into DeltaTree. - return; + return 0; } } } - data.insert(type, std::move(key), std::move(value), mode); + auto ans = data.insert(type, std::move(key), std::move(value), mode); + return ans; } void Region::remove(const std::string & cf, const TiKVKey & key) @@ -199,8 +200,8 @@ RegionPtr Region::deserialize(ReadBuffer & buf, const TiFlashRaftProxyHelper * p throw Exception( ErrorCodes::UNKNOWN_FORMAT_VERSION, "{}: unexpected version: {}, expected: {}", - binary_version, __PRETTY_FUNCTION__, + binary_version, CURRENT_VERSION); } @@ -219,6 +220,7 @@ RegionPtr Region::deserialize(ReadBuffer & buf, const TiFlashRaftProxyHelper * p // deserialize data RegionData::deserialize(buf, region->data); + region->data.reportAlloc(region->data.cf_data_size); // restore other var according to meta region->last_restart_log_applied = region->appliedIndex(); @@ -442,6 +444,11 @@ Region::Region(DB::RegionMeta && meta_, const TiFlashRaftProxyHelper * proxy_hel , proxy_helper(proxy_helper_) {} +Region::~Region() +{ + data.reportDealloc(data.cf_data_size); +} + TableID Region::getMappedTableID() const { return mapped_table_id; diff --git a/dbms/src/Storages/KVStore/Region.h b/dbms/src/Storages/KVStore/Region.h index a2370c7180a..586732d6ac2 100644 --- a/dbms/src/Storages/KVStore/Region.h +++ b/dbms/src/Storages/KVStore/Region.h @@ -35,6 +35,7 @@ namespace tests { class KVStoreTestBase; class RegionKVStoreOldTest; +class RegionKVStoreTest; } // namespace tests class Region; @@ -119,6 +120,7 @@ class Region : public std::enable_shared_from_this public: explicit Region(RegionMeta && meta_); explicit Region(RegionMeta && meta_, const TiFlashRaftProxyHelper *); + ~Region(); void insert(const std::string & cf, TiKVKey && key, TiKVValue && value, DupCheck mode = DupCheck::Deny); void insert(ColumnFamilyType type, TiKVKey && key, TiKVValue && value, DupCheck mode = DupCheck::Deny); @@ -169,7 +171,6 @@ class Region : public std::enable_shared_from_this // Check if we can read by this index. bool checkIndex(UInt64 index) const; - // Return for wait-index. std::tuple waitIndex( UInt64 index, @@ -181,7 +182,6 @@ class Region : public std::enable_shared_from_this UInt64 appliedIndex() const; // Requires RegionMeta's lock UInt64 appliedIndexTerm() const; - void notifyApplied() { meta.notifyAll(); } // Export for tests. void setApplied(UInt64 index, UInt64 term); @@ -240,10 +240,11 @@ class Region : public std::enable_shared_from_this friend class RegionMockTest; friend class tests::KVStoreTestBase; friend class tests::RegionKVStoreOldTest; + friend class tests::RegionKVStoreTest; // Private methods no need to lock mutex, normally - void doInsert(ColumnFamilyType type, TiKVKey && key, TiKVValue && value, DupCheck mode); + size_t doInsert(ColumnFamilyType type, TiKVKey && key, TiKVValue && value, DupCheck mode); void doCheckTable(const DecodedTiKVKey & key) const; void doRemove(ColumnFamilyType type, const TiKVKey & key); diff --git a/dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp index af4d6584ab2..f98791ec0d3 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp @@ -12,18 +12,114 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include +#include #include #include #include "region_kvstore_test.h" +extern std::shared_ptr root_of_kvstore_mem_trackers; + namespace DB { namespace tests { +TEST_F(RegionKVStoreTest, MemoryTracker) +try +{ + auto & ctx = TiFlashTestEnv::getGlobalContext(); + initStorages(); + KVStore & kvs = getKVS(); + auto table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); + auto start = RecordKVFormat::genKey(table_id, 0); + auto end = RecordKVFormat::genKey(table_id, 100); + auto str_key = RecordKVFormat::genKey(table_id, 1, 111); + auto [str_val_write, str_val_default] = proxy_instance->generateTiKVKeyValue(111, 999); + MockRaftStoreProxy::FailCond cond; + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {1, 2}, + {{RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, 10)}, + {RecordKVFormat::genKey(table_id, 11), RecordKVFormat::genKey(table_id, 20)}}); + + { + auto region_id = 1; + auto kvr1 = kvs.getRegion(region_id); + auto [index, term] + = proxy_instance + ->rawWrite(region_id, {str_key}, {str_val_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + UNUSED(term); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), str_key.dataSize() + str_val_default.size()); + } + + { + root_of_kvstore_mem_trackers->reset(); + RegionPtr region = tests::makeRegion(700, start, end, proxy_helper.get()); + region->insert("default", TiKVKey::copyFrom(str_key), TiKVValue::copyFrom(str_val_default)); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), str_key.dataSize() + str_val_default.size()); + region->remove("default", TiKVKey::copyFrom(str_key)); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0); + } + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0); + { + root_of_kvstore_mem_trackers->reset(); + RegionPtr region = tests::makeRegion(701, start, end, proxy_helper.get()); + region->insert("default", TiKVKey::copyFrom(str_key), TiKVValue::copyFrom(str_val_default)); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), str_key.dataSize() + str_val_default.size()); + } + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0); + { + root_of_kvstore_mem_trackers->reset(); + RegionPtr region = tests::makeRegion(702, start, end, proxy_helper.get()); + region->insert("default", TiKVKey::copyFrom(str_key), TiKVValue::copyFrom(str_val_default)); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), str_key.dataSize() + str_val_default.size()); + tryPersistRegion(kvs, 1); + reloadKVSFromDisk(); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), str_key.dataSize() + str_val_default.size()); + } + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0); + { + root_of_kvstore_mem_trackers->reset(); + RegionPtr region = tests::makeRegion(800, start, end, proxy_helper.get()); + region->insert("default", TiKVKey::copyFrom(str_key), TiKVValue::copyFrom(str_val_default)); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), str_key.dataSize() + str_val_default.size()); + region->insert("write", TiKVKey::copyFrom(str_key), TiKVValue::copyFrom(str_val_write)); + std::optional data_list_read = ReadRegionCommitCache(region, true); + ASSERT_TRUE(data_list_read); + ASSERT_EQ(1, data_list_read->size()); + RemoveRegionCommitCache(region, *data_list_read); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0); + } + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0); + { + root_of_kvstore_mem_trackers->reset(); + RegionPtr region = tests::makeRegion(900, start, end, proxy_helper.get()); + region->insert("default", TiKVKey::copyFrom(str_key), TiKVValue::copyFrom(str_val_default)); + auto str_key2 = RecordKVFormat::genKey(table_id, 20, 111); + auto [str_val_write2, str_val_default2] = proxy_instance->generateTiKVKeyValue(111, 999); + region->insert("default", TiKVKey::copyFrom(str_key2), TiKVValue::copyFrom(str_val_default2)); + auto expected = str_key.dataSize() + str_val_default.size() + str_key2.dataSize() + str_val_default2.size(); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), expected); + auto new_region = splitRegion( + region, + RegionMeta( + createPeer(901, true), + createRegionInfo(902, RecordKVFormat::genKey(table_id, 50), end), + initialApplyState())); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), expected); + region->mergeDataFrom(*new_region); + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), expected); + } + ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0); +} +CATCH + TEST_F(RegionKVStoreTest, KVStoreFailRecovery) try { diff --git a/dbms/src/Storages/KVStore/tests/region_kvstore_test.h b/dbms/src/Storages/KVStore/tests/region_kvstore_test.h index 42c8e68089d..4c0006b32ff 100644 --- a/dbms/src/Storages/KVStore/tests/region_kvstore_test.h +++ b/dbms/src/Storages/KVStore/tests/region_kvstore_test.h @@ -31,9 +31,14 @@ class RegionKVStoreTest : public KVStoreTestBase log = DB::Logger::get("RegionKVStoreTest"); test_path = TiFlashTestEnv::getTemporaryPath("/region_kvs_test"); } + + static RegionPtr splitRegion(const RegionPtr & region, RegionMeta && meta) + { + return region->splitInto(std::move(meta)); + } }; -static inline void validateSSTGeneration( +inline void validateSSTGeneration( KVStore & kvs, std::unique_ptr & proxy_instance, UInt64 region_id, diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 3de4fe697ed..b2209167cfb 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1701876187859, + "iteration": 1703085129908, "links": [], "panels": [ { @@ -8591,7 +8591,7 @@ "h": 8, "w": 24, "x": 0, - "y": 7 + "y": 103 }, "hiddenSeries": false, "id": 62, @@ -8710,7 +8710,7 @@ "h": 8, "w": 12, "x": 0, - "y": 15 + "y": 111 }, "height": "", "hiddenSeries": false, @@ -8829,7 +8829,7 @@ "h": 8, "w": 12, "x": 12, - "y": 15 + "y": 111 }, "height": "", "hiddenSeries": false, @@ -8946,7 +8946,7 @@ "h": 9, "w": 24, "x": 0, - "y": 23 + "y": 119 }, "height": "", "hiddenSeries": false, @@ -9068,7 +9068,7 @@ "h": 9, "w": 24, "x": 0, - "y": 32 + "y": 128 }, "hiddenSeries": false, "id": 90, @@ -9200,7 +9200,7 @@ "h": 8, "w": 12, "x": 0, - "y": 8 + "y": 104 }, "hiddenSeries": false, "id": 128, @@ -9343,7 +9343,7 @@ "h": 8, "w": 12, "x": 12, - "y": 8 + "y": 104 }, "hiddenSeries": false, "id": 129, @@ -9460,7 +9460,7 @@ "h": 8, "w": 12, "x": 0, - "y": 16 + "y": 112 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9522,7 +9522,7 @@ "h": 8, "w": 12, "x": 12, - "y": 16 + "y": 112 }, "hiddenSeries": false, "id": 158, @@ -9658,7 +9658,7 @@ "h": 8, "w": 12, "x": 0, - "y": 24 + "y": 120 }, "hiddenSeries": false, "id": 163, @@ -9763,7 +9763,7 @@ "h": 8, "w": 12, "x": 12, - "y": 24 + "y": 120 }, "hiddenSeries": false, "id": 162, @@ -9883,7 +9883,7 @@ "h": 8, "w": 12, "x": 0, - "y": 32 + "y": 128 }, "hiddenSeries": false, "id": 164, @@ -9992,7 +9992,7 @@ "h": 8, "w": 12, "x": 12, - "y": 32 + "y": 128 }, "hiddenSeries": false, "id": 123, @@ -10123,7 +10123,7 @@ "h": 8, "w": 12, "x": 0, - "y": 40 + "y": 136 }, "height": "", "hiddenSeries": false, @@ -10232,7 +10232,7 @@ "h": 8, "w": 12, "x": 12, - "y": 40 + "y": 136 }, "hiddenSeries": false, "id": 231, @@ -10338,7 +10338,7 @@ "h": 9, "w": 24, "x": 0, - "y": 48 + "y": 144 }, "hiddenSeries": false, "id": 232, @@ -11849,7 +11849,7 @@ ], "timeFrom": null, "timeShift": null, - "title": "Region Size Heatmap", + "title": "Region Size (by event) Heatmap", "tooltip": { "show": true, "showHistogram": true @@ -11972,6 +11972,298 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 72 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 261, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_write_flow_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write_committed\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Write Committed Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 72 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 262, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_write_flow_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"snapshot_uncommitted\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot Uncommitted Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 79 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 265, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_write_flow_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest_uncommitted\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Ingest Uncommitted Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 79 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 264, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_write_flow_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"big_write_to_region\"}[1m])) by (le, type)", + "format": "heatmap", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Big Write To Region Size Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "bytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, @@ -11988,7 +12280,7 @@ "h": 7, "w": 12, "x": 0, - "y": 72 + "y": 86 }, "hiddenSeries": false, "id": 240, @@ -12092,7 +12384,7 @@ "h": 7, "w": 12, "x": 12, - "y": 72 + "y": 86 }, "hiddenSeries": false, "id": 239, @@ -12229,7 +12521,7 @@ "h": 7, "w": 24, "x": 0, - "y": 79 + "y": 93 }, "hiddenSeries": false, "id": 75, @@ -12353,7 +12645,7 @@ "h": 7, "w": 12, "x": 0, - "y": 86 + "y": 100 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12423,7 +12715,7 @@ "h": 7, "w": 12, "x": 12, - "y": 86 + "y": 100 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12469,6 +12761,106 @@ "yBucketNumber": null, "yBucketSize": null }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 107 + }, + "hiddenSeries": false, + "id": 263, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_MemoryTrackingKVStore{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "KVStore memory", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:1524", + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:1525", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "cards": { "cardPadding": null, @@ -12493,7 +12885,7 @@ "h": 7, "w": 12, "x": 12, - "y": 93 + "y": 107 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12559,7 +12951,7 @@ "h": 7, "w": 24, "x": 0, - "y": 100 + "y": 114 }, "height": "", "hiddenSeries": false, @@ -12649,6 +13041,118 @@ "alignLevel": null } }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, + "y": 121 + }, + "height": "", + "hiddenSeries": false, + "id": 260, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatedByRow": true, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft throughput", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:164", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:165", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "cards": { "cardPadding": null, @@ -12673,7 +13177,7 @@ "h": 7, "w": 12, "x": 0, - "y": 107 + "y": 128 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12742,7 +13246,7 @@ "h": 7, "w": 12, "x": 12, - "y": 107 + "y": 128 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12812,7 +13316,7 @@ "h": 7, "w": 12, "x": 0, - "y": 114 + "y": 135 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12878,7 +13382,7 @@ "h": 7, "w": 12, "x": 12, - "y": 114 + "y": 135 }, "hiddenSeries": false, "id": 91, @@ -14710,6 +15214,7 @@ }, "yaxes": [ { + "$$hashKey": "object:1167", "decimals": null, "format": "bytes", "label": null, @@ -14719,6 +15224,7 @@ "show": true }, { + "$$hashKey": "object:1168", "format": "percentunit", "label": null, "logBase": 1, @@ -17253,7 +17759,7 @@ "h": 8, "w": 12, "x": 0, - "y": 14 + "y": 78 }, "hiddenSeries": false, "id": 246, @@ -17502,4 +18008,4 @@ "title": "Test-Cluster-TiFlash-Summary", "uid": "SVbh2xUWk", "version": 1 -} +} \ No newline at end of file