From dbfec4a525feabdc6ec05ea1a715ad5b7ebba8ff Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 1 Mar 2023 10:54:07 +0800 Subject: [PATCH 01/66] background delta flush Signed-off-by: hehechen --- contrib/tiflash-proxy | 2 +- dbms/src/Common/TiFlashMetrics.h | 10 ++- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 8 ++ .../src/Storages/DeltaMerge/DeltaMergeStore.h | 3 +- .../DeltaMerge/DeltaMergeStore_InternalBg.cpp | 31 ++++++- dbms/src/Storages/DeltaMerge/RowKeyRange.h | 5 ++ dbms/src/Storages/DeltaMerge/Segment.cpp | 4 + dbms/src/Storages/StorageDeltaMerge.cpp | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 80 ++++++++++++++++++- dbms/src/Storages/Transaction/KVStore.h | 3 + dbms/src/Storages/Transaction/ProxyFFI.cpp | 13 ++- dbms/src/Storages/Transaction/ProxyFFI.h | 3 + dbms/src/Storages/Transaction/Region.h | 12 +++ dbms/src/Storages/Transaction/RegionTable.h | 1 + 14 files changed, 168 insertions(+), 9 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 2d3b2b033cd..75cc2d64469 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 2d3b2b033cd8b2e06894ed14a6334b248f324a3d +Subproject commit 75cc2d6446940f27c0b62fe9e77c290bd7e75bac diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 6d8ea8fd81c..5a406e62a84 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -134,7 +134,11 @@ namespace DB F(type_seg_split_fg, {"type", "seg_split_fg"}), \ F(type_seg_split_ingest, {"type", "seg_split_ingest"}), \ F(type_seg_merge_bg_gc, {"type", "seg_merge_bg_gc"}), \ - F(type_place_index_update, {"type", "place_index_update"})) \ + F(type_place_index_update, {"type", "place_index_update"}), \ + F(type_compact_log_segment_bg, {"type", "compact_log_segment_bg"}), \ + F(type_compact_log_segment_fg, {"type", "compact_log_segment_fg"}), \ + F(type_compact_log_region_bg, {"type", "compact_log_region_bg"}), \ + F(type_compact_log_region_fg, {"type", "compact_log_region_fg"})) \ M(tiflash_storage_subtask_duration_seconds, "Bucketed histogram of storage's sub task duration", Histogram, \ F(type_delta_merge_bg, {{"type", "delta_merge_bg"}}, ExpBuckets{0.001, 2, 20}), \ F(type_delta_merge_bg_gc, {{"type", "delta_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ @@ -146,7 +150,9 @@ namespace DB F(type_seg_split_fg, {{"type", "seg_split_fg"}}, ExpBuckets{0.001, 2, 20}), \ F(type_seg_split_ingest, {{"type", "seg_split_ingest"}}, ExpBuckets{0.001, 2, 20}), \ F(type_seg_merge_bg_gc, {{"type", "seg_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20})) \ + F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_compact_log_bg, {{"type", "compact_log_bg"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_compact_log_fg, {{"type", "compact_log_fg"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_storage_throughput_bytes, "Calculate the throughput of tasks of storage in bytes", Gauge, /**/ \ F(type_write, {"type", "write"}), /**/ \ F(type_ingest, {"type", "ingest"}), /**/ \ diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 5f07d6c7fd4..a10bacf2809 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include @@ -116,6 +117,7 @@ std::pair DeltaMergeStore::MergeDeltaTaskPool::tryAddTask(const Back case TaskType::Compact: case TaskType::Flush: case TaskType::PlaceIndex: + case TaskType::NotifyCompactLog: is_heavy = false; // reserve some task space for heavy tasks if (max_task_num > 1 && light_tasks.size() >= static_cast(max_task_num * 0.9)) @@ -1173,6 +1175,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto delta_cache_limit_rows = dm_context->delta_cache_limit_rows; auto delta_cache_limit_bytes = dm_context->delta_cache_limit_bytes; + bool should_background_compact_log = (unsaved_rows >= delta_cache_limit_rows / 4 || unsaved_bytes >= delta_cache_limit_bytes / 4); bool should_background_flush = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes) // && (delta_rows - delta_last_try_flush_rows >= delta_cache_limit_rows || delta_bytes - delta_last_try_flush_bytes >= delta_cache_limit_bytes); @@ -1246,6 +1249,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const delta_last_try_flush_bytes = delta_bytes; LOG_DEBUG(log, "Foreground flush cache in checkSegmentUpdate, thread={} segment={}", thread_type, segment->info()); segment->flushCache(*dm_context); + triggerCompactLog(dm_context, segment, false); } else if (should_background_flush) { @@ -1260,6 +1264,10 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const try_add_background_task(BackgroundTask{TaskType::Flush, dm_context, segment}); } } + if (should_background_compact_log) + { + try_add_background_task(BackgroundTask{TaskType::NotifyCompactLog, dm_context, segment}); + } } // Need to check the latest delta (maybe updated after foreground flush). If it is updating by another thread, diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 0f5de36e872..d98be5e307e 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -193,6 +193,7 @@ class DeltaMergeStore : private boost::noncopyable Compact, Flush, PlaceIndex, + NotifyCompactLog, }; struct BackgroundTask @@ -588,7 +589,7 @@ class DeltaMergeStore : private boost::noncopyable private: void dropAllSegments(bool keep_first_segment); String getLogTracingId(const DMContext & dm_ctx); - + void triggerCompactLog(const DMContextPtr & dm_context, const SegmentPtr & segment, bool is_background); #ifndef DBMS_PUBLIC_GTEST private: #else diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index db071013477..b48c28b77cf 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -41,7 +42,6 @@ extern const char pause_until_dt_background_delta_merge[]; namespace DM { - // A callback class for scanning the DMFiles on local filesystem class LocalDMFileGcScanner final { @@ -307,6 +307,8 @@ bool DeltaMergeStore::handleBackgroundTask(bool heavy) left = task.segment; type = ThreadType::BG_Flush; break; + case TaskType::NotifyCompactLog: + triggerCompactLog(task.dm_context, task.segment, true); case TaskType::PlaceIndex: task.segment->placeDeltaIndex(*task.dm_context); break; @@ -829,5 +831,32 @@ UInt64 DeltaMergeStore::onSyncGc(Int64 limit, const GCOptions & gc_options) return gc_segments_num; } +void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const SegmentPtr & segment, bool is_background) +{ + auto & tmt = dm_context->db_context.getTMTContext(); + auto & kv_store = tmt.getKVStore(); + + if (is_background) + { + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg).Increment(); + } + else + { + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg).Increment(); + } + + Stopwatch watch; + SCOPE_EXIT({ + if (is_background) + { + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg).Observe(watch.elapsedSeconds()); + } + else + { + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(watch.elapsedSeconds()); + } + }); + kv_store->copmactLogByRowKeyRange(tmt, segment->getRowKeyRange(), physical_table_id, is_background); +} } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/RowKeyRange.h b/dbms/src/Storages/DeltaMerge/RowKeyRange.h index 96491471875..8c394d76e2c 100644 --- a/dbms/src/Storages/DeltaMerge/RowKeyRange.h +++ b/dbms/src/Storages/DeltaMerge/RowKeyRange.h @@ -277,6 +277,11 @@ inline bool operator<(const RowKeyValueRef & a, const RowKeyValueRef & b) return compare(a, b) < 0; } +inline bool operator<=(const RowKeyValueRef & a, const RowKeyValueRef & b) +{ + return compare(a, b) <= 0; +} + inline bool operator<(const StringRef & a, const RowKeyValueRef & b) { return compare(a, b) < 0; diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 85bbfcb0ac6..bd943ee6fa6 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include + #include #include #include @@ -38,6 +40,8 @@ #include #include #include +#include +#include #include #include #include diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 8a76f1d90da..7a6fb20c43a 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -1418,7 +1418,7 @@ BlockInputStreamPtr StorageDeltaMerge::status() auto & name_col = columns[0]; auto & value_col = columns[1]; - StoreStats stat; + DM::StoreStats stat; if (storeInited()) { stat = _store->getStoreStats(); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 48738cb31af..b7508eabda4 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -98,6 +98,7 @@ RegionPtr KVStore::getRegion(RegionID region_id) const return it->second; return nullptr; } +// TODO: may get regions not in segment? RegionMap KVStore::getRegionsByRangeOverlap(const RegionRange & range) const { auto manage_lock = genRegionReadLock(); @@ -365,7 +366,7 @@ bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_ } else { - return canFlushRegionDataImpl(curr_region_ptr, true, try_until_succeed, tmt, region_task_lock, index, term); + return canFlushRegionDataImpl(curr_region_ptr, false, try_until_succeed, tmt, region_task_lock, index, term); } } @@ -400,7 +401,7 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl LOG_DEBUG(log, "{} flush region due to tryFlushRegionData, index {} term {}", curr_region.toString(false), index, term); return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); } - return can_flush; + return false; } bool KVStore::forceFlushRegionDataImpl(Region & curr_region, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term) @@ -859,4 +860,79 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const return region_persister->getFileUsageStatistics(); } +// We need to get applied index before flushing cache, and can't hold region task lock when flush cache to avoid hang write cmd apply. +void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, TableID table_id, bool is_background) +{ + auto range = std::make_pair(TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.start.toRegionKey(table_id))), + TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.end.toRegionKey(table_id)))); + std::unordered_map> region_copmact_indexes; + auto region_read_lock = genRegionReadLock(); + auto region_map = getRegionsByRangeOverlap(range); + for (const auto & overlapped_region : region_map) + { + region_copmact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm()}; + } + auto storage = tmt.getStorages().get(table_id); + if (unlikely(storage == nullptr)) + { + LOG_WARNING(log, + "tryFlushRegionCacheInStorage can not get table for table id {}, ignored", + table_id); + return; + } + storage->flushCache(tmt.getContext(), rowkey_range); + for (const auto & region : region_copmact_indexes) + { + auto reion_ptr = getRegion(region.first); + if (!reion_ptr) + { + LOG_INFO(log, "region {} has been removed, ignore", region.first); + continue; + } + auto region_rowkey_range = DM::RowKeyRange::fromRegionRange( + reion_ptr->getRange(), + table_id, + storage->isCommonHandle(), + storage->getRowKeyColumnSize()); + if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) + { + notifyCompactLog(region.first, region.second.first, region.second.second, is_background); + reion_ptr->setFlushedState(region.second.first, region.second.second); + } + else + { + storage->flushCache(tmt.getContext(), region_rowkey_range, true); + notifyCompactLog(region.first, region.second.first, region.second.second, is_background); + reion_ptr->setFlushedState(region.second.first, region.second.second); + } + } +} + +// the caller guarantee that delta cache has been flushed. This function need to persiste region cache before trigger proxy to compact log. +void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background) +{ + auto region = getRegion(region_id); + if (!region) + { + LOG_INFO(log, "region {} has been removed, ignore", region_id); + return; + } + if (region->lastCompactLogTime() + Seconds{region_compact_log_period.load(std::memory_order_relaxed)} > Clock::now()) + { + return; + } + if (is_background) + { + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_bg).Increment(); + } + else + { + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_fg).Increment(); + } + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + persistRegion(*region, region_task_lock, "tryFlushRegionData"); + region->markCompactLog(); + region->cleanApproxMemCacheInfo(); + getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term); +} } // namespace DB diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 9b3f3286695..276bc0f5a0c 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -167,6 +168,8 @@ class KVStore final : private boost::noncopyable FileUsageStatistics getFileUsageStatistics() const; + void copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, TableID table_id, bool is_background); + void notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background); #ifndef DBMS_PUBLIC_GTEST private: #endif diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 7177822644c..216be36f8de 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -797,16 +797,27 @@ raft_serverpb::RegionLocalState TiFlashRaftProxyHelper::getRegionLocalState(uint return state; } +void TiFlashRaftProxyHelper::notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term) const +{ + this->fn_notify_compact_log(this->proxy_ptr, region_id, compact_index, compact_term, compact_index); +} + void HandleSafeTSUpdate(EngineStoreServerWrap * server, uint64_t region_id, uint64_t self_safe_ts, uint64_t leader_safe_ts) { RegionTable & region_table = server->tmt->getRegionTable(); region_table.updateSafeTS(region_id, leader_safe_ts, self_safe_ts); } - std::string_view buffToStrView(const BaseBuffView & buf) { return std::string_view{buf.data, buf.len}; } +FlushedState GetFlushedState(EngineStoreServerWrap * server, uint64_t region_id) +{ + auto & kvstore = server->tmt->getKVStore(); + auto region_ptr = kvstore->getRegion(region_id); + return region_ptr->getFlushedState(); +} + } // namespace DB diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index bc71e47c529..7b5de2ad101 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -116,6 +116,7 @@ struct TiFlashRaftProxyHelper : RaftStoreProxyFFIHelper TimerTask makeTimerTask(uint64_t time_ms) const; bool pollTimerTask(TimerTask & task, RawVoidPtr waker = nullptr) const; raft_serverpb::RegionLocalState getRegionLocalState(uint64_t region_id) const; + void notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term) const; }; extern "C" { @@ -169,6 +170,7 @@ CppStrWithView GetConfig(EngineStoreServerWrap *, uint8_t full); void SetStore(EngineStoreServerWrap *, BaseBuffView); void SetPBMsByBytes(MsgPBType type, RawVoidPtr ptr, BaseBuffView view); void HandleSafeTSUpdate(EngineStoreServerWrap * server, uint64_t region_id, uint64_t self_safe_ts, uint64_t leader_safe_ts); +FlushedState GetFlushedState(EngineStoreServerWrap * server, uint64_t region_id); } inline EngineStoreServerHelper GetEngineStoreServerHelper( @@ -215,6 +217,7 @@ inline EngineStoreServerHelper GetEngineStoreServerHelper( .fn_set_store = SetStore, .fn_set_pb_msg_by_bytes = SetPBMsByBytes, .fn_handle_safe_ts_update = HandleSafeTSUpdate, + .fn_get_flushed_state = GetFlushedState, }; } diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index f4aead264c9..be5ac2f29ca 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -202,6 +203,16 @@ class Region : public std::enable_shared_from_this std::pair getApproxMemCacheInfo() const; void cleanApproxMemCacheInfo() const; + void setFlushedState(uint64_t flushed_index, uint64_t flushed_term) + { + flushed_state.applied_index = flushed_index; + flushed_state.applied_term = flushed_term; + } + FlushedState getFlushedState() + { + return flushed_state; + } + private: Region() = delete; friend class RegionRaftCommandDelegate; @@ -237,6 +248,7 @@ class Region : public std::enable_shared_from_this mutable std::atomic last_compact_log_time{Timepoint::min()}; mutable std::atomic approx_mem_cache_rows{0}; mutable std::atomic approx_mem_cache_bytes{0}; + FlushedState flushed_state{0, 0}; }; class RegionRaftCommandDelegate : public Region diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index 36686b44d90..c28f61e82bb 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include From 2f873bfea6094a071de735f550fe1ecdaa592c2c Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 1 Mar 2023 17:12:13 +0800 Subject: [PATCH 02/66] fix lock Signed-off-by: hehechen --- contrib/tiflash-proxy | 2 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 88 +++++++++---------- dbms/src/Storages/Transaction/Region.cpp | 15 +++- 4 files changed, 58 insertions(+), 49 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 75cc2d64469..dc302d3e04b 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 75cc2d6446940f27c0b62fe9e77c290bd7e75bac +Subproject commit dc302d3e04b89ff617e2cd3dfddb5eb5c03b2cde diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index a10bacf2809..81dd5e95f91 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1175,7 +1175,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto delta_cache_limit_rows = dm_context->delta_cache_limit_rows; auto delta_cache_limit_bytes = dm_context->delta_cache_limit_bytes; - bool should_background_compact_log = (unsaved_rows >= delta_cache_limit_rows / 4 || unsaved_bytes >= delta_cache_limit_bytes / 4); + bool should_background_compact_log = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes); bool should_background_flush = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes) // && (delta_rows - delta_last_try_flush_rows >= delta_cache_limit_rows || delta_bytes - delta_last_try_flush_bytes >= delta_cache_limit_bytes); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index b7508eabda4..995a563eec9 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -30,6 +30,8 @@ #include #include +#include + namespace DB { namespace ErrorCodes @@ -370,7 +372,7 @@ bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_ } } -bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term) +bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8, bool, TMTContext &, const RegionTaskLock &, UInt64, UInt64) { if (curr_region_ptr == nullptr) { @@ -382,25 +384,6 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); - bool can_flush = false; - if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) - || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) - { - // if rows or bytes more than threshold, flush cache and persist mem data. - can_flush = true; - } - else - { - // if there is little data in mem, wait until time interval reached threshold. - // use random period so that lots of regions will not be persisted at same time. - auto compact_log_period = std::rand() % region_compact_log_period.load(std::memory_order_relaxed); // NOLINT - can_flush = !(curr_region.lastCompactLogTime() + Seconds{compact_log_period} > Clock::now()); - } - if (can_flush && flush_if_possible) - { - LOG_DEBUG(log, "{} flush region due to tryFlushRegionData, index {} term {}", curr_region.toString(false), index, term); - return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); - } return false; } @@ -861,17 +844,11 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const } // We need to get applied index before flushing cache, and can't hold region task lock when flush cache to avoid hang write cmd apply. +// 1. store applied index and applied term, +// 2. flush cache, +// 3. notify regions to compact log and store fushed state with applied index/term before flushing cache. void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, TableID table_id, bool is_background) { - auto range = std::make_pair(TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.start.toRegionKey(table_id))), - TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.end.toRegionKey(table_id)))); - std::unordered_map> region_copmact_indexes; - auto region_read_lock = genRegionReadLock(); - auto region_map = getRegionsByRangeOverlap(range); - for (const auto & overlapped_region : region_map) - { - region_copmact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm()}; - } auto storage = tmt.getStorages().get(table_id); if (unlikely(storage == nullptr)) { @@ -880,7 +857,41 @@ void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & table_id); return; } + auto range = std::make_pair(TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.start.toRegionKey(table_id))), + TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.end.toRegionKey(table_id)))); + std::unordered_map> region_copmact_indexes; + { + auto task_lock = genTaskLock(); + auto region_map = getRegionsByRangeOverlap(range); + for (const auto & overlapped_region : region_map) + { + auto region_rowkey_range = DM::RowKeyRange::fromRegionRange( + overlapped_region.second->getRange(), + table_id, + storage->isCommonHandle(), + storage->getRowKeyColumnSize()); + auto region_task_lock = region_manager.genRegionTaskLock(overlapped_region.first); + region_copmact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range}; + persistRegion(*overlapped_region.second, region_task_lock, "triggerCompactLog"); + } + } storage->flushCache(tmt.getContext(), rowkey_range); + // flush all segments in the range of regions. + // TODO: combine continues range to do one flush. + for (const auto & region : region_copmact_indexes) + { + auto region_rowkey_range = std::get<2>(region.second); + if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() < rowkey_range.getEnd()) + { + // This segment has flushed, skip it. + LOG_DEBUG(log, "flushed segment of region {}", region.first); + continue; + } + LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); + storage->flushCache(tmt.getContext(), std::get<2>(region.second)); + } + // forbid regions being removed. + auto task_lock = genTaskLock(); for (const auto & region : region_copmact_indexes) { auto reion_ptr = getRegion(region.first); @@ -889,22 +900,7 @@ void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & LOG_INFO(log, "region {} has been removed, ignore", region.first); continue; } - auto region_rowkey_range = DM::RowKeyRange::fromRegionRange( - reion_ptr->getRange(), - table_id, - storage->isCommonHandle(), - storage->getRowKeyColumnSize()); - if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) - { - notifyCompactLog(region.first, region.second.first, region.second.second, is_background); - reion_ptr->setFlushedState(region.second.first, region.second.second); - } - else - { - storage->flushCache(tmt.getContext(), region_rowkey_range, true); - notifyCompactLog(region.first, region.second.first, region.second.second, is_background); - reion_ptr->setFlushedState(region.second.first, region.second.second); - } + notifyCompactLog(region.first, std::get<0>(region.second), std::get<1>(region.second), is_background); } } @@ -930,7 +926,7 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_fg).Increment(); } auto region_task_lock = region_manager.genRegionTaskLock(region_id); - persistRegion(*region, region_task_lock, "tryFlushRegionData"); + region->setFlushedState(compact_index, compact_term); region->markCompactLog(); region->cleanApproxMemCacheInfo(); getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term); diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 56134925ba6..d5ee9329045 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -50,7 +50,20 @@ RegionData::WriteCFIter Region::removeDataByWriteIt(const RegionData::WriteCFIte RegionDataReadInfo Region::readDataByWriteIt(const RegionData::ConstWriteCFIter & write_it, bool need_value) const { - return data.readDataByWriteIt(write_it, need_value); + try + { + return data.readDataByWriteIt(write_it, need_value); + } + catch (DB::Exception & e) + { + e.addMessage(fmt::format("(region id {}, applied_index:{}, applied_term:{}, flushed_index:{}, flushed_term:{})", + meta.regionId(), + appliedIndex(), + appliedIndexTerm(), + flushed_state.applied_index, + flushed_state.applied_term)); + throw; + } } DecodedLockCFValuePtr Region::getLockInfo(const RegionLockReadQuery & query) const From ffd10d133201a2d2edb826b46c4fd86c25e509cd Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 29 May 2023 10:56:56 +0800 Subject: [PATCH 03/66] add lock held Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 24 ++++++++++++++++------- dbms/src/Storages/Transaction/KVStore.h | 2 +- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 995a563eec9..cbdededf2b5 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -900,12 +900,12 @@ void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & LOG_INFO(log, "region {} has been removed, ignore", region.first); continue; } - notifyCompactLog(region.first, std::get<0>(region.second), std::get<1>(region.second), is_background); + notifyCompactLog(region.first, std::get<0>(region.second), std::get<1>(region.second), is_background, false); } } // the caller guarantee that delta cache has been flushed. This function need to persiste region cache before trigger proxy to compact log. -void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background) +void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held) { auto region = getRegion(region_id); if (!region) @@ -925,10 +925,20 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 { GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_fg).Increment(); } - auto region_task_lock = region_manager.genRegionTaskLock(region_id); - region->setFlushedState(compact_index, compact_term); - region->markCompactLog(); - region->cleanApproxMemCacheInfo(); - getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term); + auto f = [&]() { + region->setFlushedState(compact_index, compact_term); + region->markCompactLog(); + region->cleanApproxMemCacheInfo(); + getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term); + }; + if (lock_held) + { + f(); + } + else + { + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + f(); + } } } // namespace DB diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 276bc0f5a0c..800688f542c 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -169,7 +169,7 @@ class KVStore final : private boost::noncopyable FileUsageStatistics getFileUsageStatistics() const; void copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, TableID table_id, bool is_background); - void notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background); + void notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held = true); #ifndef DBMS_PUBLIC_GTEST private: #endif From 08a7c457b3a6a17885a4fac1025d2678c08f95c6 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 30 May 2023 00:14:48 +0800 Subject: [PATCH 04/66] add tests Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- dbms/src/Debug/MockRaftStoreProxy.cpp | 6 +++ .../DeltaMerge/DeltaMergeStore_InternalBg.cpp | 2 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 5 +- dbms/src/Storages/Transaction/KVStore.cpp | 37 ++++++++++---- dbms/src/Storages/Transaction/KVStore.h | 2 +- dbms/src/Storages/Transaction/ProxyFFI.cpp | 1 + .../Transaction/tests/gtest_kvstore.cpp | 2 +- .../tests/gtest_proactive_flush.cpp | 50 +++++++++++++++++++ 9 files changed, 88 insertions(+), 19 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 57e17cb74b3..cca8d28fda5 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 57e17cb74b3c53e46fe815546cf9e75344ae34ff +Subproject commit cca8d28fda5ae92d512cd9673572c02c1a4b72e6 diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index c0df7514cee..efba46c21a3 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -134,6 +134,11 @@ KVGetStatus fn_get_region_local_state(RaftStoreProxyPtr ptr, uint64_t region_id, return KVGetStatus::NotFound; } +void fn_notify_compact_log(RaftStoreProxyPtr, uint64_t, uint64_t, uint64_t, uint64_t) +{ + // Do nothing +} + TiFlashRaftProxyHelper MockRaftStoreProxy::SetRaftStoreProxyFFIHelper(RaftStoreProxyPtr proxy_ptr) { TiFlashRaftProxyHelper res{}; @@ -143,6 +148,7 @@ TiFlashRaftProxyHelper MockRaftStoreProxy::SetRaftStoreProxyFFIHelper(RaftStoreP res.fn_make_async_waker = fn_make_async_waker; res.fn_handle_batch_read_index = fn_handle_batch_read_index; res.fn_get_region_local_state = fn_get_region_local_state; + res.fn_notify_compact_log = fn_notify_compact_log; { // make sure such function pointer will be set at most once. static std::once_flag flag; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index e1a0f698f38..509f6c64c18 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -947,7 +947,7 @@ void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const S GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(watch.elapsedSeconds()); } }); - kv_store->copmactLogByRowKeyRange(tmt, segment->getRowKeyRange(), physical_table_id, is_background); + kv_store->compactLogByRowKeyRange(tmt, segment->getRowKeyRange(), keyspace_id, physical_table_id, is_background); } } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index e150bf8db77..bb02bbb43e9 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include - #include #include #include @@ -50,11 +48,10 @@ #include #include #include -#include -#include #include #include #include +#include #include #include #include diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index a4d10d98eaa..7b778e41450 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -883,9 +883,9 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const // 1. store applied index and applied term, // 2. flush cache, // 3. notify regions to compact log and store fushed state with applied index/term before flushing cache. -void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, TableID table_id, bool is_background) +void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) { - auto storage = tmt.getStorages().get(table_id); + auto storage = tmt.getStorages().get(keyspace_id, table_id); if (unlikely(storage == nullptr)) { LOG_WARNING(log, @@ -895,7 +895,8 @@ void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & } auto range = std::make_pair(TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.start.toRegionKey(table_id))), TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.end.toRegionKey(table_id)))); - std::unordered_map> region_copmact_indexes; + LOG_DEBUG(log, "!!!! range {} {}", range.first.toDebugString(), range.second.toDebugString()); + std::unordered_map> region_compact_indexes; { auto task_lock = genTaskLock(); auto region_map = getRegionsByRangeOverlap(range); @@ -907,18 +908,23 @@ void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & storage->isCommonHandle(), storage->getRowKeyColumnSize()); auto region_task_lock = region_manager.genRegionTaskLock(overlapped_region.first); - region_copmact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range}; - persistRegion(*overlapped_region.second, region_task_lock, "triggerCompactLog"); + region_compact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range}; + persistRegion(*overlapped_region.second, std::make_optional(®ion_task_lock), "triggerCompactLog"); } } - storage->flushCache(tmt.getContext(), rowkey_range); // flush all segments in the range of regions. // TODO: combine continues range to do one flush. - for (const auto & region : region_copmact_indexes) + LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 1"); + for (const auto & region : region_compact_indexes) { auto region_rowkey_range = std::get<2>(region.second); - if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() < rowkey_range.getEnd()) + + LOG_DEBUG(log, "!!!! compactLogByRowKeyRange cmp {} {}", rowkey_range.getStart() <= region_rowkey_range.getStart(), region_rowkey_range.getEnd() < rowkey_range.getEnd()); + LOG_DEBUG(log, "!!!! compactLogByRowKeyRange end {} {}", region_rowkey_range.getEnd().toDebugString(), rowkey_range.getEnd().toDebugString()); + if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) { + // region_rowkey_range belongs to rowkey_range. + // E.g. [0,9223372036854775807] belongs to [-9223372036854775808,9223372036854775807]. // This segment has flushed, skip it. LOG_DEBUG(log, "flushed segment of region {}", region.first); continue; @@ -926,18 +932,24 @@ void KVStore::copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); storage->flushCache(tmt.getContext(), std::get<2>(region.second)); } + // Flush the segments that isn't related to any region. + storage->flushCache(tmt.getContext(), rowkey_range); + LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 2"); // forbid regions being removed. auto task_lock = genTaskLock(); - for (const auto & region : region_copmact_indexes) + for (const auto & region : region_compact_indexes) { - auto reion_ptr = getRegion(region.first); - if (!reion_ptr) + auto region_ptr = getRegion(region.first); + if (!region_ptr) { LOG_INFO(log, "region {} has been removed, ignore", region.first); continue; } + LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 3"); notifyCompactLog(region.first, std::get<0>(region.second), std::get<1>(region.second), is_background, false); + LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 4"); } + LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 5"); } // the caller guarantee that delta cache has been flushed. This function need to persiste region cache before trigger proxy to compact log. @@ -962,10 +974,13 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_fg).Increment(); } auto f = [&]() { + LOG_DEBUG(log, "!!!! notifyCompactLog 1"); region->setFlushedState(compact_index, compact_term); region->markCompactLog(); region->cleanApproxMemCacheInfo(); + LOG_DEBUG(log, "!!!! notifyCompactLog 2"); getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term); + LOG_DEBUG(log, "!!!! notifyCompactLog 3"); }; if (lock_held) { diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index f5305339450..987ad092205 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -175,7 +175,7 @@ class KVStore final : private boost::noncopyable FileUsageStatistics getFileUsageStatistics() const; - void copmactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, TableID table_id, bool is_background); + void compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background); void notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held = true); #ifndef DBMS_PUBLIC_GTEST private: diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 9ea0ec0b99b..bb605ed5bad 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -857,6 +857,7 @@ raft_serverpb::RegionLocalState TiFlashRaftProxyHelper::getRegionLocalState(uint void TiFlashRaftProxyHelper::notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term) const { + // TODO is it saft here to use compact_index as applied_index? this->fn_notify_compact_log(this->proxy_ptr, region_id, compact_index, compact_term, compact_index); } diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index ca1557038db..7bdd8472ea7 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -671,7 +671,7 @@ TEST_F(RegionKVStoreTest, Writes) } catch (Exception & e) { - ASSERT_EQ(e.message(), "Raw TiDB PK: 800000000000091D, Prewrite ts: 2333 can not found in default cf for key: 7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE"); + ASSERT_EQ(e.message(), "Raw TiDB PK: 800000000000091D, Prewrite ts: 2333 can not found in default cf for key: 7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE: (region id 1, applied_index:5, applied_term:5, flushed_index:0, flushed_term:0)"); ASSERT_EQ(kvs.getRegion(1)->dataInfo(), "[write 1 lock 1 ]"); kvs.getRegion(1)->tryCompactionFilter(1000); } diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index e69de29bb2d..be2f13550ae 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -0,0 +1,50 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 "kvstore_helper.h" + +namespace DB +{ +namespace tests +{ +TEST_F(RegionKVStoreTest, ProactiveFlush) +try +{ + auto ctx = TiFlashTestEnv::getGlobalContext(); + UInt64 region_id = 1; + TableID table_id; + KVStore & kvs = getKVS(); + { + initStorages(); + table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); + proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::nullopt); + auto kvr1 = kvs.getRegion(region_id); + ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); + } + { + auto kvr1 = kvs.getRegion(region_id); + ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); + auto & r1_range = kvr1->getRange()->comparableKeys(); + LOG_INFO(&Poco::Logger::get("!!!!"), "!!!!! r1 range {} {}", r1_range.first.toDebugString(), r1_range.second.toDebugString()); + + auto keyrange = DM::RowKeyRange::newAll(false, 10); + kvs.compactLogByRowKeyRange(ctx.getTMTContext(), keyrange, DB::NullspaceID, table_id, false); + } + { + } +} +CATCH + +} // namespace tests +} // namespace DB \ No newline at end of file From 6c5986f86dfdf45dc139c12485bb5f6c2dca5e1a Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 30 May 2023 11:32:39 +0800 Subject: [PATCH 05/66] add findByRangeChecked Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 35 ++++++++++++------- .../Transaction/RegionsRangeIndex.cpp | 25 +++++++++++++ .../Storages/Transaction/RegionsRangeIndex.h | 5 +++ 3 files changed, 53 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 7b778e41450..846d6294d75 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -32,6 +32,7 @@ #include #include +#include namespace DB { @@ -895,11 +896,30 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & } auto range = std::make_pair(TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.start.toRegionKey(table_id))), TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.end.toRegionKey(table_id)))); - LOG_DEBUG(log, "!!!! range {} {}", range.first.toDebugString(), range.second.toDebugString()); + LOG_DEBUG(log, "Start proactive compact region range [{},{}]", range.first.toDebugString(), range.second.toDebugString()); std::unordered_map> region_compact_indexes; { auto task_lock = genTaskLock(); - auto region_map = getRegionsByRangeOverlap(range); + auto maybe_region_map = [&]() { + auto manage_lock = genRegionReadLock(); + return manage_lock.index.findByRangeChecked(range); + }(); + + if (std::holds_alternative(maybe_region_map)) + { + auto & info = std::get(maybe_region_map); + FmtBuffer buffer; + buffer.joinStr( + std::get<1>(info).begin(), + std::get<1>(info).end(), + [&](const auto & e, FmtBuffer & b) { b.fmtAppend("{}", e); }, + " "); + std::string fmt_error = fmt::format("Find overlapped regions at {}, regions are {}, quit", std::get<0>(info).toDebugString(), buffer.toString()); + LOG_ERROR(log, fmt_error); + throw Exception(fmt_error, ErrorCodes::LOGICAL_ERROR); + } + + RegionMap & region_map = std::get(maybe_region_map); for (const auto & overlapped_region : region_map) { auto region_rowkey_range = DM::RowKeyRange::fromRegionRange( @@ -914,13 +934,10 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & } // flush all segments in the range of regions. // TODO: combine continues range to do one flush. - LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 1"); for (const auto & region : region_compact_indexes) { auto region_rowkey_range = std::get<2>(region.second); - LOG_DEBUG(log, "!!!! compactLogByRowKeyRange cmp {} {}", rowkey_range.getStart() <= region_rowkey_range.getStart(), region_rowkey_range.getEnd() < rowkey_range.getEnd()); - LOG_DEBUG(log, "!!!! compactLogByRowKeyRange end {} {}", region_rowkey_range.getEnd().toDebugString(), rowkey_range.getEnd().toDebugString()); if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) { // region_rowkey_range belongs to rowkey_range. @@ -933,8 +950,8 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & storage->flushCache(tmt.getContext(), std::get<2>(region.second)); } // Flush the segments that isn't related to any region. + // TODO Is it really necessary? storage->flushCache(tmt.getContext(), rowkey_range); - LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 2"); // forbid regions being removed. auto task_lock = genTaskLock(); for (const auto & region : region_compact_indexes) @@ -945,11 +962,8 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & LOG_INFO(log, "region {} has been removed, ignore", region.first); continue; } - LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 3"); notifyCompactLog(region.first, std::get<0>(region.second), std::get<1>(region.second), is_background, false); - LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 4"); } - LOG_DEBUG(log, "!!!! compactLogByRowKeyRange 5"); } // the caller guarantee that delta cache has been flushed. This function need to persiste region cache before trigger proxy to compact log. @@ -974,13 +988,10 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_fg).Increment(); } auto f = [&]() { - LOG_DEBUG(log, "!!!! notifyCompactLog 1"); region->setFlushedState(compact_index, compact_term); region->markCompactLog(); region->cleanApproxMemCacheInfo(); - LOG_DEBUG(log, "!!!! notifyCompactLog 2"); getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term); - LOG_DEBUG(log, "!!!! notifyCompactLog 3"); }; if (lock_held) { diff --git a/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp b/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp index a2eeceed464..4da6951aef7 100644 --- a/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp +++ b/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp @@ -73,6 +73,31 @@ RegionMap RegionsRangeIndex::findByRangeOverlap(const RegionRange & range) const return res; } +std::variant RegionsRangeIndex::findByRangeChecked(const RegionRange & range) const +{ + auto begin_it = root.lower_bound(range.first); + auto end_it = root.lower_bound(range.second); + if (begin_it->first.compare(range.first) != 0) + --begin_it; + + RegionMap res; + for (auto it = begin_it; it != end_it; ++it) + { + if (it->second.region_map.size() < 2) + res.insert(it->second.region_map.begin(), it->second.region_map.end()); + else + { + std::vector v; + for (auto iter = it->second.region_map.begin(); iter != it->second.region_map.end(); iter++) + { + v.push_back(iter->first); + } + return std::make_tuple(it->first.copy(), std::move(v)); + } + } + return res; +} + RegionsRangeIndex::RegionsRangeIndex() { clear(); diff --git a/dbms/src/Storages/Transaction/RegionsRangeIndex.h b/dbms/src/Storages/Transaction/RegionsRangeIndex.h index 671053a9bd4..24d62affc93 100644 --- a/dbms/src/Storages/Transaction/RegionsRangeIndex.h +++ b/dbms/src/Storages/Transaction/RegionsRangeIndex.h @@ -17,6 +17,7 @@ #include #include +#include namespace DB { @@ -42,6 +43,7 @@ class RegionsRangeIndex : private boost::noncopyable { public: using RootMap = std::map; + using OverlapInfo = std::tuple>; void add(const RegionPtr & new_region); @@ -49,6 +51,9 @@ class RegionsRangeIndex : private boost::noncopyable RegionMap findByRangeOverlap(const RegionRange & range) const; + // Returns a region map of all regions of range, or the id of the first region that is checked overlapped with another region. + std::variant findByRangeChecked(const RegionRange & range) const; + RegionsRangeIndex(); const RootMap & getRoot() const; From a7723f44b06e59ac1a8fef6fc97445484880390c Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 30 May 2023 13:19:07 +0800 Subject: [PATCH 06/66] f Signed-off-by: CalvinNeo --- dbms/src/Storages/DeltaMerge/DeltaMergeStore.h | 2 +- dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 2 +- dbms/src/Storages/Transaction/ProxyFFI.cpp | 2 +- dbms/src/Storages/Transaction/RegionsRangeIndex.cpp | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 29a80eddcbf..f29665235a4 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -687,7 +687,7 @@ class DeltaMergeStore : private boost::noncopyable private: void dropAllSegments(bool keep_first_segment); String getLogTracingId(const DMContext & dm_ctx); - void triggerCompactLog(const DMContextPtr & dm_context, const SegmentPtr & segment, bool is_background); + void triggerCompactLog(const DMContextPtr & dm_context, const SegmentPtr & segment, bool is_background) const; #ifndef DBMS_PUBLIC_GTEST private: #else diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index 509f6c64c18..22c7140bc4e 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -922,7 +922,7 @@ UInt64 DeltaMergeStore::onSyncGc(Int64 limit, const GCOptions & gc_options) return gc_segments_num; } -void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const SegmentPtr & segment, bool is_background) +void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const SegmentPtr & segment, bool is_background) const { auto & tmt = dm_context->db_context.getTMTContext(); auto & kv_store = tmt.getKVStore(); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 846d6294d75..933fb82eb95 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -919,7 +919,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & throw Exception(fmt_error, ErrorCodes::LOGICAL_ERROR); } - RegionMap & region_map = std::get(maybe_region_map); + auto & region_map = std::get(maybe_region_map); for (const auto & overlapped_region : region_map) { auto region_rowkey_range = DM::RowKeyRange::fromRegionRange( diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index bb605ed5bad..28d1c91535d 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -857,7 +857,7 @@ raft_serverpb::RegionLocalState TiFlashRaftProxyHelper::getRegionLocalState(uint void TiFlashRaftProxyHelper::notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term) const { - // TODO is it saft here to use compact_index as applied_index? + // TODO is it safe here to use compact_index as applied_index? this->fn_notify_compact_log(this->proxy_ptr, region_id, compact_index, compact_term, compact_index); } diff --git a/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp b/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp index 4da6951aef7..41d2ac95ef6 100644 --- a/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp +++ b/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp @@ -88,7 +88,7 @@ std::variant RegionsRangeIndex::findB else { std::vector v; - for (auto iter = it->second.region_map.begin(); iter != it->second.region_map.end(); iter++) + for (const auto & iter : it->second.region_map) { v.push_back(iter->first); } From 03a3da3e6e2b60ab192550ceba870b5b527f082f Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 30 May 2023 16:10:27 +0800 Subject: [PATCH 07/66] fix Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 3 ++- dbms/src/Storages/Transaction/RegionsRangeIndex.cpp | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index cca8d28fda5..152d78daab8 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit cca8d28fda5ae92d512cd9673572c02c1a4b72e6 +Subproject commit 152d78daab857ceef756d5b7f46a82660f32df02 diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 933fb82eb95..83ac0408ffe 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -966,7 +966,8 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & } } -// the caller guarantee that delta cache has been flushed. This function need to persiste region cache before trigger proxy to compact log. +// The caller will guarantee that delta cache has been flushed. +// This function requires region cache being persisted before notifying. void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held) { auto region = getRegion(region_id); diff --git a/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp b/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp index 41d2ac95ef6..af79ea40fc2 100644 --- a/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp +++ b/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp @@ -90,7 +90,7 @@ std::variant RegionsRangeIndex::findB std::vector v; for (const auto & iter : it->second.region_map) { - v.push_back(iter->first); + v.push_back(iter.first); } return std::make_tuple(it->first.copy(), std::move(v)); } From 9c0e7337f1c07f9c570d6595cfd1aa65573d0acd Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Wed, 31 May 2023 13:31:38 +0800 Subject: [PATCH 08/66] lock Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 83ac0408ffe..93af4eebbe4 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -896,8 +896,10 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & } auto range = std::make_pair(TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.start.toRegionKey(table_id))), TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.end.toRegionKey(table_id)))); + Stopwatch watch; + LOG_DEBUG(log, "Start proactive compact region range [{},{}]", range.first.toDebugString(), range.second.toDebugString()); - std::unordered_map> region_compact_indexes; + std::unordered_map> region_compact_indexes; { auto task_lock = genTaskLock(); auto maybe_region_map = [&]() { @@ -928,8 +930,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & storage->isCommonHandle(), storage->getRowKeyColumnSize()); auto region_task_lock = region_manager.genRegionTaskLock(overlapped_region.first); - region_compact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range}; - persistRegion(*overlapped_region.second, std::make_optional(®ion_task_lock), "triggerCompactLog"); + region_compact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range, overlapped_region.second}; } } // flush all segments in the range of regions. @@ -946,12 +947,18 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & LOG_DEBUG(log, "flushed segment of region {}", region.first); continue; } + auto region_id = region.first; + auto region_ptr = std::get<3>(region.second); LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); + auto region_task_lock = region_manager.genRegionTaskLock(region_id); storage->flushCache(tmt.getContext(), std::get<2>(region.second)); + persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } - // Flush the segments that isn't related to any region. - // TODO Is it really necessary? - storage->flushCache(tmt.getContext(), rowkey_range); + // TODO Flush the segments that isn't related to any region. Is it really necessary? + // storage->flushCache(tmt.getContext(), rowkey_range); + auto elapsed_coupled_flush = watch.elapsedMilliseconds(); + watch.restart(); + // forbid regions being removed. auto task_lock = genTaskLock(); for (const auto & region : region_compact_indexes) @@ -964,6 +971,9 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & } notifyCompactLog(region.first, std::get<0>(region.second), std::get<1>(region.second), is_background, false); } + auto elapsed_notify_proxy = watch.elapsedMilliseconds(); + + LOG_DEBUG(log, "Finished proactive compact region range [{},{}], couple_flush {} notify_proxy {}", range.first.toDebugString(), range.second.toDebugString(), elapsed_coupled_flush, elapsed_notify_proxy); } // The caller will guarantee that delta cache has been flushed. From 2105a5ae4b5d5076a0e844d90805577e3bd7cc8e Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Thu, 15 Jun 2023 15:28:16 +0800 Subject: [PATCH 09/66] fix lock Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 93af4eebbe4..2e809bb6c12 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -904,6 +904,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & auto task_lock = genTaskLock(); auto maybe_region_map = [&]() { auto manage_lock = genRegionReadLock(); + // Check if the region overlaps. return manage_lock.index.findByRangeChecked(range); }(); @@ -929,7 +930,6 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & table_id, storage->isCommonHandle(), storage->getRowKeyColumnSize()); - auto region_task_lock = region_manager.genRegionTaskLock(overlapped_region.first); region_compact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range, overlapped_region.second}; } } From 253061a60d211556ae381a6689e57fe4caf9cc57 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 16 Jun 2023 11:41:57 +0800 Subject: [PATCH 10/66] try recursive Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 3 ++- dbms/src/Storages/Transaction/RegionManager.h | 7 ++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 2e809bb6c12..d0395fb42ed 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -890,7 +890,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & if (unlikely(storage == nullptr)) { LOG_WARNING(log, - "tryFlushRegionCacheInStorage can not get table for table id {}, ignored", + "compactLogByRowKeyRange can not get table for table id {}, ignored", table_id); return; } @@ -951,6 +951,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & auto region_ptr = std::get<3>(region.second); LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); auto region_task_lock = region_manager.genRegionTaskLock(region_id); + // Both flushCache and persistRegion should be protected by region task lock. storage->flushCache(tmt.getContext(), std::get<2>(region.second)); persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } diff --git a/dbms/src/Storages/Transaction/RegionManager.h b/dbms/src/Storages/Transaction/RegionManager.h index eebeb5183b2..d02d88514b4 100644 --- a/dbms/src/Storages/Transaction/RegionManager.h +++ b/dbms/src/Storages/Transaction/RegionManager.h @@ -26,10 +26,11 @@ class RegionTaskLock; struct RegionTaskCtrl : MutexLockWrap { + typedef std::recursive_mutex Mut; /// The life time of each RegionTaskElement element should be as long as RegionManager, just return const ref. struct RegionTaskElement : private boost::noncopyable { - mutable std::mutex mutex; + mutable Mut mutex; }; /// Encapsulate the task lock for region RegionTaskLock genRegionTaskLock(RegionID region_id) const; @@ -84,10 +85,10 @@ class RegionTaskLock : private boost::noncopyable { friend struct RegionTaskCtrl; - explicit RegionTaskLock(std::mutex & mutex_) + explicit RegionTaskLock(RegionTaskCtrl::Mut & mutex_) : lock(mutex_) {} - std::lock_guard lock; + std::lock_guard lock; }; } // namespace DB From df454d5d9d241f30b737d0d36d98d160cc059d21 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 16 Jun 2023 14:58:13 +0800 Subject: [PATCH 11/66] dont lock kvstore Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index d0395fb42ed..368b16b402c 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -964,12 +964,8 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & auto task_lock = genTaskLock(); for (const auto & region : region_compact_indexes) { - auto region_ptr = getRegion(region.first); - if (!region_ptr) - { - LOG_INFO(log, "region {} has been removed, ignore", region.first); - continue; - } + // Can truncated to flushed index, which is applied_index in this case. + // Region can be removed since we don't lock kvstore here. notifyCompactLog(region.first, std::get<0>(region.second), std::get<1>(region.second), is_background, false); } auto elapsed_notify_proxy = watch.elapsedMilliseconds(); From 5686b5c8791ce2ae8001498a976f039df826f051 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 16 Jun 2023 17:03:38 +0800 Subject: [PATCH 12/66] tto be revert: remove all locks Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 368b16b402c..4331f4d4b71 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -950,9 +950,9 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & auto region_id = region.first; auto region_ptr = std::get<3>(region.second); LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); - auto region_task_lock = region_manager.genRegionTaskLock(region_id); // Both flushCache and persistRegion should be protected by region task lock. storage->flushCache(tmt.getContext(), std::get<2>(region.second)); + auto region_task_lock = region_manager.genRegionTaskLock(region_id); persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } // TODO Flush the segments that isn't related to any region. Is it really necessary? From 94c92656ef871c13f07dfdb45fd83478b04c5756 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Jun 2023 09:48:58 +0800 Subject: [PATCH 13/66] Revert "tto be revert: remove all locks" This reverts commit 5686b5c8791ce2ae8001498a976f039df826f051. --- dbms/src/Storages/Transaction/KVStore.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 4331f4d4b71..368b16b402c 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -950,9 +950,9 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & auto region_id = region.first; auto region_ptr = std::get<3>(region.second); LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); + auto region_task_lock = region_manager.genRegionTaskLock(region_id); // Both flushCache and persistRegion should be protected by region task lock. storage->flushCache(tmt.getContext(), std::get<2>(region.second)); - auto region_task_lock = region_manager.genRegionTaskLock(region_id); persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } // TODO Flush the segments that isn't related to any region. Is it really necessary? From 7db53fe9feb94d9820e044b90d4de6576ee10a30 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Jun 2023 14:02:24 +0800 Subject: [PATCH 14/66] to revert really remove lock Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 4 ++-- dbms/src/Storages/Transaction/Region.cpp | 11 +++++++---- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 368b16b402c..51bca8fcf81 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -950,9 +950,9 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & auto region_id = region.first; auto region_ptr = std::get<3>(region.second); LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); - auto region_task_lock = region_manager.genRegionTaskLock(region_id); // Both flushCache and persistRegion should be protected by region task lock. storage->flushCache(tmt.getContext(), std::get<2>(region.second)); + auto region_task_lock = region_manager.genRegionTaskLock(region_id); persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } // TODO Flush the segments that isn't related to any region. Is it really necessary? @@ -961,7 +961,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & watch.restart(); // forbid regions being removed. - auto task_lock = genTaskLock(); + // auto task_lock = genTaskLock(); for (const auto & region : region_compact_indexes) { // Can truncated to flushed index, which is applied_index in this case. diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index eca8ad86aa6..ddddd447421 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -722,16 +722,19 @@ EngineStoreApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt6 }; { - std::unique_lock lock(mutex); - - handle_write_cmd_func(); + { + // RegionTable::writeBlockByRegion may lead to persistRegion when flush proactively. + // So we can't lock here. + std::unique_lock lock(mutex); + handle_write_cmd_func(); + } // If transfer-leader happened during ingest-sst, there might be illegal data. if (0 != cmds.len) { /// Flush data right after they are committed. RegionDataReadInfoList data_list_to_remove; - RegionTable::writeBlockByRegion(context, shared_from_this(), data_list_to_remove, log, false); + RegionTable::writeBlockByRegion(context, shared_from_this(), data_list_to_remove, log, true); } meta.setApplied(index, term); From ba3596c6f967dc4ecebda5884d173a3554cb4469 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 20 Jun 2023 12:13:18 +0800 Subject: [PATCH 15/66] revert Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 3 +-- dbms/src/Storages/Transaction/Region.cpp | 2 ++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 51bca8fcf81..9e308b2d98d 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -951,8 +951,8 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & auto region_ptr = std::get<3>(region.second); LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); // Both flushCache and persistRegion should be protected by region task lock. - storage->flushCache(tmt.getContext(), std::get<2>(region.second)); auto region_task_lock = region_manager.genRegionTaskLock(region_id); + storage->flushCache(tmt.getContext(), std::get<2>(region.second)); persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } // TODO Flush the segments that isn't related to any region. Is it really necessary? @@ -961,7 +961,6 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & watch.restart(); // forbid regions being removed. - // auto task_lock = genTaskLock(); for (const auto & region : region_compact_indexes) { // Can truncated to flushed index, which is applied_index in this case. diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index ddddd447421..0e82dd74094 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -725,6 +725,8 @@ EngineStoreApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt6 { // RegionTable::writeBlockByRegion may lead to persistRegion when flush proactively. // So we can't lock here. + // Safety: Mutations to a region come from raft applying and bg flushing of storage layer. + // Both way, they must firstly acquires the RegionTask lock. std::unique_lock lock(mutex); handle_write_cmd_func(); } From 2dfaab1604c5bf52590ce76da3e027f70655d5cd Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 20 Jun 2023 23:25:03 +0800 Subject: [PATCH 16/66] fix 2 fg lock Signed-off-by: CalvinNeo --- .../DeltaMerge/DeltaMergeInterfaces.h | 58 ++++++++++++++++++ .../Storages/DeltaMerge/DeltaMergeStore.cpp | 61 +++++++++++-------- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 59 ++++++++++++++---- .../DeltaMerge/DeltaMergeStore_Ingest.cpp | 4 +- .../DeltaMerge/DeltaMergeStore_InternalBg.cpp | 16 ++--- dbms/src/Storages/StorageDeltaMerge.cpp | 7 ++- dbms/src/Storages/StorageDeltaMerge.h | 3 +- .../Storages/Transaction/ApplySnapshot.cpp | 3 + dbms/src/Storages/Transaction/KVStore.cpp | 38 ++++++++---- dbms/src/Storages/Transaction/KVStore.h | 4 +- .../Storages/Transaction/PartitionStreams.cpp | 22 ++++--- dbms/src/Storages/Transaction/Region.cpp | 14 +++-- dbms/src/Storages/Transaction/Region.h | 3 +- dbms/src/Storages/Transaction/RegionTable.h | 10 +-- 14 files changed, 217 insertions(+), 85 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h b/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h new file mode 100644 index 00000000000..9e87ec8763c --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h @@ -0,0 +1,58 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +/// In this header defines interfaces that exposed to KVStore layer. + +#pragma once + +#include + +namespace DB +{ +namespace DM +{ +struct RaftWriteResult +{ + // We will find all segments and regions by this range. + std::vector pending_flush_ranges; + KeyspaceID keyspace_id; + TableID table_id; + + RaftWriteResult(std::vector && ranges, KeyspaceID keyspace, TableID table_id_) + : pending_flush_ranges(std::move(ranges)) + , keyspace_id(keyspace) + , table_id(table_id_) + {} + + DISALLOW_COPY(RaftWriteResult); + + RaftWriteResult(RaftWriteResult && other) + { + pending_flush_ranges = std::move(other.pending_flush_ranges); + keyspace_id = other.keyspace_id; + table_id = other.table_id; + } + + RaftWriteResult & operator=(RaftWriteResult && other) + { + pending_flush_ranges = std::move(other.pending_flush_ranges); + keyspace_id = other.keyspace_id; + table_id = other.table_id; + return *this; + } +}; +typedef std::optional WriteResult; +static_assert(std::is_move_constructible_v); +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 46f43956df9..a3805f47e20 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -533,10 +533,10 @@ Block DeltaMergeStore::addExtraColumnIfNeed(const Context & db_context, const Co handle_column); } } - return std::move(block); + return block; } -void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_settings, Block & block) +DM::WriteResult DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_settings, Block & block) { LOG_TRACE(log, "Table write block, rows={} bytes={}", block.rows(), block.bytes()); @@ -544,7 +544,7 @@ void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_ const auto rows = block.rows(); if (rows == 0) - return; + return std::nullopt; auto dm_context = newDMContext(db_context, db_settings, "write"); @@ -673,9 +673,8 @@ void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_ throw Exception("Fail point random_exception_after_dt_write_done is triggered.", ErrorCodes::FAIL_POINT_ERROR); }); - // TODO: Update the tracing_id before checkSegmentUpdate - for (auto & segment : updated_segments) - checkSegmentUpdate(dm_context, segment, ThreadType::Write); + // TODO: Update the tracing_id before checkSegmentsUpdateForKVStore + return checkSegmentsUpdateForKVStore(dm_context, updated_segments.begin(), updated_segments.end(), ThreadType::Write, InputType::RaftLog); } void DeltaMergeStore::deleteRange(const Context & db_context, const DB::Settings & db_settings, const RowKeyRange & delete_range) @@ -734,7 +733,8 @@ void DeltaMergeStore::deleteRange(const Context & db_context, const DB::Settings // TODO: Update the tracing_id before checkSegmentUpdate? for (auto & segment : updated_segments) - checkSegmentUpdate(dm_context, segment, ThreadType::Write); + // We don't handle delete range from raft, the delete range is for dm's purpose only. + checkSegmentUpdate(dm_context, segment, ThreadType::Write, InputType::NotRaft); } bool DeltaMergeStore::flushCache(const Context & context, const RowKeyRange & range, bool try_until_succeed) @@ -957,7 +957,7 @@ BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, }); auto after_segment_read = [&](const DMContextPtr & dm_context_, const SegmentPtr & segment_) { - this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read); + this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read, InputType::NotRaft); }; size_t final_num_stream = std::min(num_streams, tasks.size()); String req_info; @@ -1073,7 +1073,7 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, auto after_segment_read = [&](const DMContextPtr & dm_context_, const SegmentPtr & segment_) { // TODO: Update the tracing_id before checkSegmentUpdate? - this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read); + this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read, InputType::NotRaft); }; GET_METRIC(tiflash_storage_read_tasks_count).Increment(tasks.size()); @@ -1169,7 +1169,7 @@ void DeltaMergeStore::read( auto after_segment_read = [&](const DMContextPtr & dm_context_, const SegmentPtr & segment_) { // TODO: Update the tracing_id before checkSegmentUpdate? - this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read); + this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read, InputType::NotRaft); }; GET_METRIC(tiflash_storage_read_tasks_count).Increment(tasks.size()); @@ -1309,7 +1309,7 @@ void DeltaMergeStore::waitForWrite(const DMContextPtr & dm_context, const Segmen } // checkSegmentUpdate could do foreground merge delta, so call it before sleep. - checkSegmentUpdate(dm_context, segment, ThreadType::Write); + checkSegmentUpdate(dm_context, segment, ThreadType::Write, InputType::NotRaft); size_t sleep_step = 50; // Wait at most `sleep_ms` until the delta is merged. @@ -1319,7 +1319,7 @@ void DeltaMergeStore::waitForWrite(const DMContextPtr & dm_context, const Segmen size_t ms = std::min(sleep_ms, sleep_step); std::this_thread::sleep_for(std::chrono::milliseconds(ms)); sleep_ms -= ms; - checkSegmentUpdate(dm_context, segment, ThreadType::Write); + checkSegmentUpdate(dm_context, segment, ThreadType::Write, InputType::NotRaft); } } @@ -1328,12 +1328,13 @@ void DeltaMergeStore::waitForDeleteRange(const DB::DM::DMContextPtr &, const DB: // TODO: maybe we should wait, if there are too many delete ranges? } -void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const SegmentPtr & segment, ThreadType thread_type) +bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const SegmentPtr & segment, ThreadType thread_type, InputType input_type) { - fiu_do_on(FailPoints::skip_check_segment_update, { return; }); + bool should_trigger_kvstore_flush = false; + fiu_do_on(FailPoints::skip_check_segment_update, { return should_trigger_kvstore_flush; }); if (segment->hasAbandoned()) - return; + return should_trigger_kvstore_flush; const auto & delta = segment->getDelta(); size_t delta_saved_rows = delta->getRows(/* use_unsaved */ false); @@ -1444,7 +1445,14 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const delta_last_try_flush_bytes = delta_bytes; LOG_DEBUG(log, "Foreground flush cache in checkSegmentUpdate, thread={} segment={}", thread_type, segment->info()); segment->flushCache(*dm_context); - triggerCompactLog(dm_context, segment, false); + if (input_type == InputType::RaftLog) + { + // Only the segment update is from a raft log write, will we notify KVStore to trigger a foreground flush. + // Raft Snapshot will always trigger to a KVStore fg flush. + // Raft IngestSST will trigger a KVStore fg flush at best effort, + // which means if the write cf has remained value, we still need to hold the sst file and wait for the next SST. + should_trigger_kvstore_flush = true; + } } else if (should_background_flush) { @@ -1468,7 +1476,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const // Need to check the latest delta (maybe updated after foreground flush). If it is updating by another thread, // give up adding more tasks on this version of delta. if (segment->getDelta()->isUpdating()) - return; + return should_trigger_kvstore_flush; auto try_fg_merge_delta = [&]() -> SegmentPtr { // If the table is already dropped, don't trigger foreground merge delta when executing `remove region peer`, @@ -1559,19 +1567,19 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const if (thread_type == ThreadType::Write) { if (try_fg_split(segment)) - return; + return should_trigger_kvstore_flush; if (SegmentPtr new_segment = try_fg_merge_delta(); new_segment) { // After merge delta, we better check split immediately. if (try_bg_split(new_segment)) - return; + return should_trigger_kvstore_flush; } } else if (thread_type == ThreadType::BG_MergeDelta) { if (try_bg_split(segment)) - return; + return should_trigger_kvstore_flush; } if (dm_context->enable_logical_split) @@ -1579,23 +1587,24 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const // Logical split point is calculated based on stable. Always try to merge delta into the stable // before logical split is good for calculating the split point. if (try_bg_merge_delta()) - return; + return should_trigger_kvstore_flush; if (try_bg_split(segment)) - return; + return should_trigger_kvstore_flush; } else { // During the physical split delta will be merged, so we prefer physical split over merge delta. if (try_bg_split(segment)) - return; + return should_trigger_kvstore_flush; if (try_bg_merge_delta()) - return; + return should_trigger_kvstore_flush; } if (try_bg_compact()) - return; + return should_trigger_kvstore_flush; if (try_place_delta_index()) - return; + return should_trigger_kvstore_flush; + return should_trigger_kvstore_flush; // The segment does not need any updates for now. } diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index f29665235a4..77e3e27e7af 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -198,6 +199,14 @@ class DeltaMergeStore : private boost::noncopyable BG_GC, }; + enum class InputType + { + // We are not handling data from raft, maybe it's from a scheduled background service or a replicated dm snapshot. + NotRaft, + RaftLog, + RaftSSTAndSnap, + }; + enum TaskType { Split, @@ -283,7 +292,7 @@ class DeltaMergeStore : private boost::noncopyable static Block addExtraColumnIfNeed(const Context & db_context, const ColumnDefine & handle_define, Block && block); - void write(const Context & db_context, const DB::Settings & db_settings, Block & block); + DM::WriteResult write(const Context & db_context, const DB::Settings & db_settings, Block & block); void deleteRange(const Context & db_context, const DB::Settings & db_settings, const RowKeyRange & delete_range); @@ -514,16 +523,31 @@ class DeltaMergeStore : private boost::noncopyable void waitForDeleteRange(const DMContextPtr & context, const SegmentPtr & segment); - /** - * Try to update the segment. "Update" means splitting the segment into two, merging two segments, merging the delta, etc. - * If an update is really performed, the segment will be abandoned (with `segment->hasAbandoned() == true`). - * See `segmentSplit`, `segmentMerge`, `segmentMergeDelta` for details. - * - * This may be called from multiple threads, e.g. at the foreground write moment, or in background threads. - * A `thread_type` should be specified indicating the type of the thread calling this function. - * Depend on the thread type, the "update" to do may be varied. - */ - void checkSegmentUpdate(const DMContextPtr & context, const SegmentPtr & segment, ThreadType thread_type); + // Deferencing `Iter` can get a pointer to a Segment. + template + DM::WriteResult checkSegmentsUpdateForKVStore(const DMContextPtr & context, Iter begin, Iter end, ThreadType thread_type, InputType input_type) + { + DM::WriteResult result = std::nullopt; + std::vector ranges; + if (thread_type != ThreadType::Write) + return result; + for (Iter it = begin; it != end; it++) + { + if (checkSegmentUpdate(context, *it, thread_type, input_type)) + { + ranges.push_back((*it)->getRowKeyRange()); + } + } + // TODO We can try merge ranges here. + if (!ranges.empty()) + { + result = RaftWriteResult{ + std::move(ranges), + keyspace_id, + physical_table_id}; + } + return result; + } enum class SegmentSplitReason { @@ -685,9 +709,20 @@ class DeltaMergeStore : private boost::noncopyable bool try_split_task = true); private: + /** + * Try to update the segment. "Update" means splitting the segment into two, merging two segments, merging the delta, etc. + * If an update is really performed, the segment will be abandoned (with `segment->hasAbandoned() == true`). + * See `segmentSplit`, `segmentMerge`, `segmentMergeDelta` for details. + * + * This may be called from multiple threads, e.g. at the foreground write moment, or in background threads. + * A `thread_type` should be specified indicating the type of the thread calling this function. + * Depend on the thread type, the "update" to do may be varied. + */ + bool checkSegmentUpdate(const DMContextPtr & context, const SegmentPtr & segment, ThreadType thread_type, InputType input_type); + void dropAllSegments(bool keep_first_segment); String getLogTracingId(const DMContext & dm_ctx); - void triggerCompactLog(const DMContextPtr & dm_context, const SegmentPtr & segment, bool is_background) const; + void triggerCompactLog(const DMContextPtr & dm_context, const RowKeyRange & range, bool is_background) const; #ifndef DBMS_PUBLIC_GTEST private: #else diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp index 8ccff885cac..f66bf6b7abb 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp @@ -743,7 +743,7 @@ void DeltaMergeStore::ingestFiles( // TODO: Update the tracing_id before checkSegmentUpdate? for (auto & segment : updated_segments) - checkSegmentUpdate(dm_context, segment, ThreadType::Write); + checkSegmentUpdate(dm_context, segment, ThreadType::Write, InputType::RaftSSTAndSnap); } std::vector DeltaMergeStore::ingestSegmentsUsingSplit( @@ -1081,7 +1081,7 @@ void DeltaMergeStore::ingestSegmentsFromCheckpointInfo( } for (auto & segment : updated_segments) - checkSegmentUpdate(dm_context, segment, ThreadType::Write); + checkSegmentUpdate(dm_context, segment, ThreadType::Write, InputType::NotRaft); } } // namespace DM diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index 22c7140bc4e..5bc44db4d14 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -283,7 +283,7 @@ void DeltaMergeStore::setUpBackgroundTask(const DMContextPtr & dm_context) for (auto & [end, segment] : segments) { (void)end; - checkSegmentUpdate(dm_context, segment, ThreadType::Init); + checkSegmentUpdate(dm_context, segment, ThreadType::Init, InputType::NotRaft); } } @@ -399,7 +399,7 @@ bool DeltaMergeStore::handleBackgroundTask(bool heavy) type = ThreadType::BG_Flush; break; case TaskType::NotifyCompactLog: - triggerCompactLog(task.dm_context, task.segment, true); + triggerCompactLog(task.dm_context, task.segment->getRowKeyRange(), true); case TaskType::PlaceIndex: task.segment->placeDeltaIndex(*task.dm_context); break; @@ -425,9 +425,9 @@ bool DeltaMergeStore::handleBackgroundTask(bool heavy) // continue to check whether we need to apply more tasks after this task is ended. if (left) - checkSegmentUpdate(task.dm_context, left, type); + checkSegmentUpdate(task.dm_context, left, type, InputType::NotRaft); if (right) - checkSegmentUpdate(task.dm_context, right, type); + checkSegmentUpdate(task.dm_context, right, type, InputType::NotRaft); return true; } @@ -690,7 +690,7 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMerge(const DMContextPtr & dm_context, c auto new_segment = segmentMerge(*dm_context, segments_to_merge, SegmentMergeReason::BackgroundGCThread); if (new_segment) { - checkSegmentUpdate(dm_context, new_segment, ThreadType::BG_GC); + checkSegmentUpdate(dm_context, new_segment, ThreadType::BG_GC, InputType::NotRaft); } return new_segment; @@ -813,7 +813,7 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMergeDelta(const DMContextPtr & dm_conte } segment_snap = {}; - checkSegmentUpdate(dm_context, new_segment, ThreadType::BG_GC); + checkSegmentUpdate(dm_context, new_segment, ThreadType::BG_GC, InputType::NotRaft); return new_segment; } @@ -922,7 +922,7 @@ UInt64 DeltaMergeStore::onSyncGc(Int64 limit, const GCOptions & gc_options) return gc_segments_num; } -void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const SegmentPtr & segment, bool is_background) const +void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const RowKeyRange & range, bool is_background) const { auto & tmt = dm_context->db_context.getTMTContext(); auto & kv_store = tmt.getKVStore(); @@ -947,7 +947,7 @@ void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const S GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(watch.elapsedSeconds()); } }); - kv_store->compactLogByRowKeyRange(tmt, segment->getRowKeyRange(), keyspace_id, physical_table_id, is_background); + kv_store->compactLogByRowKeyRange(tmt, range, keyspace_id, physical_table_id, is_background); } } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 42dc324b3e9..5301860c777 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -428,7 +428,8 @@ class DMBlockOutputStream : public IBlockOutputStream if (db_settings.dt_insert_max_rows == 0) { Block to_write = decorator(block); - return store->write(db_context, db_settings, to_write); + store->write(db_context, db_settings, to_write); + return; } Block new_block = decorator(block); @@ -472,7 +473,7 @@ BlockOutputStreamPtr StorageDeltaMerge::write(const ASTPtr & query, const Settin return std::make_shared(getAndMaybeInitStore(), decorator, global_context, settings); } -void StorageDeltaMerge::write(Block & block, const Settings & settings) +WriteResult StorageDeltaMerge::write(Block & block, const Settings & settings) { auto & store = getAndMaybeInitStore(); #ifndef NDEBUG @@ -533,7 +534,7 @@ void StorageDeltaMerge::write(Block & block, const Settings & settings) FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_write_to_storage); - store->write(global_context, settings, block); + return store->write(global_context, settings, block); } std::unordered_set parseSegmentSet(const ASTPtr & ast) diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 745c04681b6..e4af7cfff09 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -90,7 +91,7 @@ class StorageDeltaMerge BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override; /// Write from raft layer. - void write(Block & block, const Settings & settings); + DM::WriteResult write(Block & block, const Settings & settings); void flushCache(const Context & context) override; diff --git a/dbms/src/Storages/Transaction/ApplySnapshot.cpp b/dbms/src/Storages/Transaction/ApplySnapshot.cpp index b0e18374979..da87363a80b 100644 --- a/dbms/src/Storages/Transaction/ApplySnapshot.cpp +++ b/dbms/src/Storages/Transaction/ApplySnapshot.cpp @@ -521,6 +521,7 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec // try to flush remain data in memory. func_try_flush(); auto tmp_region = handleIngestSSTByDTFile(region, snaps, index, term, tmt); + // Merge data from tmp_region. region->finishIngestSSTByDTFile(std::move(tmp_region), index, term); // after `finishIngestSSTByDTFile`, try to flush committed data into storage func_try_flush(); @@ -533,6 +534,8 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec } else { + // We always try to flush dm cache and region if possible for every IngestSST, + // in order to have the raft log truncated and sst deleted. persistRegion(*region, ®ion_task_lock, __FUNCTION__); return EngineStoreApplyRes::Persist; } diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 9e308b2d98d..5ef7c5d290d 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -247,7 +247,7 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmd( UInt64 region_id, UInt64 index, UInt64 term, - TMTContext & tmt) const + TMTContext & tmt) { std::vector keys; std::vector vals; @@ -288,17 +288,31 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmd( tmt); } -EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) const +EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) { - auto region_persist_lock = region_manager.genRegionTaskLock(region_id); + DM::WriteResult write_result = std::nullopt; + EngineStoreApplyRes res; + { + auto region_persist_lock = region_manager.genRegionTaskLock(region_id); - const RegionPtr region = getRegion(region_id); - if (region == nullptr) + const RegionPtr region = getRegion(region_id); + if (region == nullptr) + { + return EngineStoreApplyRes::NotFound; + } + + auto && [r, w] = region->handleWriteRaftCmd(cmds, index, term, tmt); + write_result = std::move(w); + res = r; + } + if (write_result) { - return EngineStoreApplyRes::NotFound; + auto & inner = write_result.value(); + for (auto it = inner.pending_flush_ranges.begin(); it != inner.pending_flush_ranges.end(); it++) + { + compactLogByRowKeyRange(tmt, *it, inner.keyspace_id, inner.table_id, false); + } } - - auto res = region->handleWriteRaftCmd(cmds, index, term, tmt); return res; } @@ -933,15 +947,15 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & region_compact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range, overlapped_region.second}; } } - // flush all segments in the range of regions. - // TODO: combine continues range to do one flush. + // Flush all segments in the range of regions. + // TODO: combine adjacent range to do one flush. for (const auto & region : region_compact_indexes) { auto region_rowkey_range = std::get<2>(region.second); if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) { - // region_rowkey_range belongs to rowkey_range. + // `region_rowkey_range` belongs to rowkey_range. // E.g. [0,9223372036854775807] belongs to [-9223372036854775808,9223372036854775807]. // This segment has flushed, skip it. LOG_DEBUG(log, "flushed segment of region {}", region.first); @@ -955,8 +969,6 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & storage->flushCache(tmt.getContext(), std::get<2>(region.second)); persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } - // TODO Flush the segments that isn't related to any region. Is it really necessary? - // storage->flushCache(tmt.getContext(), rowkey_range); auto elapsed_coupled_flush = watch.elapsedMilliseconds(); watch.restart(); diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 987ad092205..22e972c3818 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -111,8 +111,8 @@ class KVStore final : private boost::noncopyable UInt64 region_id, UInt64 index, UInt64 term, - TMTContext & tmt) const; - EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) const; + TMTContext & tmt); + EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); bool needFlushRegionData(UInt64 region_id, TMTContext & tmt); bool tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term); diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index d68e4b15e90..ca440320fbd 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -52,7 +52,7 @@ extern const int TABLE_IS_DROPPED; } // namespace ErrorCodes -static void writeRegionDataToStorage( +static DM::WriteResult writeRegionDataToStorage( Context & context, const RegionPtrWithBlock & region, RegionDataReadInfoList & data_list_read, @@ -64,6 +64,7 @@ static void writeRegionDataToStorage( TableID table_id = region->getMappedTableID(); UInt64 region_decode_cost = -1, write_part_cost = -1; + DM::WriteResult write_result = std::nullopt; /// Declare lambda of atomic read then write to call multiple times. auto atomic_read_write = [&](bool force_decode) { /// Get storage based on table ID. @@ -145,9 +146,13 @@ static void writeRegionDataToStorage( { auto dm_storage = std::dynamic_pointer_cast(storage); if (need_decode) - dm_storage->write(*block_ptr, context.getSettingsRef()); + { + write_result = dm_storage->write(*block_ptr, context.getSettingsRef()); + } else - dm_storage->write(block, context.getSettingsRef()); + { + write_result = dm_storage->write(block, context.getSettingsRef()); + } break; } default: @@ -180,7 +185,7 @@ static void writeRegionDataToStorage( { if (atomic_read_write(false)) { - return; + return write_result; } } @@ -196,6 +201,7 @@ static void writeRegionDataToStorage( throw Exception("Write region " + std::to_string(region->id()) + " to table " + std::to_string(table_id) + " failed", ErrorCodes::LOGICAL_ERROR); } + return write_result; } } @@ -334,7 +340,7 @@ static inline void reportUpstreamLatency(const RegionDataReadInfoList & data_lis } } -void RegionTable::writeBlockByRegion( +DM::WriteResult RegionTable::writeBlockByRegion( Context & context, const RegionPtrWithBlock & region, RegionDataReadInfoList & data_list_to_remove, @@ -353,15 +359,16 @@ void RegionTable::writeBlockByRegion( } if (!data_list_read) - return; + return std::nullopt; reportUpstreamLatency(*data_list_read); - writeRegionDataToStorage(context, region, *data_list_read, log); + auto write_result = writeRegionDataToStorage(context, region, *data_list_read, log); RemoveRegionCommitCache(region, *data_list_read, lock_region); /// Save removed data to outer. data_list_to_remove = std::move(*data_list_read); + return write_result; } RegionTable::ResolveLocksAndWriteRegionRes RegionTable::resolveLocksAndWriteRegion(TMTContext & tmt, @@ -387,6 +394,7 @@ RegionTable::ResolveLocksAndWriteRegionRes RegionTable::resolveLocksAndWriteRegi if (data_list_read.empty()) return RegionException::RegionReadStatus::OK; auto & context = tmt.getContext(); + // There is no raft input here, so we can just ignore the fg flush request. writeRegionDataToStorage(context, region, data_list_read, log); RemoveRegionCommitCache(region, data_list_read); return RegionException::RegionReadStatus::OK; diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 0e82dd74094..bed27ce06fc 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -637,11 +638,11 @@ void Region::tryCompactionFilter(const Timestamp safe_point) } } -EngineStoreApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt) +std::pair Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt) { if (index <= appliedIndex()) { - return EngineStoreApplyRes::None; + return std::make_pair(EngineStoreApplyRes::None, std::nullopt); } auto & context = tmt.getContext(); Stopwatch watch; @@ -721,12 +722,15 @@ EngineStoreApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt6 approx_mem_cache_bytes += cache_written_size; }; + DM::WriteResult write_result = std::nullopt; { { // RegionTable::writeBlockByRegion may lead to persistRegion when flush proactively. // So we can't lock here. // Safety: Mutations to a region come from raft applying and bg flushing of storage layer. - // Both way, they must firstly acquires the RegionTask lock. + // 1. A raft applying process should acquire the region task lock. + // 2. While bg/fg flushing, applying raft logs should also be prevented with region task lock. + // So between here and RegionTable::writeBlockByRegion, there will be no new data applied. std::unique_lock lock(mutex); handle_write_cmd_func(); } @@ -736,7 +740,7 @@ EngineStoreApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt6 { /// Flush data right after they are committed. RegionDataReadInfoList data_list_to_remove; - RegionTable::writeBlockByRegion(context, shared_from_this(), data_list_to_remove, log, true); + write_result = RegionTable::writeBlockByRegion(context, shared_from_this(), data_list_to_remove, log, true); } meta.setApplied(index, term); @@ -744,7 +748,7 @@ EngineStoreApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt6 meta.notifyAll(); - return EngineStoreApplyRes::None; + return std::make_pair(EngineStoreApplyRes::None, std::move(write_result)); } void Region::finishIngestSSTByDTFile(RegionPtr && rhs, UInt64 index, UInt64 term) diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index 50b9bc809a3..3520156a7f5 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -197,7 +198,7 @@ class Region : public std::enable_shared_from_this TableID getMappedTableID() const; KeyspaceID getKeyspaceID() const; - EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt); + std::pair handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt); void finishIngestSSTByDTFile(RegionPtr && rhs, UInt64 index, UInt64 term); UInt64 getSnapshotEventFlag() const { return snapshot_event_flag; } diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index ffca743b4b4..7d64a14a17e 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -177,11 +177,11 @@ class RegionTable : private boost::noncopyable /// Will trigger schema sync on read error for only once, /// assuming that newer schema can always apply to older data by setting force_decode to true in RegionBlockReader::read. /// Note that table schema must be keep unchanged throughout the process of read then write, we take good care of the lock. - static void writeBlockByRegion(Context & context, - const RegionPtrWithBlock & region, - RegionDataReadInfoList & data_list_to_remove, - const LoggerPtr & log, - bool lock_region = true); + static DM::WriteResult writeBlockByRegion(Context & context, + const RegionPtrWithBlock & region, + RegionDataReadInfoList & data_list_to_remove, + const LoggerPtr & log, + bool lock_region = true); /// Check transaction locks in region, and write committed data in it into storage engine if check passed. Otherwise throw an LockException. /// The write logic is the same as #writeBlockByRegion, with some extra checks about region version and conf_version. From 6d6cc3736c74963b049a463257f98b66cf49bf3c Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Sun, 25 Jun 2023 18:39:53 +0800 Subject: [PATCH 17/66] add some tests Signed-off-by: CalvinNeo --- dbms/src/Common/FailPoint.cpp | 6 +- dbms/src/Debug/MockRaftStoreProxy.cpp | 77 +++++++++++- dbms/src/Debug/MockRaftStoreProxy.h | 10 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 33 ++++- .../DeltaMerge/DeltaMergeStore_InternalBg.cpp | 20 --- dbms/src/Storages/Transaction/KVStore.cpp | 66 ++++++++-- dbms/src/Storages/Transaction/KVStore.h | 10 ++ .../Storages/Transaction/PartitionStreams.cpp | 1 + dbms/src/Storages/Transaction/Region.cpp | 2 +- .../Storages/Transaction/RegionCFDataBase.cpp | 2 +- dbms/src/Storages/Transaction/RegionMeta.cpp | 2 +- dbms/src/Storages/Transaction/RegionMeta.h | 3 +- .../Transaction/tests/gtest_kvstore.cpp | 2 +- .../Transaction/tests/gtest_new_kvstore.cpp | 10 +- .../tests/gtest_proactive_flush.cpp | 116 ++++++++++++++++-- .../Transaction/tests/kvstore_helper.h | 4 + 16 files changed, 310 insertions(+), 54 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index b4f0e5cf6a0..c9454a31d0d 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -101,7 +101,8 @@ namespace DB M(force_set_mocked_s3_object_mtime) \ M(force_stop_background_checkpoint_upload) \ M(skip_seek_before_read_dmfile) \ - M(exception_after_large_write_exceed) + M(exception_after_large_write_exceed) \ + M(proactive_flush_force_set_type) #define APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) \ M(pause_with_alter_locks_acquired) \ @@ -120,7 +121,8 @@ namespace DB M(pause_when_ingesting_to_dt_store) \ M(pause_when_altering_dt_store) \ M(pause_after_copr_streams_acquired) \ - M(pause_query_init) + M(pause_query_init) \ + M(proactive_flush_before_persist_region) #define APPLY_FOR_RANDOM_FAILPOINTS(M) \ diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index efba46c21a3..133b54cd1b1 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -20,15 +20,19 @@ #include #include #include +#include #include #include +#include #include #include +#include #include #include #include #include + namespace DB { namespace RegionBench @@ -412,6 +416,38 @@ void MockRaftStoreProxy::debugAddRegions( } } +void MockRaftStoreProxy::loadRegionFromKVStore( + KVStore & kvs, + TMTContext & tmt, + UInt64 region_id) +{ + UNUSED(tmt); + auto kvr = kvs.getRegion(region_id); + auto ori_r = getRegion(region_id); + auto commit_index = RAFT_INIT_LOG_INDEX; + auto commit_term = RAFT_INIT_LOG_TERM; + if (!ori_r) + { + regions.emplace(region_id, std::make_shared(region_id)); + } + else + { + commit_index = ori_r->getLatestCommitIndex(); + commit_term = ori_r->getLatestCommitTerm(); + } + MockProxyRegionPtr r = getRegion(region_id); + { + r->state = kvr->mutMeta().getRegionState().getBase(); + r->apply = kvr->mutMeta().clonedApplyState(); + if (r->apply.commit_index() == 0) + { + r->apply.set_commit_index(commit_index); + r->apply.set_commit_term(commit_term); + } + } + LOG_INFO(log, "loadRegionFromKVStore [region_id={}] region_state {} apply_state {}", region_id, r->state.DebugString(), r->apply.DebugString()); +} + std::tuple MockRaftStoreProxy::normalWrite( UInt64 region_id, std::vector && keys, @@ -609,7 +645,8 @@ void MockRaftStoreProxy::doApply( TMTContext & tmt, const FailCond & cond, UInt64 region_id, - uint64_t index) + uint64_t index, + std::optional check_proactive_flush) { auto region = getRegion(region_id); assert(region != nullptr); @@ -658,7 +695,21 @@ void MockRaftStoreProxy::doApply( if (cmd.has_raw_write_request()) { // TiFlash write - kvs.handleWriteRaftCmd(std::move(request), region_id, index, term, tmt); + DB::DM::WriteResult write_task; + kvs.handleWriteRaftCmdDebug(std::move(request), region_id, index, term, tmt, write_task); + if (check_proactive_flush) + { + if (check_proactive_flush.value()) + { + // fg flush + ASSERT(write_task.has_value()); + } + else + { + // bg flush + ASSERT(!write_task.has_value()); + } + } } if (cmd.has_admin_request()) { @@ -679,6 +730,14 @@ void MockRaftStoreProxy::doApply( // TODO We should remove (0, index] here, it is enough to remove exactly index now. region->commands.erase(i); } + else if (cmd.admin().cmd_type() == raft_cmdpb::AdminCmdType::BatchSplit) + { + for (auto && sp : cmd.admin().response.splits().regions()) + { + auto r = sp.id(); + loadRegionFromKVStore(kvs, tmt, r); + } + } } // Proxy advance @@ -828,6 +887,20 @@ TableID MockRaftStoreProxy::bootstrapTable( return table_id; } +std::pair MockRaftStoreProxy::generateTiKVKeyValue(uint64_t tso, int64_t t) const +{ + WriteBufferFromOwnString buff; + writeChar(RecordKVFormat::CFModifyFlag::PutFlag, buff); + EncodeVarUInt(tso, buff); + std::string value_write = buff.releaseStr(); + buff.restart(); + auto && table_info = MockTiDB::instance().getTableInfoByID(table_id); + std::vector f{Field{std::move(t)}}; + encodeRowV1(*table_info, f, buff); + std::string value_default = buff.releaseStr(); + return std::make_pair(value_write, value_default); +} + void GCMonitor::add(RawObjType type, int64_t diff) { auto _ = genLockGuard(); diff --git a/dbms/src/Debug/MockRaftStoreProxy.h b/dbms/src/Debug/MockRaftStoreProxy.h index 6cbb070ceb2..69df61d85ab 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.h +++ b/dbms/src/Debug/MockRaftStoreProxy.h @@ -209,6 +209,11 @@ struct MockRaftStoreProxy : MutexLockWrap std::vector region_ids, std::vector> && ranges); + void loadRegionFromKVStore( + KVStore & kvs, + TMTContext & tmt, + UInt64 region_id); + /// We assume that we generate one command, and immediately commit. /// Normal write to a region. std::tuple normalWrite( @@ -280,7 +285,8 @@ struct MockRaftStoreProxy : MutexLockWrap TMTContext & tmt, const FailCond & cond, UInt64 region_id, - uint64_t index); + uint64_t index, + std::optional check_proactive_flush = std::nullopt); void replay( KVStore & kvs, @@ -294,6 +300,8 @@ struct MockRaftStoreProxy : MutexLockWrap regions.clear(); } + std::pair generateTiKVKeyValue(uint64_t tso, int64_t t) const; + MockRaftStoreProxy() { log = Logger::get("MockRaftStoreProxy"); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index a3805f47e20..62c87822398 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -100,6 +100,8 @@ extern const char random_exception_after_dt_write_done[]; extern const char force_slow_page_storage_snapshot_release[]; extern const char exception_before_drop_segment[]; extern const char exception_after_drop_segment[]; +extern const char proactive_flush_before_persist_region[]; +extern const char proactive_flush_force_set_type[]; } // namespace FailPoints namespace DM @@ -533,7 +535,7 @@ Block DeltaMergeStore::addExtraColumnIfNeed(const Context & db_context, const Co handle_column); } } - return block; + return std::move(block); } DM::WriteResult DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_settings, Block & block) @@ -1371,6 +1373,7 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto delta_cache_limit_rows = dm_context->delta_cache_limit_rows; auto delta_cache_limit_bytes = dm_context->delta_cache_limit_bytes; + bool should_background_compact_log = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes); bool should_background_flush = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes) // && (delta_rows - delta_last_try_flush_rows >= delta_cache_limit_rows @@ -1411,6 +1414,29 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const fiu_do_on(FailPoints::force_triggle_background_merge_delta, { should_background_merge_delta = true; }); fiu_do_on(FailPoints::force_triggle_foreground_flush, { should_foreground_flush = true; }); + fiu_do_on(FailPoints::proactive_flush_force_set_type, { + if (auto v = FailPointHelper::getFailPointVal(FailPoints::proactive_flush_force_set_type); v) + { + auto set_kind = std::any_cast>>(v.value()); + auto set_kind_int = set_kind->load(); + if (set_kind_int == 1) + { + LOG_INFO(log, "!!!! AAAAA 1"); + should_foreground_flush = true; + should_background_flush = false; + } + else if (set_kind_int == 2) + { + LOG_INFO(log, "!!!! AAAAA 2"); + should_foreground_flush = false; + should_background_flush = true; + } + } + }); + + LOG_INFO(log, "!!!!! segment_limit_rows {} segment_limit_bytes {} delta_cache_limit_rows {} delta_cache_limit_bytes {}, {}. should_foreground_flush {} should_background_flush {}", segment_limit_rows, segment_limit_bytes, delta_cache_limit_rows, delta_cache_limit_bytes, StackTrace().toString(), should_foreground_flush, should_background_flush); + + auto try_add_background_task = [&](const BackgroundTask & task) { if (shutdown_called.load(std::memory_order_relaxed)) return; @@ -1425,6 +1451,9 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const background_task_handle->wake(); }; + /// Note a bg flush task may still be added even when we have a fg flush here. + /// This bg flush may be better since it may update delta index. + /// Flush is always try first. if (thread_type != ThreadType::Read) { @@ -1443,7 +1472,7 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const delta_last_try_flush_rows = delta_rows; delta_last_try_flush_bytes = delta_bytes; - LOG_DEBUG(log, "Foreground flush cache in checkSegmentUpdate, thread={} segment={}", thread_type, segment->info()); + LOG_DEBUG(log, "Foreground flush cache in checkSegmentUpdate, thread={} segment={} input_type={}", thread_type, segment->info(), magic_enum::enum_name(input_type)); segment->flushCache(*dm_context); if (input_type == InputType::RaftLog) { diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index 5bc44db4d14..e6b8acd3a60 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -927,26 +927,6 @@ void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const R auto & tmt = dm_context->db_context.getTMTContext(); auto & kv_store = tmt.getKVStore(); - if (is_background) - { - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg).Increment(); - } - else - { - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg).Increment(); - } - - Stopwatch watch; - SCOPE_EXIT({ - if (is_background) - { - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg).Observe(watch.elapsedSeconds()); - } - else - { - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(watch.elapsedSeconds()); - } - }); kv_store->compactLogByRowKeyRange(tmt, range, keyspace_id, physical_table_id, is_background); } } // namespace DM diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 5ef7c5d290d..fc64a12949f 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -45,6 +45,7 @@ extern const int TABLE_IS_DROPPED; namespace FailPoints { extern const char force_fail_in_flush_region_data[]; +extern const char proactive_flush_before_persist_region[]; } // namespace FailPoints KVStore::KVStore(Context & context) @@ -242,12 +243,13 @@ RegionManager::RegionWriteLock KVStore::genRegionWriteLock(const KVStoreTaskLock return region_manager.genRegionWriteLock(); } -EngineStoreApplyRes KVStore::handleWriteRaftCmd( +EngineStoreApplyRes KVStore::handleWriteRaftCmdInner( raft_cmdpb::RaftCmdRequest && request, UInt64 region_id, UInt64 index, UInt64 term, - TMTContext & tmt) + TMTContext & tmt, + DM::WriteResult & write_result) { std::vector keys; std::vector vals; @@ -280,17 +282,17 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmd( throw Exception(fmt::format("Unsupport raft cmd {}", raft_cmdpb::CmdType_Name(type)), ErrorCodes::LOGICAL_ERROR); } } - return handleWriteRaftCmd( + return handleWriteRaftCmdInner( WriteCmdsView{.keys = keys.data(), .vals = vals.data(), .cmd_types = cmd_types.data(), .cmd_cf = cmd_cf.data(), .len = keys.size()}, region_id, index, term, - tmt); + tmt, + write_result); } -EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) +EngineStoreApplyRes KVStore::handleWriteRaftCmdInner(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt, DM::WriteResult & write_result) { - DM::WriteResult write_result = std::nullopt; EngineStoreApplyRes res; { auto region_persist_lock = region_manager.genRegionTaskLock(region_id); @@ -305,6 +307,10 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt write_result = std::move(w); res = r; } + /// Safety: + /// This call is from Proxy's applying thread of this region, so: + /// 1. No other thread can write from raft to this region even if we unlocked here. + /// 2. If `compactLogByRowKeyRange` causes a write stall, it will be forwarded to raft layer. if (write_result) { auto & inner = write_result.value(); @@ -316,6 +322,28 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt return res; } +EngineStoreApplyRes KVStore::handleWriteRaftCmd( + raft_cmdpb::RaftCmdRequest && request, + UInt64 region_id, + UInt64 index, + UInt64 term, + TMTContext & tmt) +{ + DM::WriteResult write_result; + return handleWriteRaftCmdInner(std::move(request), region_id, index, term, tmt, write_result); +} + +EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) +{ + DM::WriteResult write_result; + return handleWriteRaftCmdInner(cmds, region_id, index, term, tmt, write_result); +} + +EngineStoreApplyRes KVStore::handleWriteRaftCmdDebug(raft_cmdpb::RaftCmdRequest && request, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt, DM::WriteResult & write_result) +{ + return handleWriteRaftCmdInner(std::move(request), region_id, index, term, tmt, write_result); +} + void KVStore::handleDestroy(UInt64 region_id, TMTContext & tmt) { handleDestroy(region_id, tmt, genTaskLock()); @@ -900,6 +928,27 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const // 3. notify regions to compact log and store fushed state with applied index/term before flushing cache. void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) { + if (is_background) + { + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg).Increment(); + } + else + { + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg).Increment(); + } + + Stopwatch general_watch; + SCOPE_EXIT({ + if (is_background) + { + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg).Observe(general_watch.elapsedSeconds()); + } + else + { + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(general_watch.elapsedSeconds()); + } + }); + auto storage = tmt.getStorages().get(keyspace_id, table_id); if (unlikely(storage == nullptr)) { @@ -912,7 +961,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.end.toRegionKey(table_id)))); Stopwatch watch; - LOG_DEBUG(log, "Start proactive compact region range [{},{}]", range.first.toDebugString(), range.second.toDebugString()); + LOG_INFO(log, "Start proactive flush region range [{},{}] [table_id={}] [keyspace_id={}] [is_background={}]", range.first.toDebugString(), range.second.toDebugString(), table_id, keyspace_id, is_background); std::unordered_map> region_compact_indexes; { auto task_lock = genTaskLock(); @@ -947,6 +996,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & region_compact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range, overlapped_region.second}; } } + FAIL_POINT_PAUSE(FailPoints::proactive_flush_before_persist_region); // Flush all segments in the range of regions. // TODO: combine adjacent range to do one flush. for (const auto & region : region_compact_indexes) @@ -981,7 +1031,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & } auto elapsed_notify_proxy = watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Finished proactive compact region range [{},{}], couple_flush {} notify_proxy {}", range.first.toDebugString(), range.second.toDebugString(), elapsed_coupled_flush, elapsed_notify_proxy); + LOG_DEBUG(log, "Finished proactive flush region range [{},{}] of {} regions. [couple_flush={}] [notify_proxy={}] [table_id={}] [keyspace_id={}] [is_background={}]", range.first.toDebugString(), range.second.toDebugString(), region_compact_indexes.size(), elapsed_coupled_flush, elapsed_notify_proxy, table_id, keyspace_id, is_background); } // The caller will guarantee that delta cache has been flushed. diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 22e972c3818..4de62bd4748 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -113,6 +114,15 @@ class KVStore final : private boost::noncopyable UInt64 term, TMTContext & tmt); EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); + EngineStoreApplyRes handleWriteRaftCmdDebug(raft_cmdpb::RaftCmdRequest && request, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt, DM::WriteResult & write_result); + EngineStoreApplyRes handleWriteRaftCmdInner( + raft_cmdpb::RaftCmdRequest && request, + UInt64 region_id, + UInt64 index, + UInt64 term, + TMTContext & tmt, + DM::WriteResult & write_result); + EngineStoreApplyRes handleWriteRaftCmdInner(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt, DM::WriteResult & write_result); bool needFlushRegionData(UInt64 region_id, TMTContext & tmt); bool tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term); diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index ca440320fbd..dd8c3cc70ec 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -69,6 +69,7 @@ static DM::WriteResult writeRegionDataToStorage( auto atomic_read_write = [&](bool force_decode) { /// Get storage based on table ID. auto storage = tmt.getStorages().get(keyspace_id, table_id); + if (storage == nullptr || storage->isTombstone()) { if (!force_decode) // Need to update. diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index bed27ce06fc..f726c3fa4b2 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -197,7 +197,7 @@ Regions RegionRaftCommandDelegate::execBatchSplit( if (new_region_index == -1) throw Exception(std::string(__PRETTY_FUNCTION__) + ": region index not found", ErrorCodes::LOGICAL_ERROR); - RegionMeta new_meta(meta.getPeer(), new_region_infos[new_region_index], meta.getApplyState()); + RegionMeta new_meta(meta.getPeer(), new_region_infos[new_region_index], meta.clonedApplyState()); new_meta.setApplied(index, term); meta.assignRegionMeta(std::move(new_meta)); } diff --git a/dbms/src/Storages/Transaction/RegionCFDataBase.cpp b/dbms/src/Storages/Transaction/RegionCFDataBase.cpp index 089bc0af2dc..29d238fbd18 100644 --- a/dbms/src/Storages/Transaction/RegionCFDataBase.cpp +++ b/dbms/src/Storages/Transaction/RegionCFDataBase.cpp @@ -46,7 +46,6 @@ RegionDataRes RegionCFDataBase::insert(TiKVKey && key, TiKVValue && value auto kv_pair = Trait::genKVPair(std::move(key), raw_key, std::move(value)); if (!kv_pair) return 0; - return insert(std::move(*kv_pair), mode); } @@ -70,6 +69,7 @@ RegionDataRes RegionCFDataBase::insert(std::pair && kv_pair, prev_value = TiKVValue::copyFrom(getTiKVValue(kv_pair.second)); } auto [it, ok] = map.emplace(std::move(kv_pair)); + // We support duplicated kv pairs if they are the same in snapshot. // This is because kvs in raftstore v2's snapshot may be overlapped. // However, we still not permit duplicated kvs from raft cmd. diff --git a/dbms/src/Storages/Transaction/RegionMeta.cpp b/dbms/src/Storages/Transaction/RegionMeta.cpp index 8911b349193..234e578ca52 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.cpp +++ b/dbms/src/Storages/Transaction/RegionMeta.cpp @@ -78,7 +78,7 @@ void RegionMeta::setPeer(metapb::Peer && p) peer = p; } -raft_serverpb::RaftApplyState RegionMeta::getApplyState() const +raft_serverpb::RaftApplyState RegionMeta::clonedApplyState() const { std::lock_guard lock(mutex); return apply_state; diff --git a/dbms/src/Storages/Transaction/RegionMeta.h b/dbms/src/Storages/Transaction/RegionMeta.h index e0b993df280..7f7ecb53371 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.h +++ b/dbms/src/Storages/Transaction/RegionMeta.h @@ -81,7 +81,7 @@ class RegionMeta UInt64 confVer() const; - raft_serverpb::RaftApplyState getApplyState() const; + raft_serverpb::RaftApplyState clonedApplyState() const; void setApplied(UInt64 index, UInt64 term); void notifyAll() const; @@ -113,6 +113,7 @@ class RegionMeta metapb::Region cloneMetaRegion() const; const raft_serverpb::MergeState & getMergeState() const; raft_serverpb::MergeState cloneMergeState() const; + const RegionState & getRegionState() const { return region_state; }; RegionMeta() = delete; diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 7bdd8472ea7..5c7e9d0be1e 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -120,7 +120,7 @@ TEST_F(RegionKVStoreTest, ReadIndex) std::this_thread::sleep_for(std::chrono::milliseconds(2)); auto tar = kvs.getRegion(tar_region_id); ASSERT_EQ( - tar->handleWriteRaftCmd({}, 66, 6, ctx.getTMTContext()), + tar->handleWriteRaftCmd({}, 66, 6, ctx.getTMTContext()).first, EngineStoreApplyRes::None); } { diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index 17745af4a21..7492de7dc13 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -21,7 +21,7 @@ namespace tests TEST_F(RegionKVStoreTest, KVStoreFailRecovery) try { - auto ctx = TiFlashTestEnv::getGlobalContext(); + auto & ctx = TiFlashTestEnv::getGlobalContext(); KVStore & kvs = getKVS(); { auto applied_index = 0; @@ -156,7 +156,7 @@ CATCH TEST_F(RegionKVStoreTest, KVStoreInvalidWrites) try { - auto ctx = TiFlashTestEnv::getGlobalContext(); + auto & ctx = TiFlashTestEnv::getGlobalContext(); { auto region_id = 1; { @@ -193,7 +193,7 @@ CATCH TEST_F(RegionKVStoreTest, KVStoreAdminCommands) try { - auto ctx = TiFlashTestEnv::getGlobalContext(); + auto & ctx = TiFlashTestEnv::getGlobalContext(); // CompactLog and passive persistence { KVStore & kvs = getKVS(); @@ -336,7 +336,7 @@ static void validate(KVStore & kvs, std::unique_ptr & proxy_ TEST_F(RegionKVStoreTest, KVStoreSnapshotV1) try { - auto ctx = TiFlashTestEnv::getGlobalContext(); + auto & ctx = TiFlashTestEnv::getGlobalContext(); { UInt64 region_id = 1; TableID table_id; @@ -510,7 +510,7 @@ CATCH TEST_F(RegionKVStoreTest, KVStoreSnapshotV2) try { - auto ctx = TiFlashTestEnv::getGlobalContext(); + auto & ctx = TiFlashTestEnv::getGlobalContext(); UInt64 region_id = 1; TableID table_id; { diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index be2f13550ae..78823d1824c 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -18,30 +18,128 @@ namespace DB { namespace tests { -TEST_F(RegionKVStoreTest, ProactiveFlush) +TEST_F(RegionKVStoreTest, ProactiveFlushLiveness) try { - auto ctx = TiFlashTestEnv::getGlobalContext(); + auto & ctx = TiFlashTestEnv::getGlobalContext(); + ctx.getSettingsRef().dt_segment_limit_rows = 1000000; + ctx.getSettingsRef().dt_segment_limit_size = 1000000; + ctx.getSettingsRef().dt_segment_delta_cache_limit_rows = 0; + ctx.getSettingsRef().dt_segment_delta_cache_limit_size = 0; UInt64 region_id = 1; + UInt64 region_id2 = 7; TableID table_id; KVStore & kvs = getKVS(); + ASSERT_EQ(&ctx.getTMTContext().getContext(), &ctx); + ASSERT_EQ(ctx.getTMTContext().getContext().getSettingsRef().dt_segment_limit_size, 1000000); + MockRaftStoreProxy::FailCond cond; { initStorages(); table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); - proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::nullopt); + HandleID end_index = 100; + HandleID mid_index = 50; + auto start = RecordKVFormat::genKey(table_id, 0); + auto end = RecordKVFormat::genKey(table_id, end_index); + proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::make_pair(start.toString(), end.toString())); + + auto source_region = kvs.getRegion(region_id); + auto old_epoch = source_region->mutMeta().getMetaRegion().region_epoch(); + auto && [request, response] = MockRaftStoreProxy::composeBatchSplit( + {region_id, region_id2}, + {{RecordKVFormat::genKey(table_id, mid_index), RecordKVFormat::genKey(table_id, end_index)}, + {RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, mid_index)}}, + old_epoch); + auto && [index2, term2] = proxy_instance->adminCommand(region_id, std::move(request), std::move(response)); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); + auto kvr1 = kvs.getRegion(region_id); + auto kvr2 = kvs.getRegion(region_id2); ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); + ctx.getTMTContext().getRegionTable().updateRegion(*kvr2); } { - auto kvr1 = kvs.getRegion(region_id); - ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); - auto & r1_range = kvr1->getRange()->comparableKeys(); - LOG_INFO(&Poco::Logger::get("!!!!"), "!!!!! r1 range {} {}", r1_range.first.toDebugString(), r1_range.second.toDebugString()); + // Manually flush. + // auto kvr1 = kvs.getRegion(region_id); + // ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); + // auto & r1_range = kvr1->getRange()->comparableKeys(); - auto keyrange = DM::RowKeyRange::newAll(false, 10); - kvs.compactLogByRowKeyRange(ctx.getTMTContext(), keyrange, DB::NullspaceID, table_id, false); + // auto keyrange = DM::RowKeyRange::newAll(false, 10); + // kvs.compactLogByRowKeyRange(ctx.getTMTContext(), keyrange, DB::NullspaceID, table_id, false); + } { + // A fg flush and a bg flush will not deadlock. + DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + std::shared_ptr> ai = std::make_shared>(); + ai->store(1); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); + auto f1 = [&]() { + auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); + auto k1 = RecordKVFormat::genKey(table_id, 60, 111); + // Trigger a forground flush on region_id + auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2, std::make_optional(true)); + }; + std::thread t1(f1); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + ai->store(2); + // Force bg flush. + auto f2 = [&]() { + auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); + auto k1 = RecordKVFormat::genKey(table_id, 5, 111); + // Trigger a forground flush on region_id2 + auto [index, term] = proxy_instance->rawWrite(region_id2, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index2, term2] = proxy_instance->rawWrite(region_id2, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index2, std::make_optional(false)); + }; + std::thread t2(f2); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + t1.join(); + t2.join(); } + return; + { + // Two fg flush will not deadlock. + DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + auto f1 = [&]() { + auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); + auto k1 = RecordKVFormat::genKey(table_id, 60, 111); + // Trigger a forground flush on region_id + auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); + }; + auto f2 = [&]() { + auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); + auto k1 = RecordKVFormat::genKey(table_id, 5, 111); + // Trigger a forground flush on region_id2 + auto [index, term] = proxy_instance->rawWrite(region_id2, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index2, term2] = proxy_instance->rawWrite(region_id2, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index2); + }; + std::thread t1(f1); + std::thread t2(f2); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + t1.join(); + t2.join(); + } + { + } +} +CATCH + +TEST_F(RegionKVStoreTest, ProactiveFlushRecover) +try +{ { + // Safe to abort between flushing regions. + } { + // Safe to abort between flushCache and persistRegion. } } CATCH diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index eec095f68de..e6c940227c9 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -51,6 +51,8 @@ namespace FailPoints { extern const char skip_check_segment_update[]; extern const char force_fail_in_flush_region_data[]; +extern const char proactive_flush_before_persist_region[]; +extern const char proactive_flush_force_set_type[]; } // namespace FailPoints namespace RegionBench @@ -91,6 +93,8 @@ class RegionKVStoreTest : public ::testing::Test kvstore->setStore(store); ASSERT_EQ(kvstore->getStoreID(), store.id()); } + + LOG_INFO(Logger::get("Test"), "Finished setup"); } void TearDown() override From 9cb9ec1c027908195cb3d4125db5354ef8ab123a Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 27 Jun 2023 16:32:15 +0800 Subject: [PATCH 18/66] support test of notify Signed-off-by: CalvinNeo --- dbms/src/Common/FailPoint.cpp | 1 - dbms/src/Debug/MockRaftStoreProxy.cpp | 31 +++++++++-- dbms/src/Debug/MockRaftStoreProxy.h | 1 + .../Storages/DeltaMerge/DeltaMergeStore.cpp | 24 +++------ .../DeltaMerge/DeltaMergeStore_InternalBg.cpp | 2 +- .../tests/gtest_dm_delta_merge_store.cpp | 6 ++- dbms/src/Storages/Transaction/KVStore.cpp | 51 ++++++++++++------- dbms/src/Storages/Transaction/KVStore.h | 2 +- dbms/src/Storages/Transaction/ProxyFFI.cpp | 5 +- dbms/src/Storages/Transaction/ProxyFFI.h | 2 +- .../Transaction/tests/gtest_kvstore.cpp | 5 ++ .../tests/gtest_proactive_flush.cpp | 25 ++++----- 12 files changed, 92 insertions(+), 63 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index c9454a31d0d..efa2c152472 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -43,7 +43,6 @@ namespace DB M(exception_before_dmfile_remove_encryption) \ M(exception_before_dmfile_remove_from_disk) \ M(force_triggle_background_merge_delta) \ - M(force_triggle_foreground_flush) \ M(exception_before_mpp_register_non_root_mpp_task) \ M(exception_before_mpp_register_tunnel_for_non_root_mpp_task) \ M(exception_during_mpp_register_tunnel_for_non_root_mpp_task) \ diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 133b54cd1b1..28f2ca87914 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -138,9 +138,20 @@ KVGetStatus fn_get_region_local_state(RaftStoreProxyPtr ptr, uint64_t region_id, return KVGetStatus::NotFound; } -void fn_notify_compact_log(RaftStoreProxyPtr, uint64_t, uint64_t, uint64_t, uint64_t) +void fn_notify_compact_log(RaftStoreProxyPtr ptr, uint64_t region_id, uint64_t compact_index, uint64_t compact_term, uint64_t applied_index) { - // Do nothing + // Update flushed applied_index and truncated state. + auto & x = as_ref(ptr); + auto region = x.getRegion(region_id); + ASSERT(region); + LOG_INFO(&Poco::Logger::get("!!!!!"), "!!!! fn_notify_compact_log {} commit index {} applied_index {} compact_index {} compact_term {}", region_id, region->getLatestCommitIndex(), applied_index, compact_index, compact_term); + ASSERT(region->getLatestCommitIndex() >= applied_index); + // `applied_index` in proxy's disk can be still LT applied_index here when fg flush. + // So we use commit_index here. + if (region) + { + region->updateTruncatedState(compact_index, compact_term); + } } TiFlashRaftProxyHelper MockRaftStoreProxy::SetRaftStoreProxyFFIHelper(RaftStoreProxyPtr proxy_ptr) @@ -182,17 +193,26 @@ void MockProxyRegion::updateAppliedIndex(uint64_t index) uint64_t MockProxyRegion::getLatestAppliedIndex() { - return this->getApply().applied_index(); + auto _ = genLockGuard(); + return this->apply.applied_index(); } uint64_t MockProxyRegion::getLatestCommitTerm() { - return this->getApply().commit_term(); + auto _ = genLockGuard(); + return this->apply.commit_term(); } uint64_t MockProxyRegion::getLatestCommitIndex() { - return this->getApply().commit_index(); + auto _ = genLockGuard(); + return this->apply.commit_index(); +} + +void MockProxyRegion::updateTruncatedState(uint64_t index, uint64_t term) +{ + this->apply.mutable_truncated_state()->set_index(index); + this->apply.mutable_truncated_state()->set_term(term); } void MockProxyRegion::updateCommitIndex(uint64_t index) @@ -741,6 +761,7 @@ void MockRaftStoreProxy::doApply( } // Proxy advance + // We currently consider a flush for every command for simplify. if (cond.type == MockRaftStoreProxy::FailCond::Type::BEFORE_PROXY_ADVANCE) return; region->updateAppliedIndex(index); diff --git a/dbms/src/Debug/MockRaftStoreProxy.h b/dbms/src/Debug/MockRaftStoreProxy.h index 69df61d85ab..05a1e1bfb7a 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.h +++ b/dbms/src/Debug/MockRaftStoreProxy.h @@ -35,6 +35,7 @@ struct MockProxyRegion : MutexLockWrap uint64_t getLatestCommitTerm(); uint64_t getLatestCommitIndex(); void updateCommitIndex(uint64_t index); + void updateTruncatedState(uint64_t index, uint64_t term); void setSate(raft_serverpb::RegionLocalState); explicit MockProxyRegion(uint64_t id); UniversalWriteBatch persistMeta(); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 62c87822398..70f1aa96b13 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -95,7 +95,6 @@ extern const char skip_check_segment_update[]; extern const char pause_when_writing_to_dt_store[]; extern const char pause_when_altering_dt_store[]; extern const char force_triggle_background_merge_delta[]; -extern const char force_triggle_foreground_flush[]; extern const char random_exception_after_dt_write_done[]; extern const char force_slow_page_storage_snapshot_release[]; extern const char exception_before_drop_segment[]; @@ -1412,31 +1411,20 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const && delta_rows - delta_last_try_place_delta_index_rows >= delta_cache_limit_rows); fiu_do_on(FailPoints::force_triggle_background_merge_delta, { should_background_merge_delta = true; }); - fiu_do_on(FailPoints::force_triggle_foreground_flush, { should_foreground_flush = true; }); fiu_do_on(FailPoints::proactive_flush_force_set_type, { + // | set bg bit | bg value bit | set fg bit | fg value bit| if (auto v = FailPointHelper::getFailPointVal(FailPoints::proactive_flush_force_set_type); v) { - auto set_kind = std::any_cast>>(v.value()); + auto set_kind = std::any_cast>>(v.value()); auto set_kind_int = set_kind->load(); - if (set_kind_int == 1) - { - LOG_INFO(log, "!!!! AAAAA 1"); - should_foreground_flush = true; - should_background_flush = false; - } - else if (set_kind_int == 2) - { - LOG_INFO(log, "!!!! AAAAA 2"); - should_foreground_flush = false; - should_background_flush = true; - } + if ((set_kind_int >> 1) & 1) + should_foreground_flush = set_kind_int & 1; + if ((set_kind_int >> 3) & 1) + should_background_flush = (set_kind_int >> 2) & 1; } }); - LOG_INFO(log, "!!!!! segment_limit_rows {} segment_limit_bytes {} delta_cache_limit_rows {} delta_cache_limit_bytes {}, {}. should_foreground_flush {} should_background_flush {}", segment_limit_rows, segment_limit_bytes, delta_cache_limit_rows, delta_cache_limit_bytes, StackTrace().toString(), should_foreground_flush, should_background_flush); - - auto try_add_background_task = [&](const BackgroundTask & task) { if (shutdown_called.load(std::memory_order_relaxed)) return; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index e6b8acd3a60..abbcc971127 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -927,7 +927,7 @@ void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const R auto & tmt = dm_context->db_context.getTMTContext(); auto & kv_store = tmt.getKVStore(); - kv_store->compactLogByRowKeyRange(tmt, range, keyspace_id, physical_table_id, is_background); + kv_store->proactiveFlushCacheAndRegion(tmt, range, keyspace_id, physical_table_id, is_background); } } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp index 53b7456a761..282f7116326 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp @@ -53,11 +53,11 @@ namespace FailPoints extern const char pause_before_dt_background_delta_merge[]; extern const char pause_until_dt_background_delta_merge[]; extern const char force_triggle_background_merge_delta[]; -extern const char force_triggle_foreground_flush[]; extern const char force_set_segment_ingest_packs_fail[]; extern const char segment_merge_after_ingest_packs[]; extern const char force_set_segment_physical_split[]; extern const char force_set_page_file_write_errno[]; +extern const char proactive_flush_force_set_type[]; } // namespace FailPoints namespace DM @@ -2644,7 +2644,9 @@ try { // write and triggle flush - FailPointHelper::enableFailPoint(FailPoints::force_triggle_foreground_flush); + std::shared_ptr> ai = std::make_shared>(); + ai->store(0b11); + FailPointHelper::enableFailPoint(FailPoints::proactive_flush_force_set_type, ai); Block block = DMTestEnv::prepareSimpleWriteBlock(num_rows_write, num_rows_write * 2, false); { diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index fc64a12949f..ea009804165 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -310,13 +310,13 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmdInner(const WriteCmdsView & cmds, /// Safety: /// This call is from Proxy's applying thread of this region, so: /// 1. No other thread can write from raft to this region even if we unlocked here. - /// 2. If `compactLogByRowKeyRange` causes a write stall, it will be forwarded to raft layer. + /// 2. If `proactiveFlushCacheAndRegion` causes a write stall, it will be forwarded to raft layer. if (write_result) { auto & inner = write_result.value(); for (auto it = inner.pending_flush_ranges.begin(); it != inner.pending_flush_ranges.end(); it++) { - compactLogByRowKeyRange(tmt, *it, inner.keyspace_id, inner.table_id, false); + proactiveFlushCacheAndRegion(tmt, *it, inner.keyspace_id, inner.table_id, false); } } return res; @@ -926,7 +926,7 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const // 1. store applied index and applied term, // 2. flush cache, // 3. notify regions to compact log and store fushed state with applied index/term before flushing cache. -void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) +void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) { if (is_background) { @@ -953,7 +953,7 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & if (unlikely(storage == nullptr)) { LOG_WARNING(log, - "compactLogByRowKeyRange can not get table for table id {}, ignored", + "proactiveFlushCacheAndRegion can not get table for table id {}, ignored", table_id); return; } @@ -962,6 +962,9 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & Stopwatch watch; LOG_INFO(log, "Start proactive flush region range [{},{}] [table_id={}] [keyspace_id={}] [is_background={}]", range.first.toDebugString(), range.second.toDebugString(), table_id, keyspace_id, is_background); + /// It finds r1,r2,r3 in the following case. + /// |------ range ------| + /// |--- r1 ---|--- r2 ---|--- r3 ---| std::unordered_map> region_compact_indexes; { auto task_lock = genTaskLock(); @@ -1002,22 +1005,25 @@ void KVStore::compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & for (const auto & region : region_compact_indexes) { auto region_rowkey_range = std::get<2>(region.second); - + auto region_id = region.first; + auto region_ptr = std::get<3>(region.second); if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) { // `region_rowkey_range` belongs to rowkey_range. // E.g. [0,9223372036854775807] belongs to [-9223372036854775808,9223372036854775807]. - // This segment has flushed, skip it. - LOG_DEBUG(log, "flushed segment of region {}", region.first); - continue; + // This segment has flushed. However, we still need to persist the region. + LOG_DEBUG(log, "segment of region {} flushed, [applied_index={}] [applied_term={}]", region.first, std::get<0>(region.second), std::get<1>(region.second)); + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); + } + else + { + LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); + // Both flushCache and persistRegion should be protected by region task lock. + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + storage->flushCache(tmt.getContext(), std::get<2>(region.second)); + persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } - auto region_id = region.first; - auto region_ptr = std::get<3>(region.second); - LOG_DEBUG(log, "flush extra segment of region {}, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); - // Both flushCache and persistRegion should be protected by region task lock. - auto region_task_lock = region_manager.genRegionTaskLock(region_id); - storage->flushCache(tmt.getContext(), std::get<2>(region.second)); - persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); } auto elapsed_coupled_flush = watch.elapsedMilliseconds(); watch.restart(); @@ -1057,10 +1063,17 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_fg).Increment(); } auto f = [&]() { - region->setFlushedState(compact_index, compact_term); - region->markCompactLog(); - region->cleanApproxMemCacheInfo(); - getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term); + // So proxy can get the current compact state of this region of TiFlash's side. + // TODO This is for `exec_compact_log`. Check out what it does exactly. + // TODO flushed state is never persisted, checkout if this will lead to a problem. + auto flush_state = region->getFlushedState(); + if (flush_state.applied_index < compact_index) + { + region->setFlushedState(compact_index, compact_term); + region->markCompactLog(); + region->cleanApproxMemCacheInfo(); + getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term, compact_index); + } }; if (lock_held) { diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 4de62bd4748..e0d5fa27140 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -185,7 +185,7 @@ class KVStore final : private boost::noncopyable FileUsageStatistics getFileUsageStatistics() const; - void compactLogByRowKeyRange(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background); + void proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background); void notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held = true); #ifndef DBMS_PUBLIC_GTEST private: diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 28d1c91535d..46684ab680b 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -855,10 +855,9 @@ raft_serverpb::RegionLocalState TiFlashRaftProxyHelper::getRegionLocalState(uint return state; } -void TiFlashRaftProxyHelper::notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term) const +void TiFlashRaftProxyHelper::notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term, uint64_t applied_index) const { - // TODO is it safe here to use compact_index as applied_index? - this->fn_notify_compact_log(this->proxy_ptr, region_id, compact_index, compact_term, compact_index); + this->fn_notify_compact_log(this->proxy_ptr, region_id, compact_index, compact_term, applied_index); } void HandleSafeTSUpdate(EngineStoreServerWrap * server, uint64_t region_id, uint64_t self_safe_ts, uint64_t leader_safe_ts) diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index f72fe22913f..f12ef8d8741 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -116,7 +116,7 @@ struct TiFlashRaftProxyHelper : RaftStoreProxyFFIHelper TimerTask makeTimerTask(uint64_t time_ms) const; bool pollTimerTask(TimerTask & task, RawVoidPtr waker = nullptr) const; raft_serverpb::RegionLocalState getRegionLocalState(uint64_t region_id) const; - void notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term) const; + void notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term, uint64_t applied_index) const; }; extern "C" { diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 5c7e9d0be1e..ada3ae39fbb 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -1182,6 +1182,8 @@ TEST_F(RegionKVStoreTest, RegionRange) auto res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); ASSERT_EQ(res.size(), 3); + auto res2 = region_index.findByRangeChecked(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); + ASSERT(std::holds_alternative(res2)); region_index.add(makeRegion(4, RecordKVFormat::genKey(1, 1), RecordKVFormat::genKey(1, 4))); @@ -1194,6 +1196,9 @@ TEST_F(RegionKVStoreTest, RegionRange) res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 1), TiKVKey(""))); ASSERT_EQ(res.size(), 3); + ASSERT_TRUE(res.find(1) != res.end()); + ASSERT_TRUE(res.find(2) != res.end()); + ASSERT_TRUE(res.find(4) != res.end()); res = region_index.findByRangeOverlap( RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 2), RecordKVFormat::genKey(1, 5))); diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index 78823d1824c..71f3c1096b8 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -57,19 +57,11 @@ try ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); ctx.getTMTContext().getRegionTable().updateRegion(*kvr2); } + std::shared_ptr> ai = std::make_shared>(); { - // Manually flush. - // auto kvr1 = kvs.getRegion(region_id); - // ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); - // auto & r1_range = kvr1->getRange()->comparableKeys(); - - // auto keyrange = DM::RowKeyRange::newAll(false, 10); - // kvs.compactLogByRowKeyRange(ctx.getTMTContext(), keyrange, DB::NullspaceID, table_id, false); - } { // A fg flush and a bg flush will not deadlock. DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); - std::shared_ptr> ai = std::make_shared>(); - ai->store(1); + ai->store(0b1011); DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); auto f1 = [&]() { auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); @@ -82,7 +74,7 @@ try }; std::thread t1(f1); std::this_thread::sleep_for(std::chrono::milliseconds(100)); - ai->store(2); + ai->store(0b1110); // Force bg flush. auto f2 = [&]() { auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); @@ -98,11 +90,15 @@ try DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); t1.join(); t2.join(); + ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); + // We can't assert for region_id2, since bg flush may be be finished. } - return; + kvs.setRegionCompactLogConfig(0, 0, 0); // Every notify will take effect. + LOG_INFO(&Poco::Logger::get("!!!!"), "!!!!! next"); { // Two fg flush will not deadlock. DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + ai->store(0b1011); auto f1 = [&]() { auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); auto k1 = RecordKVFormat::genKey(table_id, 60, 111); @@ -127,8 +123,13 @@ try DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); t1.join(); t2.join(); + ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); + ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id2)->getLatestCommitIndex()); } { + // An obsolete notification triggered by another region's flush shall not override. + kvs.notifyCompactLog(region_id, 1, 5, true, false); + ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); } } CATCH From 6aa3dc6b36be1644f2df896b437e359a6e21e178 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Wed, 28 Jun 2023 18:17:34 +0800 Subject: [PATCH 19/66] settings,tests,proxy updates Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- dbms/src/Common/FailPoint.cpp | 3 +- dbms/src/Debug/MockRaftStoreProxy.cpp | 41 ++++++++++++++--- dbms/src/Debug/MockRaftStoreProxy.h | 3 +- dbms/src/Storages/Transaction/KVStore.cpp | 44 ++++++++++++------- dbms/src/Storages/Transaction/KVStore.h | 7 +-- dbms/src/Storages/Transaction/ProxyFFI.cpp | 8 ++-- dbms/src/Storages/Transaction/ProxyFFI.h | 4 +- dbms/src/Storages/Transaction/TMTContext.cpp | 4 +- .../Transaction/tests/gtest_kvstore.cpp | 5 ++- .../tests/gtest_kvstore_fast_add_peer.cpp | 2 +- .../Transaction/tests/gtest_new_kvstore.cpp | 12 ++--- .../tests/gtest_proactive_flush.cpp | 43 ++++++++++++++++-- .../tests/gtest_read_index_worker.cpp | 1 + .../Transaction/tests/kvstore_helper.h | 1 + 15 files changed, 134 insertions(+), 46 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 152d78daab8..ffd2bb1c73f 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 152d78daab857ceef756d5b7f46a82660f32df02 +Subproject commit ffd2bb1c73f15626c63d776aa6f15872196756d2 diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index efa2c152472..7df0a42a549 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -121,7 +121,8 @@ namespace DB M(pause_when_altering_dt_store) \ M(pause_after_copr_streams_acquired) \ M(pause_query_init) \ - M(proactive_flush_before_persist_region) + M(proactive_flush_before_persist_region) \ + M(passive_flush_before_persist_region) #define APPLY_FOR_RANDOM_FAILPOINTS(M) \ diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 28f2ca87914..02844b05e5a 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -145,10 +145,8 @@ void fn_notify_compact_log(RaftStoreProxyPtr ptr, uint64_t region_id, uint64_t c auto region = x.getRegion(region_id); ASSERT(region); LOG_INFO(&Poco::Logger::get("!!!!!"), "!!!! fn_notify_compact_log {} commit index {} applied_index {} compact_index {} compact_term {}", region_id, region->getLatestCommitIndex(), applied_index, compact_index, compact_term); - ASSERT(region->getLatestCommitIndex() >= applied_index); - // `applied_index` in proxy's disk can be still LT applied_index here when fg flush. - // So we use commit_index here. - if (region) + // `applied_index` in proxy's disk can still be less than the `applied_index` here when fg flush. + if (region && region->getApply().truncated_state().index() < compact_index) { region->updateTruncatedState(compact_index, compact_term); } @@ -543,7 +541,11 @@ std::tuple MockRaftStoreProxy::rawWrite( } -std::tuple MockRaftStoreProxy::adminCommand(UInt64 region_id, raft_cmdpb::AdminRequest && request, raft_cmdpb::AdminResponse && response) +std::tuple MockRaftStoreProxy::adminCommand( + UInt64 region_id, + raft_cmdpb::AdminRequest && request, + raft_cmdpb::AdminResponse && response, + std::optional forced_index) { uint64_t index = 0; uint64_t term = 0; @@ -551,7 +553,8 @@ std::tuple MockRaftStoreProxy::adminCommand(UInt64 region_id auto region = getRegion(region_id); assert(region != nullptr); // We have a new entry. - index = region->getLatestCommitIndex() + 1; + index = forced_index.value_or(region->getLatestCommitIndex() + 1); + RUNTIME_CHECK(index > region->getLatestCommitIndex()); term = region->getLatestCommitTerm(); // The new entry is committed on Proxy's side. region->updateCommitIndex(index); @@ -582,6 +585,20 @@ std::tuple MockRaftStoreProxy::compactLog(UInt64 region_id, return adminCommand(region_id, std::move(request), std::move(response)); } +std::tuple MockRaftStoreProxy::composeCompactLog(MockProxyRegionPtr region, UInt64 compact_index) +{ + raft_cmdpb::AdminRequest request; + raft_cmdpb::AdminResponse response; + request.set_cmd_type(raft_cmdpb::AdminCmdType::CompactLog); + request.mutable_compact_log()->set_compact_index(compact_index); + // Find compact term, otherwise log must have been compacted. + if (region->commands.contains(compact_index)) + { + request.mutable_compact_log()->set_compact_term(region->commands[compact_index].term); + } + return std::make_tuple(request, response); +} + std::tuple MockRaftStoreProxy::composeChangePeer(metapb::Region && meta, std::vector peer_ids, bool is_v2) { raft_cmdpb::AdminRequest request; @@ -733,6 +750,18 @@ void MockRaftStoreProxy::doApply( } if (cmd.has_admin_request()) { + if (cmd.admin().cmd_type() == raft_cmdpb::AdminCmdType::CompactLog) + { + auto res = kvs.tryFlushRegionData(region_id, false, true, tmt, index, term, region->getApply().truncated_state().index(), region->getApply().truncated_state().term()); + auto compact_index = cmd.admin().request.compact_log().compact_index(); + auto compact_term = cmd.admin().request.compact_log().compact_term(); + if (!res) { + LOG_DEBUG(log, "mock pre exec reject"); + } else { + region->updateTruncatedState(compact_index, compact_term); + LOG_DEBUG(log, "mock pre exec success, update to {},{}", compact_index, compact_term); + } + } kvs.handleAdminRaftCmd(std::move(cmd.admin().request), std::move(cmd.admin().response), region_id, index, term, tmt); } diff --git a/dbms/src/Debug/MockRaftStoreProxy.h b/dbms/src/Debug/MockRaftStoreProxy.h index 05a1e1bfb7a..06278bd5e90 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.h +++ b/dbms/src/Debug/MockRaftStoreProxy.h @@ -234,8 +234,9 @@ struct MockRaftStoreProxy : MutexLockWrap /// Create a compactLog admin command, returns (index, term) of the admin command itself. std::tuple compactLog(UInt64 region_id, UInt64 compact_index); - std::tuple adminCommand(UInt64 region_id, raft_cmdpb::AdminRequest &&, raft_cmdpb::AdminResponse &&); + std::tuple adminCommand(UInt64 region_id, raft_cmdpb::AdminRequest &&, raft_cmdpb::AdminResponse &&, std::optional forced_index = std::nullopt); + static std::tuple composeCompactLog(MockProxyRegionPtr region, UInt64 compact_index); static std::tuple composeChangePeer(metapb::Region && meta, std::vector peer_ids, bool is_v2 = true); static std::tuple composePrepareMerge(metapb::Region && target, UInt64 min_index); static std::tuple composeCommitMerge(metapb::Region && source, UInt64 commit); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index ea009804165..4cc79ead3bf 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -46,6 +46,7 @@ namespace FailPoints { extern const char force_fail_in_flush_region_data[]; extern const char proactive_flush_before_persist_region[]; +extern const char passive_flush_before_persist_region[]; } // namespace FailPoints KVStore::KVStore(Context & context) @@ -55,6 +56,7 @@ KVStore::KVStore(Context & context) , region_compact_log_period(120) , region_compact_log_min_rows(40 * 1024) , region_compact_log_min_bytes(32 * 1024 * 1024) + , region_compact_log_gap(500) { // default config about compact-log: period 120s, rows 40k, bytes 32MB. } @@ -362,18 +364,20 @@ void KVStore::handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTas removeRegion(region_id, /* remove_data */ true, tmt.getRegionTable(), task_lock, region_manager.genRegionTaskLock(region_id)); } -void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes) +void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes, UInt64 gap) { region_compact_log_period = sec; region_compact_log_min_rows = rows; region_compact_log_min_bytes = bytes; + region_compact_log_gap = gap; LOG_INFO( log, - "threshold config: period {}, rows {}, bytes {}", + "threshold config: period {}, rows {}, bytes {}, gap {}", sec, rows, - bytes); + bytes, + gap); } void KVStore::persistRegion(const Region & region, std::optional region_task_lock, const char * caller) @@ -398,13 +402,14 @@ bool KVStore::needFlushRegionData(UInt64 region_id, TMTContext & tmt) auto region_task_lock = region_manager.genRegionTaskLock(region_id); const RegionPtr curr_region_ptr = getRegion(region_id); // TODO Should handle when curr_region_ptr is null. - return canFlushRegionDataImpl(curr_region_ptr, false, false, tmt, region_task_lock, 0, 0); + return canFlushRegionDataImpl(curr_region_ptr, false, false, tmt, region_task_lock, 0, 0, 0, 0); } -bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term) +bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term, uint64_t truncated_index, uint64_t truncated_term) { auto region_task_lock = region_manager.genRegionTaskLock(region_id); const RegionPtr curr_region_ptr = getRegion(region_id); + if (curr_region_ptr == nullptr) { /// If we can't find region here, we return true so proxy can trigger a CompactLog. @@ -414,6 +419,8 @@ bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_ LOG_WARNING(log, "region {} [index: {}, term {}], not exist when flushing, maybe have exec `RemoveNode` first", region_id, index, term); return true; } + + FAIL_POINT_PAUSE(FailPoints::passive_flush_before_persist_region); if (force_persist) { auto & curr_region = *curr_region_ptr; @@ -426,11 +433,11 @@ bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_ } else { - return canFlushRegionDataImpl(curr_region_ptr, false, try_until_succeed, tmt, region_task_lock, index, term); + return canFlushRegionDataImpl(curr_region_ptr, true, try_until_succeed, tmt, region_task_lock, index, term, truncated_index, truncated_term); } } -bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8, bool, TMTContext &, const RegionTaskLock &, UInt64, UInt64) +bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term, UInt64 truncated_index, UInt64 truncated_term) { if (curr_region_ptr == nullptr) { @@ -440,6 +447,13 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8, b auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); + auto gap = region_compact_log_gap.load(); + if(index > truncated_index + gap && flush_if_possible) { + // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. + LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} log gap {}", curr_region.toString(false), index, term, truncated_index, truncated_term, gap); + return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); + } + LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); return false; @@ -1066,14 +1080,14 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 // So proxy can get the current compact state of this region of TiFlash's side. // TODO This is for `exec_compact_log`. Check out what it does exactly. // TODO flushed state is never persisted, checkout if this will lead to a problem. - auto flush_state = region->getFlushedState(); - if (flush_state.applied_index < compact_index) - { - region->setFlushedState(compact_index, compact_term); - region->markCompactLog(); - region->cleanApproxMemCacheInfo(); - getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term, compact_index); - } + + // We will notify even if `flush_state.applied_index` is greater than compact_index, + // since this greater `applied_index` may not trigger a compact log. + // We will maintain the biggest on Proxy's side. + region->setFlushedState(compact_index, compact_term); + region->markCompactLog(); + region->cleanApproxMemCacheInfo(); + getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term, compact_index); }; if (lock_held) { diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index e0d5fa27140..6a007e597da 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -125,7 +125,7 @@ class KVStore final : private boost::noncopyable EngineStoreApplyRes handleWriteRaftCmdInner(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt, DM::WriteResult & write_result); bool needFlushRegionData(UInt64 region_id, TMTContext & tmt); - bool tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term); + bool tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term, uint64_t truncated_index, uint64_t truncated_term); /** * Only used in tests. In production we will call preHandleSnapshotToFiles + applyPreHandledSnapshot. @@ -144,7 +144,7 @@ class KVStore final : private boost::noncopyable void applyPreHandledSnapshot(const RegionPtrWrap &, TMTContext & tmt); void handleDestroy(UInt64 region_id, TMTContext & tmt); - void setRegionCompactLogConfig(UInt64, UInt64, UInt64); + void setRegionCompactLogConfig(UInt64, UInt64, UInt64, UInt64); EngineStoreApplyRes handleIngestSST(UInt64 region_id, SSTViewVec, UInt64 index, UInt64 term, TMTContext & tmt); RegionPtr genRegionPtr(metapb::Region && region, UInt64 peer_id, UInt64 index, UInt64 term); const TiFlashRaftProxyHelper * getProxyHelper() const { return proxy_helper; } @@ -256,7 +256,7 @@ class KVStore final : private boost::noncopyable /// Notice that if flush_if_possible is set to false, we only check if a flush is allowed by rowsize/size/interval. /// It will not check if a flush will eventually succeed. /// In other words, `canFlushRegionDataImpl(flush_if_possible=true)` can return false. - bool canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term); + bool canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term, UInt64 truncated_index, UInt64 truncated_term); bool forceFlushRegionDataImpl(Region & curr_region, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term); void persistRegion(const Region & region, std::optional region_task_lock, const char * caller); @@ -282,6 +282,7 @@ class KVStore final : private boost::noncopyable std::atomic region_compact_log_period; std::atomic region_compact_log_min_rows; std::atomic region_compact_log_min_bytes; + std::atomic region_compact_log_gap; mutable std::mutex bg_gc_region_data_mutex; std::list bg_gc_region_data; diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 46684ab680b..e8ccaf0b7af 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -145,12 +145,12 @@ uint8_t NeedFlushData(EngineStoreServerWrap * server, uint64_t region_id) } } -uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t flush_pattern, uint64_t index, uint64_t term) +uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t flush_pattern, uint64_t index, uint64_t term, uint64_t truncated_index, uint64_t truncated_term) { try { auto & kvstore = server->tmt->getKVStore(); - return kvstore->tryFlushRegionData(region_id, false, flush_pattern, *server->tmt, index, term); + return kvstore->tryFlushRegionData(region_id, false, flush_pattern, *server->tmt, index, term, truncated_index, truncated_term); } catch (...) { @@ -879,8 +879,10 @@ std::string_view buffToStrView(const BaseBuffView & buf) return std::string_view{buf.data, buf.len}; } -FlushedState GetFlushedState(EngineStoreServerWrap * server, uint64_t region_id) +FlushedState GetFlushedState(EngineStoreServerWrap * server, uint64_t region_id, uint8_t acquire_lock) { + // TODO + UNUSED(acquire_lock); auto & kvstore = server->tmt->getKVStore(); auto region_ptr = kvstore->getRegion(region_id); return region_ptr->getFlushedState(); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index f12ef8d8741..9dee0b5a5d6 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -133,7 +133,7 @@ uint8_t NeedFlushData(EngineStoreServerWrap * server, uint64_t region_id); // `flush_pattern` values: // 0: try, but can fail. // 1: try until succeed. -uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t flush_pattern, uint64_t index, uint64_t term); +uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t flush_pattern, uint64_t index, uint64_t term, uint64_t truncated_index, uint64_t truncated_term); RawCppPtr CreateWriteBatch(const EngineStoreServerWrap * dummy); void WriteBatchPutPage(RawVoidPtr ptr, BaseBuffView page_id, BaseBuffView value); void WriteBatchDelPage(RawVoidPtr ptr, BaseBuffView page_id); @@ -171,7 +171,7 @@ void SetStore(EngineStoreServerWrap *, BaseBuffView); void SetPBMsByBytes(MsgPBType type, RawVoidPtr ptr, BaseBuffView view); void HandleSafeTSUpdate(EngineStoreServerWrap * server, uint64_t region_id, uint64_t self_safe_ts, uint64_t leader_safe_ts); FastAddPeerRes FastAddPeer(EngineStoreServerWrap * server, uint64_t region_id, uint64_t new_peer_id); -FlushedState GetFlushedState(EngineStoreServerWrap * server, uint64_t region_id); +FlushedState GetFlushedState(EngineStoreServerWrap * server, uint64_t region_id, uint8_t acquire_lock); } inline EngineStoreServerHelper GetEngineStoreServerHelper( diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index 523af11353f..bdf14233cdf 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -291,6 +291,7 @@ void TMTContext::reloadConfig(const Poco::Util::AbstractConfiguration & config) static constexpr const char * COMPACT_LOG_MIN_PERIOD = "flash.compact_log_min_period"; static constexpr const char * COMPACT_LOG_MIN_ROWS = "flash.compact_log_min_rows"; static constexpr const char * COMPACT_LOG_MIN_BYTES = "flash.compact_log_min_bytes"; + static constexpr const char * COMPACT_LOG_MIN_GAP = "flash.compact_log_min_gap"; static constexpr const char * BATCH_READ_INDEX_TIMEOUT_MS = "flash.batch_read_index_timeout_ms"; static constexpr const char * WAIT_INDEX_TIMEOUT_MS = "flash.wait_index_timeout_ms"; static constexpr const char * WAIT_REGION_READY_TIMEOUT_SEC = "flash.wait_region_ready_timeout_sec"; @@ -299,7 +300,8 @@ void TMTContext::reloadConfig(const Poco::Util::AbstractConfiguration & config) // default config about compact-log: period 120s, rows 40k, bytes 32MB. getKVStore()->setRegionCompactLogConfig(std::max(config.getUInt64(COMPACT_LOG_MIN_PERIOD, 120), 1), std::max(config.getUInt64(COMPACT_LOG_MIN_ROWS, 40 * 1024), 1), - std::max(config.getUInt64(COMPACT_LOG_MIN_BYTES, 32 * 1024 * 1024), 1)); + std::max(config.getUInt64(COMPACT_LOG_MIN_BYTES, 32 * 1024 * 1024), 1), + std::max(config.getUInt64(COMPACT_LOG_MIN_GAP, 500), 1)); { batch_read_index_timeout_ms = config.getUInt64(BATCH_READ_INDEX_TIMEOUT_MS, DEFAULT_BATCH_READ_INDEX_TIMEOUT_MS); wait_index_timeout_ms = config.getUInt64(WAIT_INDEX_TIMEOUT_MS, DEFAULT_WAIT_INDEX_TIMEOUT_MS); diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index ada3ae39fbb..fb7cf8cc7cb 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -37,7 +37,7 @@ try raft_cmdpb::AdminResponse response; auto region = kvs.getRegion(1); region->markCompactLog(); - kvs.setRegionCompactLogConfig(100000, 1000, 1000); + kvs.setRegionCompactLogConfig(100000, 1000, 1000, 0); request.mutable_compact_log(); request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); // CompactLog always returns true now, even if we can't do a flush. @@ -45,7 +45,7 @@ try ASSERT_EQ(kvs.handleAdminRaftCmd(std::move(request), std::move(response), 1, 5, 1, ctx.getTMTContext()), EngineStoreApplyRes::Persist); // Filter - ASSERT_EQ(kvs.tryFlushRegionData(1, false, false, ctx.getTMTContext(), 0, 0), false); + ASSERT_EQ(kvs.tryFlushRegionData(1, false, false, ctx.getTMTContext(), 0, 0, 0, 0), false); } } CATCH @@ -153,6 +153,7 @@ TEST_F(RegionKVStoreTest, ReadIndex) // Test read index // Note `batchReadIndex` always returns latest committed index in our mock class. + // See `RawMockReadIndexTask::poll`. kvs.asyncRunReadIndexWorkers(); SCOPE_EXIT({ kvs.stopReadIndexWorkers(); diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp index b81ff68b808..ae6ff90a9db 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp @@ -177,7 +177,7 @@ void persistAfterWrite(Context & ctx, KVStore & kvs, std::unique_ptrwrite(std::move(wb), nullptr); // There shall be data to flush. ASSERT_EQ(kvs.needFlushRegionData(region_id, ctx.getTMTContext()), true); - ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, false, ctx.getTMTContext(), 0, 0), true); + ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, false, ctx.getTMTContext(), 0, 0, 0, 0), true); } TEST_F(RegionKVStoreTestFAP, RestoreRaftState) diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index 7492de7dc13..e4d85d7287a 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -215,11 +215,11 @@ try ASSERT_EQ(kvr1->appliedIndex(), applied_index + 1); kvr1->markCompactLog(); - kvs.setRegionCompactLogConfig(0, 0, 0); + kvs.setRegionCompactLogConfig(0, 0, 0, 0); auto [index2, term2] = proxy_instance->compactLog(region_id, index); // In tryFlushRegionData we will call handleWriteRaftCmd, which will already cause an advance. // Notice kvs is not tmt->getKVStore(), so we can't use the ProxyFFI version. - ASSERT_TRUE(kvs.tryFlushRegionData(region_id, false, true, ctx.getTMTContext(), index2, term)); + ASSERT_TRUE(kvs.tryFlushRegionData(region_id, false, true, ctx.getTMTContext(), index2, term, 0, 0)); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 2); ASSERT_EQ(kvr1->appliedIndex(), applied_index + 2); @@ -230,13 +230,13 @@ try ASSERT_EQ(kvs.needFlushRegionData(region_id, ctx.getTMTContext()), true); // If flush fails, and we don't insist a success. FailPointHelper::enableFailPoint(FailPoints::force_fail_in_flush_region_data); - ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, false, ctx.getTMTContext(), 0, 0), false); + ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, false, ctx.getTMTContext(), 0, 0, 0, 0), false); FailPointHelper::disableFailPoint(FailPoints::force_fail_in_flush_region_data); // Force flush until succeed only for testing. - ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, true, ctx.getTMTContext(), 0, 0), true); + ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, true, ctx.getTMTContext(), 0, 0, 0, 0), true); // Non existing region. // Flush and CompactLog will not panic. - ASSERT_EQ(kvs.tryFlushRegionData(1999, false, true, ctx.getTMTContext(), 0, 0), true); + ASSERT_EQ(kvs.tryFlushRegionData(1999, false, true, ctx.getTMTContext(), 0, 0, 0, 0), true); raft_cmdpb::AdminRequest request; raft_cmdpb::AdminResponse response; request.mutable_compact_log(); @@ -293,7 +293,7 @@ try ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), region_id, 25, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); { - kvs.setRegionCompactLogConfig(0, 0, 0); + kvs.setRegionCompactLogConfig(0, 0, 0, 0); request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); ASSERT_EQ(kvs.handleAdminRaftCmd(std::move(request), std::move(response2), region_id, 26, 6, ctx.getTMTContext()), EngineStoreApplyRes::Persist); } diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index 71f3c1096b8..1c387b7efe0 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -61,6 +61,7 @@ try { // A fg flush and a bg flush will not deadlock. DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::passive_flush_before_persist_region); ai->store(0b1011); DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); auto f1 = [&]() { @@ -88,13 +89,14 @@ try std::thread t2(f2); std::this_thread::sleep_for(std::chrono::milliseconds(100)); DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::passive_flush_before_persist_region); t1.join(); t2.join(); ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); // We can't assert for region_id2, since bg flush may be be finished. + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); } - kvs.setRegionCompactLogConfig(0, 0, 0); // Every notify will take effect. - LOG_INFO(&Poco::Logger::get("!!!!"), "!!!!! next"); + kvs.setRegionCompactLogConfig(0, 0, 0, 500); // Every notify will take effect. { // Two fg flush will not deadlock. DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); @@ -119,18 +121,50 @@ try }; std::thread t1(f1); std::thread t2(f2); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + std::this_thread::sleep_for(std::chrono::milliseconds(200)); DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); t1.join(); t2.join(); ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id2)->getLatestCommitIndex()); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); } { // An obsolete notification triggered by another region's flush shall not override. kvs.notifyCompactLog(region_id, 1, 5, true, false); ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); } + { + // Passive flush and fg proactive flush of the same region will not deadlock, since they must be executed by order in one thread. + // Passive flush and fg proactive flush will not deadlock. + ai->store(0b1011); // Force fg + DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::passive_flush_before_persist_region); + auto f1 = [&]() { + auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); + auto k1 = RecordKVFormat::genKey(table_id, 60, 111); + // Trigger a forground flush on region_id + auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2, std::make_optional(true)); + }; + auto f2 = [&]() { + auto r2 = proxy_instance->getRegion(region_id2); + auto && [request, response] = MockRaftStoreProxy::composeCompactLog(r2, 555); + auto && [index2, term] = proxy_instance->adminCommand(region_id2, std::move(request), std::move(response), 600); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index2); + }; + std::thread t1(f1); + std::thread t2(f2); + std::this_thread::sleep_for(std::chrono::milliseconds(200)); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::passive_flush_before_persist_region); + t2.join(); + ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), 555); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + t1.join(); + ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), 555); + } } CATCH @@ -139,7 +173,8 @@ try { { // Safe to abort between flushing regions. - } { + } + { // Safe to abort between flushCache and persistRegion. } } diff --git a/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp b/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp index 95e98fe9eaa..2302e1cc484 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp @@ -303,6 +303,7 @@ void ReadIndexTest::testNormal() { std::vector reqs; { + // One request of start_ts = 10 for every region. reqs.reserve(proxy_instance.size()); for (size_t i = 0; i < proxy_instance.size(); ++i) { diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index e6c940227c9..ba5538cce24 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -53,6 +53,7 @@ extern const char skip_check_segment_update[]; extern const char force_fail_in_flush_region_data[]; extern const char proactive_flush_before_persist_region[]; extern const char proactive_flush_force_set_type[]; +extern const char passive_flush_before_persist_region[]; } // namespace FailPoints namespace RegionBench From 6b7b2567ab10ae9d4e0c4f18e5e7feb3caad5835 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 3 Jul 2023 20:31:56 +0800 Subject: [PATCH 20/66] fmt Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- dbms/src/Common/FailPoint.cpp | 18 +++++---- dbms/src/Common/TiFlashMetrics.h | 2 + dbms/src/Debug/MockRaftStoreProxy.cpp | 7 +++- dbms/src/Storages/Transaction/KVStore.cpp | 35 +++++++++++------ .../tests/gtest_proactive_flush.cpp | 25 +++++++++--- .../Transaction/tests/kvstore_helper.h | 1 + dbms/src/Storages/tests/hit_log.cpp | 38 +++++++++---------- .../tests/remove_symlink_directory.cpp | 7 ++-- dbms/src/Storages/tests/seek_speed_test.cpp | 15 ++++---- 10 files changed, 91 insertions(+), 59 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index ffd2bb1c73f..6097c5f2f86 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit ffd2bb1c73f15626c63d776aa6f15872196756d2 +Subproject commit 6097c5f2f86a45072e737caacc8b2796ce5f3670 diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 7df0a42a549..988c32d595e 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -101,7 +101,9 @@ namespace DB M(force_stop_background_checkpoint_upload) \ M(skip_seek_before_read_dmfile) \ M(exception_after_large_write_exceed) \ - M(proactive_flush_force_set_type) + M(proactive_flush_force_set_type) \ + M(proactive_flush_between_persist_cache_and_region) \ + M(proactive_flush_between_persist_regions) #define APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) \ M(pause_with_alter_locks_acquired) \ @@ -114,13 +116,13 @@ namespace DB M(pause_after_copr_streams_acquired_once) \ M(pause_before_register_non_root_mpp_task) -#define APPLY_FOR_PAUSEABLE_FAILPOINTS(M) \ - M(pause_when_reading_from_dt_stream) \ - M(pause_when_writing_to_dt_store) \ - M(pause_when_ingesting_to_dt_store) \ - M(pause_when_altering_dt_store) \ - M(pause_after_copr_streams_acquired) \ - M(pause_query_init) \ +#define APPLY_FOR_PAUSEABLE_FAILPOINTS(M) \ + M(pause_when_reading_from_dt_stream) \ + M(pause_when_writing_to_dt_store) \ + M(pause_when_ingesting_to_dt_store) \ + M(pause_when_altering_dt_store) \ + M(pause_after_copr_streams_acquired) \ + M(pause_query_init) \ M(proactive_flush_before_persist_region) \ M(passive_flush_before_persist_region) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 92b6b4e8b36..35b904d84a9 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -261,6 +261,8 @@ namespace DB F(type_write, {{"type", "write"}}, ExpBuckets{0.001, 2, 30})) \ M(tiflash_raft_write_data_to_storage_duration_seconds, "Bucketed histogram of writting region into storage layer", Histogram, \ F(type_decode, {{"type", "decode"}}, ExpBuckets{0.0005, 2, 20}), F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ + M(tiflash_raft_raft_log_lag_count, "Bucketed histogram of applying write command Raft logs", Histogram, \ + F(type_compact_index, {{"type", "compact_index"}}, EqualWidthBuckets{0, 200, 5})) \ /* required by DBaaS */ \ M(tiflash_server_info, "Indicate the tiflash server info, and the value is the start timestamp (s).", Gauge, \ F(start_time, {"version", TiFlashBuildInfo::getReleaseVersion()}, {"hash", TiFlashBuildInfo::getGitHash()})) \ diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 02844b05e5a..8c877f774be 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -755,9 +755,12 @@ void MockRaftStoreProxy::doApply( auto res = kvs.tryFlushRegionData(region_id, false, true, tmt, index, term, region->getApply().truncated_state().index(), region->getApply().truncated_state().term()); auto compact_index = cmd.admin().request.compact_log().compact_index(); auto compact_term = cmd.admin().request.compact_log().compact_term(); - if (!res) { + if (!res) + { LOG_DEBUG(log, "mock pre exec reject"); - } else { + } + else + { region->updateTruncatedState(compact_index, compact_term); LOG_DEBUG(log, "mock pre exec success, update to {},{}", compact_index, compact_term); } diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 4cc79ead3bf..dcf6dbb51bb 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -47,6 +47,8 @@ namespace FailPoints extern const char force_fail_in_flush_region_data[]; extern const char proactive_flush_before_persist_region[]; extern const char passive_flush_before_persist_region[]; +extern const char proactive_flush_between_persist_cache_and_region[]; +extern const char proactive_flush_between_persist_regions[]; } // namespace FailPoints KVStore::KVStore(Context & context) @@ -447,15 +449,20 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); - auto gap = region_compact_log_gap.load(); - if(index > truncated_index + gap && flush_if_possible) { - // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. - LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} log gap {}", curr_region.toString(false), index, term, truncated_index, truncated_term, gap); - return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); - } - - LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); + auto current_gap = index - truncated_index; + GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); + auto gap_threshold = region_compact_log_gap.load(); + if (flush_if_possible) + { + if (index > truncated_index + gap_threshold) + { + // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. + LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap {}/{}", curr_region.toString(false), index, term, truncated_index, truncated_term, current_gap, gap_threshold); + return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); + } + LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); + } return false; } @@ -936,10 +943,6 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const return region_persister->getFileUsageStatistics(); } -// We need to get applied index before flushing cache, and can't hold region task lock when flush cache to avoid hang write cmd apply. -// 1. store applied index and applied term, -// 2. flush cache, -// 3. notify regions to compact log and store fushed state with applied index/term before flushing cache. void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) { if (is_background) @@ -1034,6 +1037,14 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan { LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); // Both flushCache and persistRegion should be protected by region task lock. + // We can avoid flushCache with a region lock held, if we save some meta info before flushing cache. + // Merely store applied_index is not enough, considering some cmds leads to modification of other meta data. + // After flushCache, we will persist region and notify Proxy with the previously stored meta info. + // However, this solution still involves region task lock in this function. + // Meanwhile, other write/admin cmds may be executed, they requires we acquire lock here: + // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. + // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. + auto region_task_lock = region_manager.genRegionTaskLock(region_id); storage->flushCache(tmt.getContext(), std::get<2>(region.second)); persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index 1c387b7efe0..b6e0dcf33e8 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -18,10 +18,11 @@ namespace DB { namespace tests { -TEST_F(RegionKVStoreTest, ProactiveFlushLiveness) -try + +std::tuple RegionKVStoreTest::prepareForProactiveFlushTest() { auto & ctx = TiFlashTestEnv::getGlobalContext(); + // Allow enough large segment size. ctx.getSettingsRef().dt_segment_limit_rows = 1000000; ctx.getSettingsRef().dt_segment_limit_size = 1000000; ctx.getSettingsRef().dt_segment_delta_cache_limit_rows = 0; @@ -30,8 +31,6 @@ try UInt64 region_id2 = 7; TableID table_id; KVStore & kvs = getKVS(); - ASSERT_EQ(&ctx.getTMTContext().getContext(), &ctx); - ASSERT_EQ(ctx.getTMTContext().getContext().getSettingsRef().dt_segment_limit_size, 1000000); MockRaftStoreProxy::FailCond cond; { initStorages(); @@ -57,6 +56,20 @@ try ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); ctx.getTMTContext().getRegionTable().updateRegion(*kvr2); } + return std::make_tuple(table_id, region_id, region_id2); +} + +TEST_F(RegionKVStoreTest, ProactiveFlushLiveness) +try +{ + auto & ctx = TiFlashTestEnv::getGlobalContext(); + auto tp = prepareForProactiveFlushTest(); + auto table_id = std::get<0>(tp); + auto region_id = std::get<1>(tp); + auto region_id2 = std::get<2>(tp); + MockRaftStoreProxy::FailCond cond; + KVStore & kvs = getKVS(); + std::shared_ptr> ai = std::make_shared>(); { // A fg flush and a bg flush will not deadlock. @@ -171,10 +184,10 @@ CATCH TEST_F(RegionKVStoreTest, ProactiveFlushRecover) try { + auto ctx = TiFlashTestEnv::getGlobalContext(); { // Safe to abort between flushing regions. - } - { + } { // Safe to abort between flushCache and persistRegion. } } diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index ba5538cce24..ecd577f71c5 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -141,6 +141,7 @@ class RegionKVStoreTest : public ::testing::Test } protected: + std::tuple prepareForProactiveFlushTest(); static void testRaftSplit(KVStore & kvs, TMTContext & tmt); static void testRaftMerge(KVStore & kvs, TMTContext & tmt); static void testRaftMergeRollback(KVStore & kvs, TMTContext & tmt); diff --git a/dbms/src/Storages/tests/hit_log.cpp b/dbms/src/Storages/tests/hit_log.cpp index c556fae3f80..d61fe655194 100644 --- a/dbms/src/Storages/tests/hit_log.cpp +++ b/dbms/src/Storages/tests/hit_log.cpp @@ -12,28 +12,24 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include -#include - -#include -#include - -#include -#include -#include -#include - -#include -#include #include #include +#include +#include #include - -#include +#include +#include +#include +#include +#include +#include +#include #include +#include -#include +#include +#include +#include using namespace DB; @@ -42,8 +38,7 @@ using namespace DB; int main(int argc, char ** argv) try { - NamesAndTypesList names_and_types_list - { + NamesAndTypesList names_and_types_list{ {"WatchID", std::make_shared()}, {"JavaEnable", std::make_shared()}, {"Title", std::make_shared()}, @@ -114,7 +109,10 @@ try /// create a hit log table StoragePtr table = StorageLog::create( - "./", "HitLog", ColumnsDescription{names_and_types_list}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE); + "./", + "HitLog", + ColumnsDescription{names_and_types_list}, + DEFAULT_MAX_COMPRESS_BLOCK_SIZE); table->startup(); /// create a description of how to read data from the tab separated dump diff --git a/dbms/src/Storages/tests/remove_symlink_directory.cpp b/dbms/src/Storages/tests/remove_symlink_directory.cpp index 07f1a13a433..b60598231ec 100644 --- a/dbms/src/Storages/tests/remove_symlink_directory.cpp +++ b/dbms/src/Storages/tests/remove_symlink_directory.cpp @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include +#include #include #include -#include +#include + +#include int main(int, char **) diff --git a/dbms/src/Storages/tests/seek_speed_test.cpp b/dbms/src/Storages/tests/seek_speed_test.cpp index b76f4fed1e3..69ee216cb3d 100644 --- a/dbms/src/Storages/tests/seek_speed_test.cpp +++ b/dbms/src/Storages/tests/seek_speed_test.cpp @@ -12,14 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include -#include #include -#include + +#include #include +#include #include -#include /** We test the hypothesis that skipping unnecessary parts of seek-forward never degrades overall read speed. * Before the measurements, it is desirable to discard disk cache: `echo 3 > /proc/sys/vm/drop_caches`. @@ -35,7 +36,7 @@ int main(int argc, const char ** argv) if (argc < 5 || argc > 6) { std::cerr << "Usage:\n" - << argv[0] << " file bytes_in_block min_skip_bytes max_skip_bytes [buffer_size]" << std::endl; + << argv[0] << " file bytes_in_block min_skip_bytes max_skip_bytes [buffer_size]" << std::endl; return 0; } @@ -72,11 +73,11 @@ int main(int argc, const char ** argv) } watch.stop(); - std::cout << checksum << std::endl; /// don't optimize + std::cout << checksum << std::endl; /// don't optimize std::cout << "Read " << bytes_read << " out of " << size << " bytes in " - << std::setprecision(4) << watch.elapsedSeconds() << " seconds (" - << bytes_read / watch.elapsedSeconds() / 1000000 << " MB/sec.)" << std::endl; + << std::setprecision(4) << watch.elapsedSeconds() << " seconds (" + << bytes_read / watch.elapsedSeconds() / 1000000 << " MB/sec.)" << std::endl; return 0; } From c4b31732cead590d5d846bdef4443703a6e71827 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Thu, 6 Jul 2023 10:38:07 +0800 Subject: [PATCH 21/66] a Signed-off-by: CalvinNeo --- dbms/src/Common/FailPoint.cpp | 18 ++-- dbms/src/Common/TiFlashMetrics.h | 2 + .../Storages/DeltaMerge/DeltaMergeStore.cpp | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 56 ++++++----- .../tests/gtest_proactive_flush.cpp | 95 +++++++++++++++---- .../Transaction/tests/kvstore_helper.h | 5 +- 6 files changed, 120 insertions(+), 58 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 988c32d595e..22af4b2581a 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -116,15 +116,15 @@ namespace DB M(pause_after_copr_streams_acquired_once) \ M(pause_before_register_non_root_mpp_task) -#define APPLY_FOR_PAUSEABLE_FAILPOINTS(M) \ - M(pause_when_reading_from_dt_stream) \ - M(pause_when_writing_to_dt_store) \ - M(pause_when_ingesting_to_dt_store) \ - M(pause_when_altering_dt_store) \ - M(pause_after_copr_streams_acquired) \ - M(pause_query_init) \ - M(proactive_flush_before_persist_region) \ - M(passive_flush_before_persist_region) +#define APPLY_FOR_PAUSEABLE_FAILPOINTS(M) \ + M(pause_when_reading_from_dt_stream) \ + M(pause_when_writing_to_dt_store) \ + M(pause_when_ingesting_to_dt_store) \ + M(pause_when_altering_dt_store) \ + M(pause_after_copr_streams_acquired) \ + M(pause_query_init) \ + M(pause_proactive_flush_before_persist_region) \ + M(pause_passive_flush_before_persist_region) #define APPLY_FOR_RANDOM_FAILPOINTS(M) \ diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 35b904d84a9..7490cbb1506 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -263,6 +263,8 @@ namespace DB F(type_decode, {{"type", "decode"}}, ExpBuckets{0.0005, 2, 20}), F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ M(tiflash_raft_raft_log_lag_count, "Bucketed histogram of applying write command Raft logs", Histogram, \ F(type_compact_index, {{"type", "compact_index"}}, EqualWidthBuckets{0, 200, 5})) \ + M(tiflash_raft_raft_events_count, "Raft event counter", Counter, \ + F(type_pre_exec_compact, {{"type", "pre_exec_compact"}})) \ /* required by DBaaS */ \ M(tiflash_server_info, "Indicate the tiflash server info, and the value is the start timestamp (s).", Gauge, \ F(start_time, {"version", TiFlashBuildInfo::getReleaseVersion()}, {"hash", TiFlashBuildInfo::getGitHash()})) \ diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 70f1aa96b13..43bba309f8b 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -99,7 +99,7 @@ extern const char random_exception_after_dt_write_done[]; extern const char force_slow_page_storage_snapshot_release[]; extern const char exception_before_drop_segment[]; extern const char exception_after_drop_segment[]; -extern const char proactive_flush_before_persist_region[]; +extern const char pause_proactive_flush_before_persist_region[]; extern const char proactive_flush_force_set_type[]; } // namespace FailPoints diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index dcf6dbb51bb..da44eb7b08d 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -45,8 +45,8 @@ extern const int TABLE_IS_DROPPED; namespace FailPoints { extern const char force_fail_in_flush_region_data[]; -extern const char proactive_flush_before_persist_region[]; -extern const char passive_flush_before_persist_region[]; +extern const char pause_proactive_flush_before_persist_region[]; +extern const char pause_passive_flush_before_persist_region[]; extern const char proactive_flush_between_persist_cache_and_region[]; extern const char proactive_flush_between_persist_regions[]; } // namespace FailPoints @@ -422,7 +422,7 @@ bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_ return true; } - FAIL_POINT_PAUSE(FailPoints::passive_flush_before_persist_region); + FAIL_POINT_PAUSE(FailPoints::pause_passive_flush_before_persist_region); if (force_persist) { auto & curr_region = *curr_region_ptr; @@ -450,6 +450,8 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); auto current_gap = index - truncated_index; + + GET_METRIC(tiflash_raft_raft_events_count, type_pre_exec_compact).Increment(1); GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); auto gap_threshold = region_compact_log_gap.load(); if (flush_if_possible) @@ -1016,43 +1018,45 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan region_compact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range, overlapped_region.second}; } } - FAIL_POINT_PAUSE(FailPoints::proactive_flush_before_persist_region); + FAIL_POINT_PAUSE(FailPoints::pause_proactive_flush_before_persist_region); // Flush all segments in the range of regions. // TODO: combine adjacent range to do one flush. + std::string reason = fmt::format("proactiveFlush{}", is_background ? "Bg" : "Fg"); for (const auto & region : region_compact_indexes) { auto region_rowkey_range = std::get<2>(region.second); auto region_id = region.first; auto region_ptr = std::get<3>(region.second); - if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) - { - // `region_rowkey_range` belongs to rowkey_range. - // E.g. [0,9223372036854775807] belongs to [-9223372036854775808,9223372036854775807]. - // This segment has flushed. However, we still need to persist the region. - LOG_DEBUG(log, "segment of region {} flushed, [applied_index={}] [applied_term={}]", region.first, std::get<0>(region.second), std::get<1>(region.second)); - auto region_task_lock = region_manager.genRegionTaskLock(region_id); - persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); - } - else { - LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); - // Both flushCache and persistRegion should be protected by region task lock. - // We can avoid flushCache with a region lock held, if we save some meta info before flushing cache. - // Merely store applied_index is not enough, considering some cmds leads to modification of other meta data. - // After flushCache, we will persist region and notify Proxy with the previously stored meta info. - // However, this solution still involves region task lock in this function. - // Meanwhile, other write/admin cmds may be executed, they requires we acquire lock here: - // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. - // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. - auto region_task_lock = region_manager.genRegionTaskLock(region_id); - storage->flushCache(tmt.getContext(), std::get<2>(region.second)); - persistRegion(*region_ptr, std::make_optional(®ion_task_lock), "triggerCompactLog"); + if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) + { + // `region_rowkey_range` belongs to rowkey_range. + // E.g. [0,9223372036854775807] belongs to [-9223372036854775808,9223372036854775807]. + // This segment has flushed. However, we still need to persist the region. + LOG_DEBUG(log, "segment of region {} flushed, [applied_index={}] [applied_term={}]", region.first, std::get<0>(region.second), std::get<1>(region.second)); + } + else + { + LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); + // Both flushCache and persistRegion should be protected by region task lock. + // We can avoid flushCache with a region lock held, if we save some meta info before flushing cache. + // Merely store applied_index is not enough, considering some cmds leads to modification of other meta data. + // After flushCache, we will persist region and notify Proxy with the previously stored meta info. + // However, this solution still involves region task lock in this function. + // Meanwhile, other write/admin cmds may be executed, they requires we acquire lock here: + // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. + // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. + storage->flushCache(tmt.getContext(), std::get<2>(region.second)); + } + fiu_do_on(FailPoints::proactive_flush_between_persist_cache_and_region, return;); + persistRegion(*region_ptr, std::make_optional(®ion_task_lock), reason.c_str()); } } auto elapsed_coupled_flush = watch.elapsedMilliseconds(); watch.restart(); + fiu_do_on(FailPoints::proactive_flush_between_persist_regions, return;); // forbid regions being removed. for (const auto & region : region_compact_indexes) { diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index b6e0dcf33e8..aeb62480e40 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -71,16 +71,16 @@ try KVStore & kvs = getKVS(); std::shared_ptr> ai = std::make_shared>(); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); { // A fg flush and a bg flush will not deadlock. - DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); - DB::FailPointHelper::enableFailPoint(DB::FailPoints::passive_flush_before_persist_region); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); ai->store(0b1011); - DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); auto f1 = [&]() { auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); auto k1 = RecordKVFormat::genKey(table_id, 60, 111); - // Trigger a forground flush on region_id + // Trigger a fg flush on region_id auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); @@ -93,7 +93,7 @@ try auto f2 = [&]() { auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); auto k1 = RecordKVFormat::genKey(table_id, 5, 111); - // Trigger a forground flush on region_id2 + // Trigger a fg flush on region_id2 auto [index, term] = proxy_instance->rawWrite(region_id2, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); auto [index2, term2] = proxy_instance->rawWrite(region_id2, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index); @@ -101,23 +101,22 @@ try }; std::thread t2(f2); std::this_thread::sleep_for(std::chrono::milliseconds(100)); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::passive_flush_before_persist_region); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); t1.join(); t2.join(); ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); // We can't assert for region_id2, since bg flush may be be finished. - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); } kvs.setRegionCompactLogConfig(0, 0, 0, 500); // Every notify will take effect. { // Two fg flush will not deadlock. - DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); ai->store(0b1011); auto f1 = [&]() { auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); auto k1 = RecordKVFormat::genKey(table_id, 60, 111); - // Trigger a forground flush on region_id + // Trigger a fg flush on region_id auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); @@ -126,7 +125,7 @@ try auto f2 = [&]() { auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); auto k1 = RecordKVFormat::genKey(table_id, 5, 111); - // Trigger a forground flush on region_id2 + // Trigger a fg flush on region_id2 auto [index, term] = proxy_instance->rawWrite(region_id2, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); auto [index2, term2] = proxy_instance->rawWrite(region_id2, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index); @@ -135,12 +134,11 @@ try std::thread t1(f1); std::thread t2(f2); std::this_thread::sleep_for(std::chrono::milliseconds(200)); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); t1.join(); t2.join(); ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id2)->getLatestCommitIndex()); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); } { // An obsolete notification triggered by another region's flush shall not override. @@ -151,12 +149,11 @@ try // Passive flush and fg proactive flush of the same region will not deadlock, since they must be executed by order in one thread. // Passive flush and fg proactive flush will not deadlock. ai->store(0b1011); // Force fg - DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); - DB::FailPointHelper::enableFailPoint(DB::FailPoints::passive_flush_before_persist_region); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); auto f1 = [&]() { auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); auto k1 = RecordKVFormat::genKey(table_id, 60, 111); - // Trigger a forground flush on region_id + // Trigger a fg flush on region_id auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); @@ -171,25 +168,83 @@ try std::thread t1(f1); std::thread t2(f2); std::this_thread::sleep_for(std::chrono::milliseconds(200)); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::passive_flush_before_persist_region); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); t2.join(); ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), 555); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_before_persist_region); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); t1.join(); ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), 555); } + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); } CATCH TEST_F(RegionKVStoreTest, ProactiveFlushRecover) try { - auto ctx = TiFlashTestEnv::getGlobalContext(); + auto & ctx = TiFlashTestEnv::getGlobalContext(); + std::shared_ptr> ai = std::make_shared>(); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); { // Safe to abort between flushing regions. - } { + auto tp = prepareForProactiveFlushTest(); + auto table_id = std::get<0>(tp); + auto region_id = std::get<1>(tp); + auto region_id2 = std::get<2>(tp); + MockRaftStoreProxy::FailCond cond; + + DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_between_persist_cache_and_region); + KVStore & kvs = getKVS(); + auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); + auto k2 = RecordKVFormat::genKey(table_id, 5, 111); + // Will not trigger a fg flush on region_id2 + auto [index2, term2] = proxy_instance->rawWrite(region_id2, {k2, k2}, {value_default, value_write}, {WriteCmdType::Put, WriteCmdType::Put}, {ColumnFamilyType::Default, ColumnFamilyType::Write}); + + // Abort before persistRegion, but with DM flushed. + cond.type = MockRaftStoreProxy::FailCond::Type::BEFORE_PROXY_ADVANCE; + ai->store(0b1011); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index2); + + // If reload here, the data is lost since we don't persistRegion. + // However, meta is not advanced either in KVStore or Proxy. + auto & kvs2 = reloadKVSFromDisk(); + auto kvr2 = kvs2.getRegion(region_id2); + auto r2 = proxy_instance->getRegion(region_id2); + ASSERT_EQ(kvr2->appliedIndex() + 1, index2); + ASSERT_EQ(r2->getLatestAppliedIndex() + 1, index2); + + cond.type = MockRaftStoreProxy::FailCond::Type::NORMAL; + ai->store(0b1010); + // No data lost. + proxy_instance->doApply(kvs2, ctx.getTMTContext(), cond, region_id2, index2); + auto [index22, term22] = proxy_instance->rawWrite(region_id2, {k2, k2}, {value_default, value_write}, {WriteCmdType::Put, WriteCmdType::Put}, {ColumnFamilyType::Default, ColumnFamilyType::Write}); + // There is no flush after write, so will throw when duplicate key. + EXPECT_THROW(proxy_instance->doApply(kvs2, ctx.getTMTContext(), cond, region_id2, index22), Exception); + + ai->store(0b1011); + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_between_persist_cache_and_region); + auto kvr1 = kvs2.getRegion(region_id); + auto r1 = proxy_instance->getRegion(region_id); + auto && [value_write1, value_default1] = proxy_instance->generateTiKVKeyValue(111, 999); + auto k1 = RecordKVFormat::genKey(table_id, 60, 111); + // Trigger a fg flush on region_id + auto [index1, term1] = proxy_instance->rawWrite(region_id, {k1, k1}, {value_default1, value_write1}, {WriteCmdType::Put, WriteCmdType::Put}, {ColumnFamilyType::Default, ColumnFamilyType::Write}); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index1); + + auto & kvs3 = reloadKVSFromDisk(); + { + auto kvr1 = kvs3.getRegion(region_id); + auto r1 = proxy_instance->getRegion(region_id); + ASSERT_EQ(kvr1->appliedIndex(), r1->getLatestAppliedIndex()); + auto kvr2 = kvs3.getRegion(region_id2); + auto r2 = proxy_instance->getRegion(region_id2); + ASSERT_EQ(kvr2->appliedIndex(), r2->getLatestAppliedIndex()); + } + } + { // Safe to abort between flushCache and persistRegion. } + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); } CATCH diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index ecd577f71c5..ca52cbea633 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -51,9 +51,10 @@ namespace FailPoints { extern const char skip_check_segment_update[]; extern const char force_fail_in_flush_region_data[]; -extern const char proactive_flush_before_persist_region[]; +extern const char pause_proactive_flush_before_persist_region[]; extern const char proactive_flush_force_set_type[]; -extern const char passive_flush_before_persist_region[]; +extern const char pause_passive_flush_before_persist_region[]; +extern const char proactive_flush_between_persist_cache_and_region[]; } // namespace FailPoints namespace RegionBench From c080671be788bd8debb278e44c785389fd6c3fa5 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Wed, 12 Jul 2023 15:10:46 +0800 Subject: [PATCH 22/66] remove get flush state when exec_compact_log Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- dbms/src/Common/TiFlashMetrics.h | 6 +- dbms/src/Debug/MockRaftStoreProxy.cpp | 37 ++++++-- dbms/src/Debug/MockRaftStoreProxy.h | 6 +- dbms/src/Storages/Transaction/KVStore.cpp | 70 ++++++++++---- .../Transaction/tests/gtest_kvstore.cpp | 2 +- .../Transaction/tests/gtest_new_kvstore.cpp | 54 +---------- .../tests/gtest_proactive_flush.cpp | 95 ++++++++++++++++++- 8 files changed, 189 insertions(+), 83 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index e04d1a81bde..90e8a744b5b 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit e04d1a81bde78d5cc1cf98b8b3970bfe35db0987 +Subproject commit 90e8a744b5b6c1399c60a5238310523c57672cf0 diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index c1a0a1f6ee8..08f723ea859 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -166,7 +166,11 @@ namespace DB F(type_seg_merge_bg_gc, {{"type", "seg_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20}), \ F(type_compact_log_bg, {{"type", "compact_log_bg"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_fg, {{"type", "compact_log_fg"}}, ExpBuckets{0.001, 2, 20})) \ + F(type_compact_log_fg, {{"type", "compact_log_fg"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_compact_log_fg_breakdown_kvs, {{"type", "compact_log_fg_breakdown_kvs"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_compact_log_fg_breakdown_dm, {{"type", "compact_log_fg_breakdown_dm"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_compact_log_bg_breakdown_kvs, {{"type", "compact_log_bg_breakdown_kvs"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_compact_log_bg_breakdown_dm, {{"type", "compact_log_bg_breakdown_dm"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_storage_throughput_bytes, "Calculate the throughput of tasks of storage in bytes", Gauge, /**/ \ F(type_write, {"type", "write"}), /**/ \ F(type_ingest, {"type", "ingest"}), /**/ \ diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 28257048485..7b1168fc127 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -149,7 +149,7 @@ void fn_notify_compact_log(RaftStoreProxyPtr ptr, uint64_t region_id, uint64_t c // `applied_index` in proxy's disk can still be less than the `applied_index` here when fg flush. if (region && region->getApply().truncated_state().index() < compact_index) { - region->updateTruncatedState(compact_index, compact_term); + region->tryUpdateTruncatedState(compact_index, compact_term); } } @@ -199,6 +199,19 @@ void MockProxyRegion::updateAppliedIndex(uint64_t index) this->apply.set_applied_index(index); } +void MockProxyRegion::persistAppliedIndex() +{ + // Assume persist after every advance for simplicity. + // So do nothing here. +} + +uint64_t MockProxyRegion::getPersistedAppliedIndex() +{ + // Assume persist after every advance for simplicity. + auto _ = genLockGuard(); + return this->apply.applied_index(); +} + uint64_t MockProxyRegion::getLatestAppliedIndex() { auto _ = genLockGuard(); @@ -217,10 +230,13 @@ uint64_t MockProxyRegion::getLatestCommitIndex() return this->apply.commit_index(); } -void MockProxyRegion::updateTruncatedState(uint64_t index, uint64_t term) +void MockProxyRegion::tryUpdateTruncatedState(uint64_t index, uint64_t term) { - this->apply.mutable_truncated_state()->set_index(index); - this->apply.mutable_truncated_state()->set_term(term); + if (index > this->apply.truncated_state().index()) + { + this->apply.mutable_truncated_state()->set_index(index); + this->apply.mutable_truncated_state()->set_term(term); + } } void MockProxyRegion::updateCommitIndex(uint64_t index) @@ -229,7 +245,7 @@ void MockProxyRegion::updateCommitIndex(uint64_t index) this->apply.set_commit_index(index); } -void MockProxyRegion::setSate(raft_serverpb::RegionLocalState s) +void MockProxyRegion::setState(raft_serverpb::RegionLocalState s) { auto _ = genLockGuard(); this->state = s; @@ -766,7 +782,7 @@ void MockRaftStoreProxy::doApply( } else { - region->updateTruncatedState(compact_index, compact_term); + region->tryUpdateTruncatedState(compact_index, compact_term); LOG_DEBUG(log, "mock pre exec success, update to {},{}", compact_index, compact_term); } } @@ -775,6 +791,8 @@ void MockRaftStoreProxy::doApply( if (cond.type == MockRaftStoreProxy::FailCond::Type::BEFORE_KVSTORE_ADVANCE) { + // We reset applied to old one. + // TODO persistRegion to cowork with restore. kvs.getRegion(region_id)->setApplied(old_applied, old_applied_term); return; } @@ -798,10 +816,15 @@ void MockRaftStoreProxy::doApply( } // Proxy advance - // We currently consider a flush for every command for simplify. + // In raftstore v1, applied_index in ApplyFsm is advanced before forward to TiFlash. + // However, it is after persisted applied state that ApplyFsm will notify raft to advance. + // So keeping a in-memory applied_index is ambiguious here. + // We currently consider a flush for every command for simplify, + // so in-memory applied_index equals to persisted applied_index. if (cond.type == MockRaftStoreProxy::FailCond::Type::BEFORE_PROXY_ADVANCE) return; region->updateAppliedIndex(index); + region->persistAppliedIndex(); } void MockRaftStoreProxy::replay( diff --git a/dbms/src/Debug/MockRaftStoreProxy.h b/dbms/src/Debug/MockRaftStoreProxy.h index cb3ec5267f2..db50c7dfc63 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.h +++ b/dbms/src/Debug/MockRaftStoreProxy.h @@ -30,13 +30,15 @@ struct MockProxyRegion : MutexLockWrap { raft_serverpb::RegionLocalState getState(); raft_serverpb::RaftApplyState getApply(); + void persistAppliedIndex(); void updateAppliedIndex(uint64_t index); + uint64_t getPersistedAppliedIndex(); uint64_t getLatestAppliedIndex(); uint64_t getLatestCommitTerm(); uint64_t getLatestCommitIndex(); void updateCommitIndex(uint64_t index); - void updateTruncatedState(uint64_t index, uint64_t term); - void setSate(raft_serverpb::RegionLocalState); + void tryUpdateTruncatedState(uint64_t index, uint64_t term); + void setState(raft_serverpb::RegionLocalState); explicit MockProxyRegion(uint64_t id); UniversalWriteBatch persistMeta(); void addPeer(uint64_t store_id, uint64_t peer_id, metapb::PeerRole role); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 83cb4774227..73cc22e651d 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -452,25 +452,33 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl } auto & curr_region = *curr_region_ptr; + bool can_flush = false; auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); + if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) + || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) + { + // if rows or bytes more than threshold, flush cache and persist mem data. + can_flush = true; + } + auto gap_threshold = region_compact_log_gap.load(); + if (index > truncated_index + gap_threshold) + { + can_flush = true; + } auto current_gap = index - truncated_index; GET_METRIC(tiflash_raft_raft_events_count, type_pre_exec_compact).Increment(1); GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); - auto gap_threshold = region_compact_log_gap.load(); - if (flush_if_possible) - { - if (index > truncated_index + gap_threshold) - { - // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. - LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap {}/{}", curr_region.toString(false), index, term, truncated_index, truncated_term, current_gap, gap_threshold); - return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); - } + LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}, gap {}/{}", curr_region.toString(false), rows, size_bytes, current_gap, gap_threshold); - LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); + if (can_flush && flush_if_possible) + { + // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. + LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap {}/{}", curr_region.toString(false), index, term, truncated_index, truncated_term, current_gap, gap_threshold); + return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); } - return false; + return can_flush; } bool KVStore::forceFlushRegionDataImpl(Region & curr_region, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term) @@ -974,14 +982,20 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan } Stopwatch general_watch; + UInt64 total_dm_flush_millis = 0; + UInt64 total_kvs_flush_millis = 0; SCOPE_EXIT({ if (is_background) { GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg).Observe(general_watch.elapsedSeconds()); + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg_breakdown_dm).Observe(total_dm_flush_millis / 1000.0); + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg_breakdown_kvs).Observe(total_kvs_flush_millis / 1000.0); } else { GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(general_watch.elapsedSeconds()); + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg_breakdown_dm).Observe(total_dm_flush_millis / 1000.0); + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg_breakdown_kvs).Observe(total_kvs_flush_millis / 1000.0); } }); @@ -1064,10 +1078,18 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan // Meanwhile, other write/admin cmds may be executed, they requires we acquire lock here: // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. + Stopwatch watch2; + watch2.restart(); storage->flushCache(tmt.getContext(), std::get<2>(region.second)); + total_dm_flush_millis += watch2.elapsedSecondsFromLastTime(); } fiu_do_on(FailPoints::proactive_flush_between_persist_cache_and_region, return;); - persistRegion(*region_ptr, std::make_optional(®ion_task_lock), reason.c_str()); + { + Stopwatch watch2; + watch2.restart(); + persistRegion(*region_ptr, std::make_optional(®ion_task_lock), reason.c_str()); + total_kvs_flush_millis += watch2.elapsedMilliseconds(); + } } } auto elapsed_coupled_flush = watch.elapsedMilliseconds(); @@ -1086,8 +1108,17 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan LOG_DEBUG(log, "Finished proactive flush region range [{},{}] of {} regions. [couple_flush={}] [notify_proxy={}] [table_id={}] [keyspace_id={}] [is_background={}]", range.first.toDebugString(), range.second.toDebugString(), region_compact_indexes.size(), elapsed_coupled_flush, elapsed_notify_proxy, table_id, keyspace_id, is_background); } -// The caller will guarantee that delta cache has been flushed. -// This function requires region cache being persisted before notifying. +/// The function will notify Proxy to schedule a CheckCompact task. When this task is handled, +/// When handling, it will try to update `applied_index` and `truncated_state`, and then persist. +/// The updated `truncated_state` will not exceed the recorded `max_compact_index`. +/// `max_compact_index` is updated by CompactLog, whether it is filtered. +/// Requirements: +/// 1. The caller will guarantee that delta cache has been flushed. +/// 1. Region cache being persisted before notifying. +/// The truncated_index is mono-increase since: +/// 1. Every non-filtered passive flush uses what is from the `CompactLog`, +/// and `entry_storage::first_index`/`compact_raft_log` will guard that. +/// 1. Every proactive flush uses the newest `applied_index`. void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held) { auto region = getRegion(region_id); @@ -1110,15 +1141,16 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 } auto f = [&]() { // So proxy can get the current compact state of this region of TiFlash's side. - // TODO This is for `exec_compact_log`. Check out what it does exactly. - // TODO flushed state is never persisted, checkout if this will lead to a problem. - // We will notify even if `flush_state.applied_index` is greater than compact_index, - // since this greater `applied_index` may not trigger a compact log. - // We will maintain the biggest on Proxy's side. + // TODO Passive `CompactLog`flush will not update this field, + // which make this not usable in `exec_compact_log`. Pending fix. + // TODO flushed state is never persisted, check if it will lead to a problem. region->setFlushedState(compact_index, compact_term); region->markCompactLog(); region->cleanApproxMemCacheInfo(); + // We will notify even if `flush_state.applied_index` is greater than `compact_index`, + // since this greater `applied_index` may not trigger a compact log. + // We will maintain the biggest on Proxy's side. getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term, compact_index); }; if (lock_held) diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 1f4bf10c6ac..39a4b4865d4 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -1091,7 +1091,7 @@ try { // A snapshot can set region to Tombstone. - proxy_instance->getRegion(22)->setSate(({ + proxy_instance->getRegion(22)->setState(({ raft_serverpb::RegionLocalState s; s.set_state(::raft_serverpb::PeerState::Tombstone); s; diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index 1de1720c03c..df49479fc1f 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -103,6 +103,8 @@ try proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index); ASSERT_EQ(kvr1->appliedIndex(), applied_index); + ASSERT_NE(kvr1->appliedIndex(), index); + // The persisted applied_index is `applied_index`. kvs.tryPersistRegion(region_id); } { @@ -196,57 +198,6 @@ TEST_F(RegionKVStoreTest, KVStoreAdminCommands) try { auto & ctx = TiFlashTestEnv::getGlobalContext(); - // CompactLog and passive persistence - { - KVStore & kvs = getKVS(); - UInt64 region_id = 1; - { - auto applied_index = 0; - proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::nullopt); - MockRaftStoreProxy::FailCond cond; - - auto kvr1 = kvs.getRegion(region_id); - auto r1 = proxy_instance->getRegion(region_id); - ASSERT_NE(r1, nullptr); - ASSERT_NE(kvr1, nullptr); - applied_index = r1->getLatestAppliedIndex(); - ASSERT_EQ(r1->getLatestAppliedIndex(), kvr1->appliedIndex()); - auto [index, term] = proxy_instance->normalWrite(region_id, {33}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); - ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 1); - ASSERT_EQ(kvr1->appliedIndex(), applied_index + 1); - - kvr1->markCompactLog(); - kvs.setRegionCompactLogConfig(0, 0, 0, 0); - auto && [request, response] = MockRaftStoreProxy::composeCompactLog(r1, index); - auto && [index2, term2] = proxy_instance->adminCommand(region_id, std::move(request), std::move(response)); - // In tryFlushRegionData we will call handleWriteRaftCmd, which will already cause an advance. - // Notice kvs is not tmt->getKVStore(), so we can't use the ProxyFFI version. - ASSERT_TRUE(kvs.tryFlushRegionData(region_id, false, true, ctx.getTMTContext(), index2, term, 0, 0)); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); - ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 2); - ASSERT_EQ(kvr1->appliedIndex(), applied_index + 2); - } - { - proxy_instance->normalWrite(region_id, {34}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); - // There shall be data to flush. - ASSERT_EQ(kvs.needFlushRegionData(region_id, ctx.getTMTContext()), true); - // If flush fails, and we don't insist a success. - FailPointHelper::enableFailPoint(FailPoints::force_fail_in_flush_region_data); - ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, false, ctx.getTMTContext(), 0, 0, 0, 0), false); - FailPointHelper::disableFailPoint(FailPoints::force_fail_in_flush_region_data); - // Force flush until succeed only for testing. - ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, true, ctx.getTMTContext(), 0, 0, 0, 0), true); - // Non existing region. - // Flush and CompactLog will not panic. - ASSERT_EQ(kvs.tryFlushRegionData(1999, false, true, ctx.getTMTContext(), 0, 0, 0, 0), true); - raft_cmdpb::AdminRequest request; - raft_cmdpb::AdminResponse response; - request.mutable_compact_log(); - request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 1999, 22, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound); - } - } { KVStore & kvs = getKVS(); UInt64 region_id = 2; @@ -835,6 +786,7 @@ try auto kvr1 = kvs.getRegion(region_id); auto r1 = proxy_instance->getRegion(region_id); + // Trigger a row2col. auto && [req, res] = MockRaftStoreProxy::composeCompactLog(r1, 10); proxy_instance->adminCommand(region_id, std::move(req), std::move(res), 20); EXPECT_THROW(proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, 20), Exception); diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index aeb62480e40..88e22d0b98f 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -19,6 +19,65 @@ namespace DB namespace tests { + +TEST_F(RegionKVStoreTest, KVStorePassivePersistence) +try +{ + auto & ctx = TiFlashTestEnv::getGlobalContext(); + // CompactLog and passive persistence + { + KVStore & kvs = getKVS(); + UInt64 region_id = 1; + { + auto applied_index = 0; + proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::nullopt); + MockRaftStoreProxy::FailCond cond; + + auto kvr1 = kvs.getRegion(region_id); + auto r1 = proxy_instance->getRegion(region_id); + ASSERT_NE(r1, nullptr); + ASSERT_NE(kvr1, nullptr); + applied_index = r1->getLatestAppliedIndex(); + ASSERT_EQ(r1->getLatestAppliedIndex(), kvr1->appliedIndex()); + auto [index, term] = proxy_instance->normalWrite(region_id, {33}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); + ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 1); + ASSERT_EQ(kvr1->appliedIndex(), applied_index + 1); + + kvr1->markCompactLog(); + kvs.setRegionCompactLogConfig(0, 0, 0, 0); + auto && [request, response] = MockRaftStoreProxy::composeCompactLog(r1, index); + auto && [index2, term2] = proxy_instance->adminCommand(region_id, std::move(request), std::move(response)); + // In tryFlushRegionData we will call handleWriteRaftCmd, which will already cause an advance. + // Notice kvs is not tmt->getKVStore(), so we can't use the ProxyFFI version. + ASSERT_TRUE(kvs.tryFlushRegionData(region_id, false, true, ctx.getTMTContext(), index2, term, 0, 0)); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); + ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 2); + ASSERT_EQ(kvr1->appliedIndex(), applied_index + 2); + } + { + proxy_instance->normalWrite(region_id, {34}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + // There shall be data to flush. + ASSERT_EQ(kvs.needFlushRegionData(region_id, ctx.getTMTContext()), true); + // If flush fails, and we don't insist a success. + FailPointHelper::enableFailPoint(FailPoints::force_fail_in_flush_region_data); + ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, false, ctx.getTMTContext(), 0, 0, 0, 0), false); + FailPointHelper::disableFailPoint(FailPoints::force_fail_in_flush_region_data); + // Force flush until succeed only for testing. + ASSERT_EQ(kvs.tryFlushRegionData(region_id, false, true, ctx.getTMTContext(), 0, 0, 0, 0), true); + // Non existing region. + // Flush and CompactLog will not panic. + ASSERT_EQ(kvs.tryFlushRegionData(1999, false, true, ctx.getTMTContext(), 0, 0, 0, 0), true); + raft_cmdpb::AdminRequest request; + raft_cmdpb::AdminResponse response; + request.mutable_compact_log(); + request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); + ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 1999, 22, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound); + } + } +} +CATCH + std::tuple RegionKVStoreTest::prepareForProactiveFlushTest() { auto & ctx = TiFlashTestEnv::getGlobalContext(); @@ -59,6 +118,39 @@ std::tuple RegionKVStoreTest::prepareForProactiveF return std::make_tuple(table_id, region_id, region_id2); } +TEST_F(RegionKVStoreTest, ProactiveFlushConsistency) +try +{ + auto & ctx = TiFlashTestEnv::getGlobalContext(); + auto tp = prepareForProactiveFlushTest(); + // auto table_id = std::get<0>(tp); + auto region_id = std::get<1>(tp); + // auto region_id2 = std::get<2>(tp); + MockRaftStoreProxy::FailCond cond; + KVStore & kvs = getKVS(); + + std::shared_ptr> ai = std::make_shared>(); + DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); + ai->store(0b0000); + + { + // Newer passive and older proactive. + auto kvr1 = kvs.getRegion(region_id); + auto r1 = proxy_instance->getRegion(region_id); + uint64_t compact_index = 10; + auto && [request, response] = MockRaftStoreProxy::composeCompactLog(r1, compact_index); + auto && [index1, term] = proxy_instance->adminCommand(region_id, std::move(request), std::move(response), 11); + kvs.setRegionCompactLogConfig(0, 0, 0, 500); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index1); + UNUSED(term); + kvs.notifyCompactLog(region_id, 1, 5, false, false); + ASSERT_EQ(r1->getApply().truncated_state().index(), compact_index); + } + + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); +} +CATCH + TEST_F(RegionKVStoreTest, ProactiveFlushLiveness) try { @@ -146,7 +238,8 @@ try ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); } { - // Passive flush and fg proactive flush of the same region will not deadlock, since they must be executed by order in one thread. + // Passive flush and fg proactive flush of the same region will not deadlock, + // since they must be executed by order in one thread. // Passive flush and fg proactive flush will not deadlock. ai->store(0b1011); // Force fg DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); From 3b93e12330eb7b7cb78368cc6894495d2baacae3 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Thu, 20 Jul 2023 17:10:41 +0800 Subject: [PATCH 23/66] metrics: part1 Signed-off-by: CalvinNeo --- dbms/src/Common/TiFlashMetrics.h | 37 ++++++++----- .../Storages/Transaction/ApplySnapshot.cpp | 6 +- dbms/src/Storages/Transaction/KVStore.cpp | 55 ++++++++++++++----- dbms/src/Storages/Transaction/KVStore.h | 28 +++++++++- 4 files changed, 94 insertions(+), 32 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 08f723ea859..dcc498f92cf 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -125,12 +125,6 @@ namespace DB M(tiflash_schema_apply_duration_seconds, "Bucketed histogram of ddl apply duration", Histogram, \ F(type_sync_schema_apply_duration, {{"type", "sync_schema_duration"}}, ExpBuckets{0.001, 2, 20}), \ F(type_sync_table_schema_apply_duration, {{"type", "sync_table_schema_duration"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_raft_read_index_count, "Total number of raft read index", Counter) \ - M(tiflash_stale_read_count, "Total number of stale read", Counter) \ - M(tiflash_raft_read_index_duration_seconds, "Bucketed histogram of raft read index duration", Histogram, \ - F(type_raft_read_index_duration, {{"type", "tmt_raft_read_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_raft_wait_index_duration_seconds, "Bucketed histogram of raft wait index duration", Histogram, \ - F(type_raft_wait_index_duration, {{"type", "tmt_raft_wait_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_syncing_data_freshness, "The freshness of tiflash data with tikv data", Histogram, \ F(type_syncing_data_freshness, {{"type", "data_freshness"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_storage_read_tasks_count, "Total number of storage engine read tasks", Counter) \ @@ -152,7 +146,9 @@ namespace DB F(type_compact_log_segment_bg, {"type", "compact_log_segment_bg"}), \ F(type_compact_log_segment_fg, {"type", "compact_log_segment_fg"}), \ F(type_compact_log_region_bg, {"type", "compact_log_region_bg"}), \ - F(type_compact_log_region_fg, {"type", "compact_log_region_fg"})) \ + F(type_compact_log_region_fg, {"type", "compact_log_region_fg"}), \ + F(type_compact_log_segment_bg_breakdown_kvs, {"type", "compact_log_segment_bg_breakdown_kvs"}), \ + F(type_compact_log_segment_fg_breakdown_kvs, {"type", "compact_log_segment_fg_breakdown_kvs"})) \ M(tiflash_storage_subtask_duration_seconds, "Bucketed histogram of storage's sub task duration", Histogram, \ F(type_delta_merge_bg, {{"type", "delta_merge_bg"}}, ExpBuckets{0.001, 2, 20}), \ F(type_delta_merge_bg_gc, {{"type", "delta_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ @@ -263,20 +259,35 @@ namespace DB F(type_apply_snapshot_predecode_sst2dt, {{"type", "snapshot_predecode_sst2dt"}}, ExpBuckets{0.05, 2, 10}), \ F(type_apply_snapshot_predecode_upload, {{"type", "snapshot_predecode_upload"}}, ExpBuckets{0.05, 2, 10}), \ F(type_apply_snapshot_flush, {{"type", "snapshot_flush"}}, ExpBuckets{0.05, 2, 10})) \ - M(tiflash_raft_process_keys, "Total number of keys processed in some types of Raft commands", Counter, \ - F(type_apply_snapshot, {"type", "apply_snapshot"}), F(type_ingest_sst, {"type", "ingest_sst"})) \ - M(tiflash_raft_apply_write_command_duration_seconds, "Bucketed histogram of applying write command Raft logs", Histogram, \ + M(tiflash_raft_apply_write_command_duration_seconds, "Bucketed histogram of applying write command Raft logs", \ + Histogram, /* like tiflash_raft_command_duration_seconds but are smaller tasks */ \ F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20}), \ F(type_admin, {{"type", "admin"}}, ExpBuckets{0.0005, 2, 20}), \ F(type_flush_region, {{"type", "flush_region"}}, ExpBuckets{0.0005, 2, 20})) \ + M(tiflash_raft_process_keys, "Total number of keys processed in some types of Raft commands", Counter, \ + F(type_apply_snapshot, {"type", "apply_snapshot"}), F(type_ingest_sst, {"type", "ingest_sst"})) \ M(tiflash_raft_upstream_latency, "The latency that tikv sends raft log to tiflash.", Histogram, \ F(type_write, {{"type", "write"}}, ExpBuckets{0.001, 2, 30})) \ M(tiflash_raft_write_data_to_storage_duration_seconds, "Bucketed histogram of writting region into storage layer", Histogram, \ - F(type_decode, {{"type", "decode"}}, ExpBuckets{0.0005, 2, 20}), F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ - M(tiflash_raft_raft_log_lag_count, "Bucketed histogram of applying write command Raft logs", Histogram, \ + F(type_decode, {{"type", "decode"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ + M(tiflash_raft_raft_log_lag_count, "Bucketed histogram raft index lag", Histogram, \ F(type_compact_index, {{"type", "compact_index"}}, EqualWidthBuckets{0, 200, 5})) \ M(tiflash_raft_raft_events_count, "Raft event counter", Counter, \ - F(type_pre_exec_compact, {{"type", "pre_exec_compact"}})) \ + F(type_pre_exec_compact, {{"type", "pre_exec_compact"}}), \ + F(type_flush_apply_snapshot, {{"type", "flush_apply_snapshot"}}), \ + F(type_flush_ingest_sst, {{"type", "flush_ingest_sst"}}), \ + F(type_flush_useless_admin, {{"type", "flush_useless_admin"}}), \ + F(type_flush_useful_admin, {{"type", "flush_useful_admin"}}), \ + F(type_flush_passive, {{"type", "flush_passive"}}), \ + F(type_flush_proactive, {{"type", "flush_proactive"}}), \ + F(type_exec_compact, {{"type", "exec_compact"}})) \ + M(tiflash_raft_read_index_count, "Total number of raft read index", Counter) \ + M(tiflash_stale_read_count, "Total number of stale read", Counter) \ + M(tiflash_raft_read_index_duration_seconds, "Bucketed histogram of raft read index duration", Histogram, \ + F(type_raft_read_index_duration, {{"type", "tmt_raft_read_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_raft_wait_index_duration_seconds, "Bucketed histogram of raft wait index duration", Histogram, \ + F(type_raft_wait_index_duration, {{"type", "tmt_raft_wait_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ /* required by DBaaS */ \ M(tiflash_server_info, "Indicate the tiflash server info, and the value is the start timestamp (s).", Gauge, \ F(start_time, {"version", TiFlashBuildInfo::getReleaseVersion()}, {"hash", TiFlashBuildInfo::getGitHash()})) \ diff --git a/dbms/src/Storages/Transaction/ApplySnapshot.cpp b/dbms/src/Storages/Transaction/ApplySnapshot.cpp index 10697ef0cf5..99aa53305cd 100644 --- a/dbms/src/Storages/Transaction/ApplySnapshot.cpp +++ b/dbms/src/Storages/Transaction/ApplySnapshot.cpp @@ -87,7 +87,7 @@ void KVStore::checkAndApplyPreHandledSnapshot(const RegionPtrWrap & new_region, old_region->setStateApplying(); tmt.getRegionTable().tryWriteBlockByRegionAndFlush(old_region, false); tryFlushRegionCacheInStorage(tmt, *old_region, log); - persistRegion(*old_region, ®ion_lock, "save previous region before apply"); + persistRegion(*old_region, ®ion_lock, PersistRegionReason::ApplySnapshotPrevRegion, ""); } } @@ -277,7 +277,7 @@ void KVStore::onSnapshot(const RegionPtrWrap & new_region_wrap, RegionPtr old_re manage_lock.index.add(new_region); } - persistRegion(*new_region, ®ion_lock, "save current region after apply"); + persistRegion(*new_region, ®ion_lock, PersistRegionReason::ApplySnapshotCurRegion, ""); tmt.getRegionTable().shrinkRegionRange(*new_region); } @@ -562,7 +562,7 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec { // We always try to flush dm cache and region if possible for every IngestSST, // in order to have the raft log truncated and sst deleted. - persistRegion(*region, ®ion_task_lock, __FUNCTION__); + persistRegion(*region, ®ion_task_lock, PersistRegionReason::IngestSst, ""); return EngineStoreApplyRes::Persist; } } diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 73cc22e651d..1bffee9dc48 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -186,7 +186,7 @@ void KVStore::tryPersistRegion(RegionID region_id) auto region = getRegion(region_id); if (region) { - persistRegion(*region, std::nullopt, ""); + persistRegion(*region, std::nullopt, PersistRegionReason::Debug, ""); } } @@ -387,11 +387,13 @@ void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes, U gap); } -void KVStore::persistRegion(const Region & region, std::optional region_task_lock, const char * caller) +void KVStore::persistRegion(const Region & region, std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg) { RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); if (region_task_lock.has_value()) { + auto reason_id = magic_enum::enum_underlying(reason); + std::string caller = fmt::format("{} {}", PersistRegionReasonMap[reason_id], extra_msg); LOG_INFO(log, "Start to persist {}, cache size: {} bytes for `{}`", region.toString(true), region.dataSize(), caller); region_persister->persist(region, *region_task_lock.value()); LOG_DEBUG(log, "Persist {} done", region.toString(false)); @@ -402,6 +404,30 @@ void KVStore::persistRegion(const Region & region, std::optionalpersist(region); LOG_INFO(log, "After persisted {}, cache {} bytes", region.toString(false), region.dataSize()); } + switch (reason) + { + case PersistRegionReason::UselessAdminCommand: + GET_METRIC(tiflash_raft_raft_events_count, type_flush_useless_admin).Increment(1); + break; + case PersistRegionReason::AdminCommand: + GET_METRIC(tiflash_raft_raft_events_count, type_flush_useful_admin).Increment(1); + break; + case PersistRegionReason::Flush: + GET_METRIC(tiflash_raft_raft_events_count, type_flush_passive).Increment(1); + break; + case PersistRegionReason::ProactiveFlush: + GET_METRIC(tiflash_raft_raft_events_count, type_flush_proactive).Increment(1); + break; + case PersistRegionReason::ApplySnapshotPrevRegion: + case PersistRegionReason::ApplySnapshotCurRegion: + GET_METRIC(tiflash_raft_raft_events_count, type_flush_apply_snapshot).Increment(1); + break; + case PersistRegionReason::IngestSst: + GET_METRIC(tiflash_raft_raft_events_count, type_flush_ingest_sst).Increment(1); + break; + default: + break; + } } bool KVStore::needFlushRegionData(UInt64 region_id, TMTContext & tmt) @@ -474,6 +500,7 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl if (can_flush && flush_if_possible) { + GET_METRIC(tiflash_raft_raft_events_count, type_exec_compact).Increment(1); // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap {}/{}", curr_region.toString(false), index, term, truncated_index, truncated_term, current_gap, gap_threshold); return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); @@ -491,7 +518,7 @@ bool KVStore::forceFlushRegionDataImpl(Region & curr_region, bool try_until_succ } if (tryFlushRegionCacheInStorage(tmt, curr_region, log, try_until_succeed)) { - persistRegion(curr_region, ®ion_task_lock, "tryFlushRegionData"); + persistRegion(curr_region, ®ion_task_lock, PersistRegionReason::Flush, ""); curr_region.markCompactLog(); curr_region.cleanApproxMemCacheInfo(); GET_METRIC(tiflash_raft_apply_write_command_duration_seconds, type_flush_region).Observe(watch.elapsedSeconds()); @@ -547,7 +574,7 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( || cmd_type == raft_cmdpb::AdminCmdType::BatchSwitchWitness) { tryFlushRegionCacheInStorage(tmt, curr_region, log); - persistRegion(curr_region, ®ion_task_lock, fmt::format("admin cmd useless {}", cmd_type).c_str()); + persistRegion(curr_region, ®ion_task_lock, PersistRegionReason::UselessAdminCommand, fmt::format("{}", cmd_type).c_str()); return EngineStoreApplyRes::Persist; } return EngineStoreApplyRes::None; @@ -630,7 +657,7 @@ EngineStoreApplyRes KVStore::handleAdminRaftCmd(raft_cmdpb::AdminRequest && requ const auto persist_and_sync = [&](const Region & region) { tryFlushRegionCacheInStorage(tmt, region, log); - persistRegion(region, ®ion_task_lock, "admin raft cmd"); + persistRegion(region, ®ion_task_lock, PersistRegionReason::AdminCommand, ""); }; const auto handle_batch_split = [&](Regions & split_regions) { @@ -972,27 +999,23 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) { - if (is_background) - { - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg).Increment(); - } - else - { - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg).Increment(); - } - Stopwatch general_watch; UInt64 total_dm_flush_millis = 0; UInt64 total_kvs_flush_millis = 0; + UInt64 total_kvs_flush_count = 0; SCOPE_EXIT({ if (is_background) { + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg).Increment(); + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg_breakdown_kvs).Increment(total_kvs_flush_count); GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg).Observe(general_watch.elapsedSeconds()); GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg_breakdown_dm).Observe(total_dm_flush_millis / 1000.0); GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg_breakdown_kvs).Observe(total_kvs_flush_millis / 1000.0); } else { + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg).Increment(); + GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg_breakdown_kvs).Increment(total_kvs_flush_count); GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(general_watch.elapsedSeconds()); GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg_breakdown_dm).Observe(total_dm_flush_millis / 1000.0); GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg_breakdown_kvs).Observe(total_kvs_flush_millis / 1000.0); @@ -1087,7 +1110,8 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan { Stopwatch watch2; watch2.restart(); - persistRegion(*region_ptr, std::make_optional(®ion_task_lock), reason.c_str()); + persistRegion(*region_ptr, std::make_optional(®ion_task_lock), PersistRegionReason::ProactiveFlush, reason.c_str()); + total_kvs_flush_count += 1; total_kvs_flush_millis += watch2.elapsedMilliseconds(); } } @@ -1145,6 +1169,7 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 // TODO Passive `CompactLog`flush will not update this field, // which make this not usable in `exec_compact_log`. Pending fix. // TODO flushed state is never persisted, check if it will lead to a problem. + // TODO Why don't we just use persisted applied_index in RegionPersister? region->setFlushedState(compact_index, compact_term); region->markCompactLog(); region->cleanApproxMemCacheInfo(); diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 93751167ba6..84175e49405 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -21,6 +21,8 @@ #include #include +#include + namespace TiDB { struct TableInfo; @@ -79,6 +81,30 @@ class RegionPersister; struct CheckpointInfo; using CheckpointInfoPtr = std::shared_ptr; +enum class PersistRegionReason +{ + Debug, + UselessAdminCommand, + AdminCommand, + Flush, + ProactiveFlush, + ApplySnapshotPrevRegion, + ApplySnapshotCurRegion, + IngestSst +}; + +constexpr const char * PersistRegionReasonMap[magic_enum::enum_count()] = { + "debug", + "admin cmd useless", + "admin raft cmd", + "tryFlushRegionData", + "ProactiveFlush", + "save previous region before apply", + "save current region after apply", + "ingestsst"}; + +static_assert(magic_enum::enum_count() == sizeof(PersistRegionReasonMap) / sizeof(const char *)); + /// TODO: brief design document. class KVStore final : private boost::noncopyable { @@ -263,7 +289,7 @@ class KVStore final : private boost::noncopyable bool canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term, UInt64 truncated_index, UInt64 truncated_term); bool forceFlushRegionDataImpl(Region & curr_region, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term); - void persistRegion(const Region & region, std::optional region_task_lock, const char * caller); + void persistRegion(const Region & region, std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg); void releaseReadIndexWorkers(); void handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTaskLock &); From 2ea06c284514a2b43342a8ce01b0562cc929fa4c Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Thu, 20 Jul 2023 17:16:24 +0800 Subject: [PATCH 24/66] z Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/RegionManager.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/Transaction/RegionManager.h b/dbms/src/Storages/Transaction/RegionManager.h index 24e17484059..09e2f6b1ba7 100644 --- a/dbms/src/Storages/Transaction/RegionManager.h +++ b/dbms/src/Storages/Transaction/RegionManager.h @@ -26,6 +26,7 @@ class RegionTaskLock; struct RegionTaskCtrl : MutexLockWrap { + // TODO This lock may be changed back to simple mutex. typedef std::recursive_mutex Mut; /// The life time of each RegionTaskElement element should be as long as RegionManager, just return const ref. struct RegionTaskElement : private boost::noncopyable From a0ecfe5b24645db0f0e00f0e48f7cf58e1bc0fba Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 25 Jul 2023 12:53:26 +0800 Subject: [PATCH 25/66] update jul26 Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- dbms/src/Storages/Transaction/ReadIndexWorker.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 90e8a744b5b..c6aa3728744 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 90e8a744b5b6c1399c60a5238310523c57672cf0 +Subproject commit c6aa3728744654d5708c916707cb05273928fd16 diff --git a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp index 680771fdc44..abc5c855dc2 100644 --- a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp +++ b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp @@ -526,6 +526,7 @@ void ReadIndexDataNode::runOneRound(const TiFlashRaftProxyHelper & helper, const auto _ = genLockGuard(); { + // Find the task with the maximum ts in all `waiting_tasks`. Timestamp max_ts = 0; ReadIndexFuturePtr max_ts_task = nullptr; { From eb698cca243a71c9207900200dd5a9affdf94512 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 25 Jul 2023 14:20:56 +0800 Subject: [PATCH 26/66] fix proxy Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index c6aa3728744..76e7cd4cc82 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit c6aa3728744654d5708c916707cb05273928fd16 +Subproject commit 76e7cd4cc8209e25672642fcda26638b6a70a058 From ac8e88fce840d2a0d971a4d7d1041aa77fa8d976 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 25 Jul 2023 16:03:46 +0800 Subject: [PATCH 27/66] get flushed state Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 5 ----- dbms/src/Storages/Transaction/ProxyFFI.cpp | 9 --------- dbms/src/Storages/Transaction/ProxyFFI.h | 2 -- dbms/src/Storages/Transaction/Region.cpp | 6 ++---- dbms/src/Storages/Transaction/Region.h | 10 ---------- dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp | 2 +- 7 files changed, 4 insertions(+), 32 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 76e7cd4cc82..04a72f1227a 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 76e7cd4cc8209e25672642fcda26638b6a70a058 +Subproject commit 04a72f1227ad382703d050e44c704ef54746d42f diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 1bffee9dc48..8a34067f0dc 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -1166,11 +1166,6 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 auto f = [&]() { // So proxy can get the current compact state of this region of TiFlash's side. - // TODO Passive `CompactLog`flush will not update this field, - // which make this not usable in `exec_compact_log`. Pending fix. - // TODO flushed state is never persisted, check if it will lead to a problem. - // TODO Why don't we just use persisted applied_index in RegionPersister? - region->setFlushedState(compact_index, compact_term); region->markCompactLog(); region->cleanApproxMemCacheInfo(); // We will notify even if `flush_state.applied_index` is greater than `compact_index`, diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 0cf4403b9ce..b5b0993da33 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -879,13 +879,4 @@ std::string_view buffToStrView(const BaseBuffView & buf) return std::string_view{buf.data, buf.len}; } -FlushedState GetFlushedState(EngineStoreServerWrap * server, uint64_t region_id, uint8_t acquire_lock) -{ - // TODO - UNUSED(acquire_lock); - auto & kvstore = server->tmt->getKVStore(); - auto region_ptr = kvstore->getRegion(region_id); - return region_ptr->getFlushedState(); -} - } // namespace DB diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index 9dee0b5a5d6..b14a888fa77 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -171,7 +171,6 @@ void SetStore(EngineStoreServerWrap *, BaseBuffView); void SetPBMsByBytes(MsgPBType type, RawVoidPtr ptr, BaseBuffView view); void HandleSafeTSUpdate(EngineStoreServerWrap * server, uint64_t region_id, uint64_t self_safe_ts, uint64_t leader_safe_ts); FastAddPeerRes FastAddPeer(EngineStoreServerWrap * server, uint64_t region_id, uint64_t new_peer_id); -FlushedState GetFlushedState(EngineStoreServerWrap * server, uint64_t region_id, uint8_t acquire_lock); } inline EngineStoreServerHelper GetEngineStoreServerHelper( @@ -219,7 +218,6 @@ inline EngineStoreServerHelper GetEngineStoreServerHelper( .fn_set_pb_msg_by_bytes = SetPBMsByBytes, .fn_handle_safe_ts_update = HandleSafeTSUpdate, .fn_fast_add_peer = FastAddPeer, - .fn_get_flushed_state = GetFlushedState, }; } diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 6a15ba9eb42..26fd89d0609 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -56,12 +56,10 @@ std::optional Region::readDataByWriteIt(const RegionData::Co } catch (DB::Exception & e) { - e.addMessage(fmt::format("(region id {}, applied_index:{}, applied_term:{}, flushed_index:{}, flushed_term:{})", + e.addMessage(fmt::format("(region id {}, applied_index:{}, applied_term:{})", meta.regionId(), appliedIndex(), - appliedIndexTerm(), - flushed_state.applied_index, - flushed_state.applied_term)); + appliedIndexTerm())); throw; } } diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index 841dc24d02d..cf2fd397481 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -202,15 +202,6 @@ class Region : public std::enable_shared_from_this std::pair getApproxMemCacheInfo() const; void cleanApproxMemCacheInfo() const; - void setFlushedState(uint64_t flushed_index, uint64_t flushed_term) - { - flushed_state.applied_index = flushed_index; - flushed_state.applied_term = flushed_term; - } - FlushedState getFlushedState() - { - return flushed_state; - } RegionMeta & mutMeta() { return meta; } RaftstoreVer getClusterRaftstoreVer(); @@ -255,7 +246,6 @@ class Region : public std::enable_shared_from_this mutable std::atomic last_compact_log_time{Timepoint::min()}; mutable std::atomic approx_mem_cache_rows{0}; mutable std::atomic approx_mem_cache_bytes{0}; - FlushedState flushed_state{0, 0}; }; class RegionRaftCommandDelegate : public Region diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 39a4b4865d4..0cb1f341e11 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -669,7 +669,7 @@ TEST_F(RegionKVStoreTest, Writes) } catch (Exception & e) { - ASSERT_EQ(e.message(), "Raw TiDB PK: 800000000000091D, Prewrite ts: 2333 can not found in default cf for key: 7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE: (region id 1, applied_index:5, applied_term:5, flushed_index:0, flushed_term:0)"); + ASSERT_EQ(e.message(), "Raw TiDB PK: 800000000000091D, Prewrite ts: 2333 can not found in default cf for key: 7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE: (region id 1, applied_index:5, applied_term:5)"); ASSERT_EQ(kvs.getRegion(1)->dataInfo(), "[write 1 lock 1 ]"); kvs.getRegion(1)->tryCompactionFilter(1000); } From d4f0961b21aed1d6d6a8f6c4b799546dc876c065 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 1 Aug 2023 18:11:19 +0800 Subject: [PATCH 28/66] mew log Signed-off-by: CalvinNeo --- dbms/src/Common/TiFlashMetrics.h | 3 +++ dbms/src/Storages/Transaction/KVStore.cpp | 12 +++++++++--- dbms/src/Storages/Transaction/KVStore.h | 4 ++-- 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index dcc498f92cf..95844e9566f 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -281,6 +281,9 @@ namespace DB F(type_flush_useful_admin, {{"type", "flush_useful_admin"}}), \ F(type_flush_passive, {{"type", "flush_passive"}}), \ F(type_flush_proactive, {{"type", "flush_proactive"}}), \ + F(type_flush_log_gap, {{"type", "flush_log_gap"}}), \ + F(type_flush_size, {{"type", "flush_size"}}), \ + F(type_flush_rowcount, {{"type", "flush_rowcount"}}), \ F(type_exec_compact, {{"type", "exec_compact"}})) \ M(tiflash_raft_read_index_count, "Total number of raft read index", Counter) \ M(tiflash_stale_read_count, "Total number of stale read", Counter) \ diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 8a34067f0dc..d425322f5aa 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -480,15 +480,21 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl bool can_flush = false; auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); - if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) - || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) + + if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed)) + { + GET_METRIC(tiflash_raft_raft_events_count, type_flush_rowcount).Increment(1); + can_flush = true; + } + if (size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) { - // if rows or bytes more than threshold, flush cache and persist mem data. + GET_METRIC(tiflash_raft_raft_events_count, type_flush_size).Increment(1); can_flush = true; } auto gap_threshold = region_compact_log_gap.load(); if (index > truncated_index + gap_threshold) { + GET_METRIC(tiflash_raft_raft_events_count, type_flush_log_gap).Increment(1); can_flush = true; } diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 84175e49405..4cf992ddc6d 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -84,9 +84,9 @@ using CheckpointInfoPtr = std::shared_ptr; enum class PersistRegionReason { Debug, - UselessAdminCommand, + UselessAdminCommand, // Does not include passive CompactLog AdminCommand, - Flush, + Flush, // passive CompactLog ProactiveFlush, ApplySnapshotPrevRegion, ApplySnapshotCurRegion, From 4de6a633a6037bbf8cb89758c7772e7f777aa66c Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Thu, 3 Aug 2023 11:10:13 +0800 Subject: [PATCH 29/66] aaaa Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 38 +++++++++++++++-------- dbms/src/Storages/Transaction/Region.cpp | 6 ++++ dbms/src/Storages/Transaction/Region.h | 2 ++ 3 files changed, 33 insertions(+), 13 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index d425322f5aa..010d77e9cf0 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -1087,6 +1087,7 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan auto region_rowkey_range = std::get<2>(region.second); auto region_id = region.first; auto region_ptr = std::get<3>(region.second); + auto applied_index = std::get<1>(region.second); { auto region_task_lock = region_manager.genRegionTaskLock(region_id); if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) @@ -1108,17 +1109,35 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. Stopwatch watch2; - watch2.restart(); storage->flushCache(tmt.getContext(), std::get<2>(region.second)); total_dm_flush_millis += watch2.elapsedSecondsFromLastTime(); } fiu_do_on(FailPoints::proactive_flush_between_persist_cache_and_region, return;); { Stopwatch watch2; - watch2.restart(); - persistRegion(*region_ptr, std::make_optional(®ion_task_lock), PersistRegionReason::ProactiveFlush, reason.c_str()); - total_kvs_flush_count += 1; - total_kvs_flush_millis += watch2.elapsedMilliseconds(); + int skip_reason = 0; + if (region_ptr->lastCompactLogTime() + Seconds{region_compact_log_period.load(std::memory_order_relaxed)} > Clock::now()) + { + skip_reason = 1; + } + else if (region_ptr->lastCompactLogApplied() + 15 < applied_index) + { + skip_reason = 2; + } + if (skip_reason) + { + LOG_INFO(log, "skip flush region {} for skip reason {}, region range:[{},{}], flushed segment range:[{},{}]", region_id, skip_reason, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); + } + else + { + persistRegion(*region_ptr, std::make_optional(®ion_task_lock), PersistRegionReason::ProactiveFlush, reason.c_str()); + // So proxy can get the current compact state of this region of TiFlash's side. + region_ptr->markCompactLog(); + region_ptr->cleanApproxMemCacheInfo(); + // TODO this metric is not necessary. + total_kvs_flush_count += 1; + total_kvs_flush_millis += watch2.elapsedMilliseconds(); + } } } } @@ -1157,10 +1176,7 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 LOG_INFO(log, "region {} has been removed, ignore", region_id); return; } - if (region->lastCompactLogTime() + Seconds{region_compact_log_period.load(std::memory_order_relaxed)} > Clock::now()) - { - return; - } + if (is_background) { GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_bg).Increment(); @@ -1170,10 +1186,6 @@ void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_fg).Increment(); } auto f = [&]() { - // So proxy can get the current compact state of this region of TiFlash's side. - - region->markCompactLog(); - region->cleanApproxMemCacheInfo(); // We will notify even if `flush_state.applied_index` is greater than `compact_index`, // since this greater `applied_index` may not trigger a compact log. // We will maintain the biggest on Proxy's side. diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 26fd89d0609..138a07ec2e5 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -494,6 +494,7 @@ std::string Region::dataInfo() const void Region::markCompactLog() const { last_compact_log_time = Clock::now(); + last_compact_log_applied = appliedIndex(); } Timepoint Region::lastCompactLogTime() const @@ -501,6 +502,11 @@ Timepoint Region::lastCompactLogTime() const return last_compact_log_time; } +UInt64 Region::lastCompactLogApplied() const +{ + return last_compact_log_applied; +} + Region::CommittedScanner Region::createCommittedScanner(bool use_lock, bool need_value) { return Region::CommittedScanner(this->shared_from_this(), use_lock, need_value); diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index cf2fd397481..e47c3e8a591 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -151,6 +151,7 @@ class Region : public std::enable_shared_from_this void markCompactLog() const; Timepoint lastCompactLogTime() const; + UInt64 lastCompactLogApplied() const; friend bool operator==(const Region & region1, const Region & region2) { @@ -244,6 +245,7 @@ class Region : public std::enable_shared_from_this std::atomic snapshot_event_flag{1}; const TiFlashRaftProxyHelper * proxy_helper{nullptr}; mutable std::atomic last_compact_log_time{Timepoint::min()}; + mutable std::atomic last_compact_log_applied{0}; mutable std::atomic approx_mem_cache_rows{0}; mutable std::atomic approx_mem_cache_bytes{0}; }; From e9e7fcec8ce84bc545d7f02023b817d7a8e4ca8f Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 11:02:31 +0800 Subject: [PATCH 30/66] f Signed-off-by: CalvinNeo --- dbms/src/Common/TiFlashMetrics.h | 16 +-- dbms/src/Storages/Transaction/KVStore.cpp | 122 +++++++++++------- dbms/src/Storages/Transaction/Region.cpp | 8 +- .../tests/gtest_proactive_flush.cpp | 10 +- .../Transaction/tests/kvstore_helper.h | 1 + 5 files changed, 96 insertions(+), 61 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 95844e9566f..8208d4d382b 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -146,9 +146,7 @@ namespace DB F(type_compact_log_segment_bg, {"type", "compact_log_segment_bg"}), \ F(type_compact_log_segment_fg, {"type", "compact_log_segment_fg"}), \ F(type_compact_log_region_bg, {"type", "compact_log_region_bg"}), \ - F(type_compact_log_region_fg, {"type", "compact_log_region_fg"}), \ - F(type_compact_log_segment_bg_breakdown_kvs, {"type", "compact_log_segment_bg_breakdown_kvs"}), \ - F(type_compact_log_segment_fg_breakdown_kvs, {"type", "compact_log_segment_fg_breakdown_kvs"})) \ + F(type_compact_log_region_fg, {"type", "compact_log_region_fg"})) \ M(tiflash_storage_subtask_duration_seconds, "Bucketed histogram of storage's sub task duration", Histogram, \ F(type_delta_merge_bg, {{"type", "delta_merge_bg"}}, ExpBuckets{0.001, 2, 20}), \ F(type_delta_merge_bg_gc, {{"type", "delta_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ @@ -163,10 +161,8 @@ namespace DB F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20}), \ F(type_compact_log_bg, {{"type", "compact_log_bg"}}, ExpBuckets{0.001, 2, 20}), \ F(type_compact_log_fg, {{"type", "compact_log_fg"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_fg_breakdown_kvs, {{"type", "compact_log_fg_breakdown_kvs"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_fg_breakdown_dm, {{"type", "compact_log_fg_breakdown_dm"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_bg_breakdown_kvs, {{"type", "compact_log_bg_breakdown_kvs"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_bg_breakdown_dm, {{"type", "compact_log_bg_breakdown_dm"}}, ExpBuckets{0.001, 2, 20})) \ + F(type_compact_log_fg_dm, {{"type", "compact_log_fg_dm"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_compact_log_bg_dm, {{"type", "compact_log_bg_dm"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_storage_throughput_bytes, "Calculate the throughput of tasks of storage in bytes", Gauge, /**/ \ F(type_write, {"type", "write"}), /**/ \ F(type_ingest, {"type", "ingest"}), /**/ \ @@ -272,7 +268,8 @@ namespace DB F(type_decode, {{"type", "decode"}}, ExpBuckets{0.0005, 2, 20}), \ F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ M(tiflash_raft_raft_log_lag_count, "Bucketed histogram raft index lag", Histogram, \ - F(type_compact_index, {{"type", "compact_index"}}, EqualWidthBuckets{0, 200, 5})) \ + F(type_compact_index, {{"type", "compact_index"}}, EqualWidthBuckets{0, 200, 5}), \ + F(type_applied_index, {{"type", "applied_index"}}, EqualWidthBuckets{0, 200, 5})) \ M(tiflash_raft_raft_events_count, "Raft event counter", Counter, \ F(type_pre_exec_compact, {{"type", "pre_exec_compact"}}), \ F(type_flush_apply_snapshot, {{"type", "flush_apply_snapshot"}}), \ @@ -285,6 +282,9 @@ namespace DB F(type_flush_size, {{"type", "flush_size"}}), \ F(type_flush_rowcount, {{"type", "flush_rowcount"}}), \ F(type_exec_compact, {{"type", "exec_compact"}})) \ + M(tiflash_raft_region_flush_size, "Bucketed histogram of region flushed size", Histogram, \ + F(type_flushed, {{"type", "flushed"}}, ExpBuckets{32, 2, 16}), \ + F(type_unflushed, {{"type", "unflushed"}}, ExpBuckets{32, 2, 16})) \ M(tiflash_raft_read_index_count, "Total number of raft read index", Counter) \ M(tiflash_stale_read_count, "Total number of stale read", Counter) \ M(tiflash_raft_read_index_duration_seconds, "Bucketed histogram of raft read index duration", Histogram, \ diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 010d77e9cf0..65aba13361e 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -61,6 +62,7 @@ KVStore::KVStore(Context & context) , region_compact_log_gap(500) { // default config about compact-log: period 120s, rows 40k, bytes 32MB. + LOG_INFO(log, "KVStore inited"); } void KVStore::restore(PathPool & path_pool, const TiFlashRaftProxyHelper * proxy_helper) @@ -498,19 +500,26 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl can_flush = true; } - auto current_gap = index - truncated_index; + auto current_gap = index > truncated_index ? index - truncated_index : 0; + auto last_compact_log_applied = curr_region.lastCompactLogApplied(); + auto current_applied_gap = index > last_compact_log_applied ? index - last_compact_log_applied : 0; GET_METRIC(tiflash_raft_raft_events_count, type_pre_exec_compact).Increment(1); GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); - LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}, gap {}/{}", curr_region.toString(false), rows, size_bytes, current_gap, gap_threshold); + LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}, gap {}/{} applied gap {}", curr_region.toString(false), rows, size_bytes, current_gap, gap_threshold, current_applied_gap); if (can_flush && flush_if_possible) { GET_METRIC(tiflash_raft_raft_events_count, type_exec_compact).Increment(1); // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap {}/{}", curr_region.toString(false), index, term, truncated_index, truncated_term, current_gap, gap_threshold); + GET_METRIC(tiflash_raft_region_flush_size, type_flushed).Observe(size_bytes); return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); } + else + { + GET_METRIC(tiflash_raft_region_flush_size, type_unflushed).Observe(size_bytes); + } return can_flush; } @@ -1007,24 +1016,18 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan { Stopwatch general_watch; UInt64 total_dm_flush_millis = 0; - UInt64 total_kvs_flush_millis = 0; - UInt64 total_kvs_flush_count = 0; SCOPE_EXIT({ if (is_background) { GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg).Increment(); - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg_breakdown_kvs).Increment(total_kvs_flush_count); GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg).Observe(general_watch.elapsedSeconds()); - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg_breakdown_dm).Observe(total_dm_flush_millis / 1000.0); - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg_breakdown_kvs).Observe(total_kvs_flush_millis / 1000.0); + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg_dm).Observe(total_dm_flush_millis / 1000.0); } else { GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg).Increment(); - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg_breakdown_kvs).Increment(total_kvs_flush_count); GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(general_watch.elapsedSeconds()); - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg_breakdown_dm).Observe(total_dm_flush_millis / 1000.0); - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg_breakdown_kvs).Observe(total_kvs_flush_millis / 1000.0); + GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg_dm).Observe(total_dm_flush_millis / 1000.0); } }); @@ -1044,8 +1047,17 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan /// It finds r1,r2,r3 in the following case. /// |------ range ------| /// |--- r1 ---|--- r2 ---|--- r3 ---| - std::unordered_map> region_compact_indexes; + struct RegionCompactInfo { + UInt64 applied_index; + UInt64 applied_term; + DM::RowKeyRange rowkey_range; + RegionPtr region_ptr; + bool skip_flush; + }; + std::unordered_map region_compact_indexes; + { + // Get the regions. auto task_lock = genTaskLock(); auto maybe_region_map = [&]() { auto manage_lock = genRegionMgrReadLock(); @@ -1075,31 +1087,58 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan table_id, storage->isCommonHandle(), storage->getRowKeyColumnSize()); - region_compact_indexes[overlapped_region.first] = {overlapped_region.second->appliedIndex(), overlapped_region.second->appliedIndexTerm(), region_rowkey_range, overlapped_region.second}; + region_compact_indexes[overlapped_region.first] = { + overlapped_region.second->appliedIndex(), + overlapped_region.second->appliedIndexTerm(), + region_rowkey_range, + overlapped_region.second, + false}; } } FAIL_POINT_PAUSE(FailPoints::pause_proactive_flush_before_persist_region); // Flush all segments in the range of regions. // TODO: combine adjacent range to do one flush. std::string reason = fmt::format("proactiveFlush{}", is_background ? "Bg" : "Fg"); - for (const auto & region : region_compact_indexes) + for (auto & region_compact_info : region_compact_indexes) { - auto region_rowkey_range = std::get<2>(region.second); - auto region_id = region.first; - auto region_ptr = std::get<3>(region.second); - auto applied_index = std::get<1>(region.second); + const auto & region_rowkey_range = region_compact_info.second.rowkey_range; + auto region_id = region_compact_info.first; + auto region_ptr = region_compact_info.second.region_ptr; + auto applied_index = region_compact_info.second.applied_index; { auto region_task_lock = region_manager.genRegionTaskLock(region_id); + enum class SkipReason + { + None = 0, + Time = 1, + Log = 2 + }; + SkipReason skip_reason = SkipReason::None; + if (region_ptr->lastCompactLogTime() + Seconds{region_compact_log_period.load(std::memory_order_relaxed)} > Clock::now()) + { + skip_reason = SkipReason::Time; + } + else if (region_ptr->lastCompactLogApplied() + 15 < applied_index) + { + skip_reason = SkipReason::Log; + } + if (skip_reason != SkipReason::None) + { + LOG_INFO(log, "skip flush region {} for skip reason {}, region range:[{},{}], flushed segment range:[{},{}]", region_id, magic_enum::enum_name(skip_reason), region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); + region_compact_info.second.skip_flush = true; + continue; + } if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) { // `region_rowkey_range` belongs to rowkey_range. // E.g. [0,9223372036854775807] belongs to [-9223372036854775808,9223372036854775807]. - // This segment has flushed. However, we still need to persist the region. - LOG_DEBUG(log, "segment of region {} flushed, [applied_index={}] [applied_term={}]", region.first, std::get<0>(region.second), std::get<1>(region.second)); + // This segment has flushed, and the region is locked. + // However, write may come between we lock regions. + LOG_DEBUG(log, "segment of region {} flushed, [applied_index={}] [applied_term={}]", region_compact_info.first, region_compact_info.second.applied_index, region_compact_info.second.applied_term); } else { - LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}]", region.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); + LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}]", region_compact_info.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); // Both flushCache and persistRegion should be protected by region task lock. // We can avoid flushCache with a region lock held, if we save some meta info before flushing cache. // Merely store applied_index is not enough, considering some cmds leads to modification of other meta data. @@ -1109,48 +1148,37 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. Stopwatch watch2; - storage->flushCache(tmt.getContext(), std::get<2>(region.second)); + storage->flushCache(tmt.getContext(), region_rowkey_range); total_dm_flush_millis += watch2.elapsedSecondsFromLastTime(); } fiu_do_on(FailPoints::proactive_flush_between_persist_cache_and_region, return;); { - Stopwatch watch2; - int skip_reason = 0; - if (region_ptr->lastCompactLogTime() + Seconds{region_compact_log_period.load(std::memory_order_relaxed)} > Clock::now()) - { - skip_reason = 1; - } - else if (region_ptr->lastCompactLogApplied() + 15 < applied_index) - { - skip_reason = 2; - } - if (skip_reason) - { - LOG_INFO(log, "skip flush region {} for skip reason {}, region range:[{},{}], flushed segment range:[{},{}]", region_id, skip_reason, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); - } - else - { - persistRegion(*region_ptr, std::make_optional(®ion_task_lock), PersistRegionReason::ProactiveFlush, reason.c_str()); - // So proxy can get the current compact state of this region of TiFlash's side. - region_ptr->markCompactLog(); - region_ptr->cleanApproxMemCacheInfo(); - // TODO this metric is not necessary. - total_kvs_flush_count += 1; - total_kvs_flush_millis += watch2.elapsedMilliseconds(); - } + persistRegion(*region_ptr, std::make_optional(®ion_task_lock), PersistRegionReason::ProactiveFlush, reason.c_str()); + // So proxy can get the current compact state of this region of TiFlash's side. + region_ptr->markCompactLog(); + region_ptr->cleanApproxMemCacheInfo(); } + // Drop region task lock. } + // Flush the origin cache again, in case writes has been written between handing regions. + storage->flushCache(tmt.getContext(), rowkey_range); + SYNC_FOR("after_proactiveFlushCacheAndRegion::loop_region"); } auto elapsed_coupled_flush = watch.elapsedMilliseconds(); watch.restart(); fiu_do_on(FailPoints::proactive_flush_between_persist_regions, return;); // forbid regions being removed. - for (const auto & region : region_compact_indexes) + for (const auto & region_compact_info : region_compact_indexes) { // Can truncated to flushed index, which is applied_index in this case. // Region can be removed since we don't lock kvstore here. - notifyCompactLog(region.first, std::get<0>(region.second), std::get<1>(region.second), is_background, false); + if (!region_compact_info.second.skip_flush) + { + auto region_id = region_compact_info.first; + LOG_DEBUG(log, "skip notify compactlog region {}", region_id); + notifyCompactLog(region_id, region_compact_info.second.applied_index, region_compact_info.second.applied_term, is_background, false); + } } auto elapsed_notify_proxy = watch.elapsedMilliseconds(); diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 138a07ec2e5..326bb621c51 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -494,7 +494,13 @@ std::string Region::dataInfo() const void Region::markCompactLog() const { last_compact_log_time = Clock::now(); - last_compact_log_applied = appliedIndex(); + uint64_t current_applied_index = 0; + if (last_compact_log_applied != 0) + { + uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; + GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); + } + last_compact_log_applied = current_applied_index; } Timepoint Region::lastCompactLogTime() const diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index 88e22d0b98f..e81d990eb87 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -14,6 +14,7 @@ #include "kvstore_helper.h" + namespace DB { namespace tests @@ -90,6 +91,7 @@ std::tuple RegionKVStoreTest::prepareForProactiveF UInt64 region_id2 = 7; TableID table_id; KVStore & kvs = getKVS(); + ctx.getTMTContext().debugSetKVStore(kvstore); MockRaftStoreProxy::FailCond cond; { initStorages(); @@ -272,14 +274,14 @@ try } CATCH -TEST_F(RegionKVStoreTest, ProactiveFlushRecover) +TEST_F(RegionKVStoreTest, ProactiveFlushRecover1) try { auto & ctx = TiFlashTestEnv::getGlobalContext(); std::shared_ptr> ai = std::make_shared>(); + // Safe to abort between flushCache and persistRegion. DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); { - // Safe to abort between flushing regions. auto tp = prepareForProactiveFlushTest(); auto table_id = std::get<0>(tp); auto region_id = std::get<1>(tp); @@ -334,9 +336,7 @@ try ASSERT_EQ(kvr2->appliedIndex(), r2->getLatestAppliedIndex()); } } - { - // Safe to abort between flushCache and persistRegion. - } + DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); } CATCH diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index 7b610663f38..7ea0152ff3a 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include From 2d080044fdb40ee7879aa21222bad6427a2361ba Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 13:09:18 +0800 Subject: [PATCH 31/66] a Signed-off-by: CalvinNeo --- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 30 ++++++++--------- dbms/src/Storages/Transaction/KVStore.cpp | 33 +++++++++++-------- 2 files changed, 35 insertions(+), 28 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 3283d15975a..bcbcc2d1893 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1267,11 +1267,11 @@ void DeltaMergeStore::waitForDeleteRange(const DB::DM::DMContextPtr &, const DB: bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const SegmentPtr & segment, ThreadType thread_type, InputType input_type) { - bool should_trigger_kvstore_flush = false; - fiu_do_on(FailPoints::skip_check_segment_update, { return should_trigger_kvstore_flush; }); + bool should_trigger_foreground_kvstore_flush = false; + fiu_do_on(FailPoints::skip_check_segment_update, { return should_trigger_foreground_kvstore_flush; }); if (segment->hasAbandoned()) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; const auto & delta = segment->getDelta(); size_t delta_saved_rows = delta->getRows(/* use_unsaved */ false); @@ -1404,7 +1404,7 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const // Raft Snapshot will always trigger to a KVStore fg flush. // Raft IngestSST will trigger a KVStore fg flush at best effort, // which means if the write cf has remained value, we still need to hold the sst file and wait for the next SST. - should_trigger_kvstore_flush = true; + should_trigger_foreground_kvstore_flush = true; } } else if (should_background_flush) @@ -1429,7 +1429,7 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const // Need to check the latest delta (maybe updated after foreground flush). If it is updating by another thread, // give up adding more tasks on this version of delta. if (segment->getDelta()->isUpdating()) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; auto try_fg_merge_delta = [&]() -> SegmentPtr { // If the table is already dropped, don't trigger foreground merge delta when executing `remove region peer`, @@ -1520,19 +1520,19 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const if (thread_type == ThreadType::Write) { if (try_fg_split(segment)) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; if (SegmentPtr new_segment = try_fg_merge_delta(); new_segment) { // After merge delta, we better check split immediately. if (try_bg_split(new_segment)) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; } } else if (thread_type == ThreadType::BG_MergeDelta) { if (try_bg_split(segment)) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; } if (dm_context->enable_logical_split) @@ -1540,24 +1540,24 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const // Logical split point is calculated based on stable. Always try to merge delta into the stable // before logical split is good for calculating the split point. if (try_bg_merge_delta()) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; if (try_bg_split(segment)) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; } else { // During the physical split delta will be merged, so we prefer physical split over merge delta. if (try_bg_split(segment)) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; if (try_bg_merge_delta()) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; } if (try_bg_compact()) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; if (try_place_delta_index()) - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; - return should_trigger_kvstore_flush; + return should_trigger_foreground_kvstore_flush; // The segment does not need any updates for now. } diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 65aba13361e..fadfed81774 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -1105,6 +1105,7 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan auto region_id = region_compact_info.first; auto region_ptr = region_compact_info.second.region_ptr; auto applied_index = region_compact_info.second.applied_index; + auto last_flushed_applied = region_ptr->lastCompactLogApplied(); { auto region_task_lock = region_manager.genRegionTaskLock(region_id); enum class SkipReason @@ -1128,26 +1129,32 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan region_compact_info.second.skip_flush = true; continue; } - if (rowkey_range.getStart() <= region_rowkey_range.getStart() && region_rowkey_range.getEnd() <= rowkey_range.getEnd()) + // Both flushCache and persistRegion should be protected by region task lock. + // We can avoid flushCache with a region lock held, if we save some meta info before flushing cache in memory. + // After flushCache, we will persist region and notify Proxy with the previously stored meta info. + // Meanwhile, other write/admin cmds may be executed, we have to handle the following cases: + // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. + // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. + // Merely persisting applied_index is not enough, consider some cmds leads to modification of other meta data. + + if (rowkey_range.getStart() <= region_rowkey_range.getStart() + && region_rowkey_range.getEnd() <= rowkey_range.getEnd() + && last_flushed_applied >= applied_index) { // `region_rowkey_range` belongs to rowkey_range. // E.g. [0,9223372036854775807] belongs to [-9223372036854775808,9223372036854775807]. - // This segment has flushed, and the region is locked. - // However, write may come between we lock regions. - LOG_DEBUG(log, "segment of region {} flushed, [applied_index={}] [applied_term={}]", region_compact_info.first, region_compact_info.second.applied_index, region_compact_info.second.applied_term); + // This segment has been flushed, and the region is locked. + // However, writes may come between we lock regions. + + // TODO We can save the applied_index of every region, before the last time we flushCache. + // And we will persistRegion according to this applied_index, following the upper note. + storage->flushCache(tmt.getContext(), region_rowkey_range); + LOG_DEBUG(log, "segment of region {} flushed, [applied_index={}] [applied_term={}] [last_flushed_applied={}]", region_compact_info.first, region_compact_info.second.applied_index, region_compact_info.second.applied_term, last_flushed_applied); } else { - LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}]", region_compact_info.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); - // Both flushCache and persistRegion should be protected by region task lock. - // We can avoid flushCache with a region lock held, if we save some meta info before flushing cache. - // Merely store applied_index is not enough, considering some cmds leads to modification of other meta data. - // After flushCache, we will persist region and notify Proxy with the previously stored meta info. - // However, this solution still involves region task lock in this function. - // Meanwhile, other write/admin cmds may be executed, they requires we acquire lock here: - // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. - // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. Stopwatch watch2; + LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}] [last_flushed_applied={}]", region_compact_info.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString(), last_flushed_applied); storage->flushCache(tmt.getContext(), region_rowkey_range); total_dm_flush_millis += watch2.elapsedSecondsFromLastTime(); } From 979f1cdda0d96bd2c099f43f01fce73a89de7781 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 13:51:46 +0800 Subject: [PATCH 32/66] remove some proactive code Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- dbms/src/Debug/MockRaftStoreProxy.cpp | 2 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 11 +- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 2 +- .../DeltaMerge/DeltaMergeStore_InternalBg.cpp | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 264 ++--------------- dbms/src/Storages/Transaction/ProxyFFI.cpp | 1 + .../tests/gtest_proactive_flush.cpp | 265 ------------------ 8 files changed, 37 insertions(+), 512 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 04a72f1227a..745272e78ab 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 04a72f1227ad382703d050e44c704ef54746d42f +Subproject commit 745272e78ab590befb41bed248dd9e06a5bd9d9f diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 7b1168fc127..39917d45b0a 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -141,11 +141,11 @@ KVGetStatus fn_get_region_local_state(RaftStoreProxyPtr ptr, uint64_t region_id, void fn_notify_compact_log(RaftStoreProxyPtr ptr, uint64_t region_id, uint64_t compact_index, uint64_t compact_term, uint64_t applied_index) { + UNUSED(applied_index); // Update flushed applied_index and truncated state. auto & x = as_ref(ptr); auto region = x.getRegion(region_id); ASSERT(region); - LOG_INFO(&Poco::Logger::get("!!!!!"), "!!!! fn_notify_compact_log {} commit index {} applied_index {} compact_index {} compact_term {}", region_id, region->getLatestCommitIndex(), applied_index, compact_index, compact_term); // `applied_index` in proxy's disk can still be less than the `applied_index` here when fg flush. if (region && region->getApply().truncated_state().index() < compact_index) { diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index bcbcc2d1893..8ce8717143a 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -130,7 +130,7 @@ std::pair DeltaMergeStore::MergeDeltaTaskPool::tryAddTask(const Back case TaskType::Compact: case TaskType::Flush: case TaskType::PlaceIndex: - case TaskType::NotifyCompactLog: + case TaskType::FlushDTAndKVStore: is_heavy = false; // reserve some task space for heavy tasks if (max_task_num > 1 && light_tasks.size() >= static_cast(max_task_num * 0.9)) @@ -1420,10 +1420,11 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const try_add_background_task(BackgroundTask{TaskType::Flush, dm_context, segment}); } } - if (should_background_compact_log) - { - try_add_background_task(BackgroundTask{TaskType::NotifyCompactLog, dm_context, segment}); - } + // TODO Will enable once TiKV supports. + // if (should_background_compact_log) + // { + // try_add_background_task(BackgroundTask{TaskType::FlushDTAndKVStore, dm_context, segment}); + // } } // Need to check the latest delta (maybe updated after foreground flush). If it is updating by another thread, diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 2d64034572b..83f018a155c 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -214,7 +214,7 @@ class DeltaMergeStore : private boost::noncopyable Compact, Flush, PlaceIndex, - NotifyCompactLog, + FlushDTAndKVStore, }; struct BackgroundTask diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index 7447038cb9e..033bb526dac 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -399,7 +399,7 @@ bool DeltaMergeStore::handleBackgroundTask(bool heavy) left = task.segment; type = ThreadType::BG_Flush; break; - case TaskType::NotifyCompactLog: + case TaskType::FlushDTAndKVStore: triggerCompactLog(task.dm_context, task.segment->getRowKeyRange(), true); case TaskType::PlaceIndex: task.segment->placeDeltaIndex(*task.dm_context); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index fadfed81774..b83ae36bbcd 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -59,7 +59,7 @@ KVStore::KVStore(Context & context) , region_compact_log_period(120) , region_compact_log_min_rows(40 * 1024) , region_compact_log_min_bytes(32 * 1024 * 1024) - , region_compact_log_gap(500) + , region_compact_log_gap(200) { // default config about compact-log: period 120s, rows 40k, bytes 32MB. LOG_INFO(log, "KVStore inited"); @@ -322,14 +322,15 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmdInner(const WriteCmdsView & cmds, /// This call is from Proxy's applying thread of this region, so: /// 1. No other thread can write from raft to this region even if we unlocked here. /// 2. If `proactiveFlushCacheAndRegion` causes a write stall, it will be forwarded to raft layer. - if (write_result) - { - auto & inner = write_result.value(); - for (auto it = inner.pending_flush_ranges.begin(); it != inner.pending_flush_ranges.end(); it++) - { - proactiveFlushCacheAndRegion(tmt, *it, inner.keyspace_id, inner.table_id, false); - } - } + // TODO We will enable this once TiKV supports. + // if (write_result) + // { + // auto & inner = write_result.value(); + // for (auto it = inner.pending_flush_ranges.begin(); it != inner.pending_flush_ranges.end(); it++) + // { + // proactiveFlushCacheAndRegion(tmt, *it, inner.keyspace_id, inner.table_id, false); + // } + // } return res; } @@ -494,25 +495,27 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl can_flush = true; } auto gap_threshold = region_compact_log_gap.load(); - if (index > truncated_index + gap_threshold) + + auto last_compact_log_applied = curr_region.lastCompactLogApplied(); + auto current_applied_gap = index > last_compact_log_applied ? index - last_compact_log_applied : 0; + + // TODO We will use truncated_index once Proxy/TiKV supports. + if (index > last_compact_log_applied + gap_threshold) { GET_METRIC(tiflash_raft_raft_events_count, type_flush_log_gap).Increment(1); can_flush = true; } - auto current_gap = index > truncated_index ? index - truncated_index : 0; - auto last_compact_log_applied = curr_region.lastCompactLogApplied(); - auto current_applied_gap = index > last_compact_log_applied ? index - last_compact_log_applied : 0; - GET_METRIC(tiflash_raft_raft_events_count, type_pre_exec_compact).Increment(1); - GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); - LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}, gap {}/{} applied gap {}", curr_region.toString(false), rows, size_bytes, current_gap, gap_threshold, current_applied_gap); + // GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); + GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(current_applied_gap); + LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}, gap {}/{}", curr_region.toString(false), rows, size_bytes, current_applied_gap, gap_threshold); if (can_flush && flush_if_possible) { GET_METRIC(tiflash_raft_raft_events_count, type_exec_compact).Increment(1); // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. - LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap {}/{}", curr_region.toString(false), index, term, truncated_index, truncated_term, current_gap, gap_threshold); + LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap {}/{}", curr_region.toString(false), index, term, truncated_index, truncated_term, current_applied_gap, gap_threshold); GET_METRIC(tiflash_raft_region_flush_size, type_flushed).Observe(size_bytes); return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); } @@ -1014,226 +1017,11 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) { - Stopwatch general_watch; - UInt64 total_dm_flush_millis = 0; - SCOPE_EXIT({ - if (is_background) - { - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_bg).Increment(); - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg).Observe(general_watch.elapsedSeconds()); - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_bg_dm).Observe(total_dm_flush_millis / 1000.0); - } - else - { - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_segment_fg).Increment(); - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg).Observe(general_watch.elapsedSeconds()); - GET_METRIC(tiflash_storage_subtask_duration_seconds, type_compact_log_fg_dm).Observe(total_dm_flush_millis / 1000.0); - } - }); - - auto storage = tmt.getStorages().get(keyspace_id, table_id); - if (unlikely(storage == nullptr)) - { - LOG_WARNING(log, - "proactiveFlushCacheAndRegion can not get table for table id {}, ignored", - table_id); - return; - } - auto range = std::make_pair(TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.start.toRegionKey(table_id))), - TiKVRangeKey::makeTiKVRangeKey(RecordKVFormat::encodeAsTiKVKey(*rowkey_range.end.toRegionKey(table_id)))); - Stopwatch watch; - - LOG_INFO(log, "Start proactive flush region range [{},{}] [table_id={}] [keyspace_id={}] [is_background={}]", range.first.toDebugString(), range.second.toDebugString(), table_id, keyspace_id, is_background); - /// It finds r1,r2,r3 in the following case. - /// |------ range ------| - /// |--- r1 ---|--- r2 ---|--- r3 ---| - struct RegionCompactInfo - { - UInt64 applied_index; - UInt64 applied_term; - DM::RowKeyRange rowkey_range; - RegionPtr region_ptr; - bool skip_flush; - }; - std::unordered_map region_compact_indexes; - { - // Get the regions. - auto task_lock = genTaskLock(); - auto maybe_region_map = [&]() { - auto manage_lock = genRegionMgrReadLock(); - // Check if the region overlaps. - return manage_lock.index.findByRangeChecked(range); - }(); - - if (std::holds_alternative(maybe_region_map)) - { - auto & info = std::get(maybe_region_map); - FmtBuffer buffer; - buffer.joinStr( - std::get<1>(info).begin(), - std::get<1>(info).end(), - [&](const auto & e, FmtBuffer & b) { b.fmtAppend("{}", e); }, - " "); - std::string fmt_error = fmt::format("Find overlapped regions at {}, regions are {}, quit", std::get<0>(info).toDebugString(), buffer.toString()); - LOG_ERROR(log, fmt_error); - throw Exception(fmt_error, ErrorCodes::LOGICAL_ERROR); - } - - auto & region_map = std::get(maybe_region_map); - for (const auto & overlapped_region : region_map) - { - auto region_rowkey_range = DM::RowKeyRange::fromRegionRange( - overlapped_region.second->getRange(), - table_id, - storage->isCommonHandle(), - storage->getRowKeyColumnSize()); - region_compact_indexes[overlapped_region.first] = { - overlapped_region.second->appliedIndex(), - overlapped_region.second->appliedIndexTerm(), - region_rowkey_range, - overlapped_region.second, - false}; - } - } - FAIL_POINT_PAUSE(FailPoints::pause_proactive_flush_before_persist_region); - // Flush all segments in the range of regions. - // TODO: combine adjacent range to do one flush. - std::string reason = fmt::format("proactiveFlush{}", is_background ? "Bg" : "Fg"); - for (auto & region_compact_info : region_compact_indexes) - { - const auto & region_rowkey_range = region_compact_info.second.rowkey_range; - auto region_id = region_compact_info.first; - auto region_ptr = region_compact_info.second.region_ptr; - auto applied_index = region_compact_info.second.applied_index; - auto last_flushed_applied = region_ptr->lastCompactLogApplied(); - { - auto region_task_lock = region_manager.genRegionTaskLock(region_id); - enum class SkipReason - { - None = 0, - Time = 1, - Log = 2 - }; - SkipReason skip_reason = SkipReason::None; - if (region_ptr->lastCompactLogTime() + Seconds{region_compact_log_period.load(std::memory_order_relaxed)} > Clock::now()) - { - skip_reason = SkipReason::Time; - } - else if (region_ptr->lastCompactLogApplied() + 15 < applied_index) - { - skip_reason = SkipReason::Log; - } - if (skip_reason != SkipReason::None) - { - LOG_INFO(log, "skip flush region {} for skip reason {}, region range:[{},{}], flushed segment range:[{},{}]", region_id, magic_enum::enum_name(skip_reason), region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString()); - region_compact_info.second.skip_flush = true; - continue; - } - // Both flushCache and persistRegion should be protected by region task lock. - // We can avoid flushCache with a region lock held, if we save some meta info before flushing cache in memory. - // After flushCache, we will persist region and notify Proxy with the previously stored meta info. - // Meanwhile, other write/admin cmds may be executed, we have to handle the following cases: - // For write cmds, we need to support replay from KVStore level, like enhancing duplicate key detection. - // For admin cmds, it can cause insertion/deletion of regions, so it can't be replayed currently. - // Merely persisting applied_index is not enough, consider some cmds leads to modification of other meta data. - - if (rowkey_range.getStart() <= region_rowkey_range.getStart() - && region_rowkey_range.getEnd() <= rowkey_range.getEnd() - && last_flushed_applied >= applied_index) - { - // `region_rowkey_range` belongs to rowkey_range. - // E.g. [0,9223372036854775807] belongs to [-9223372036854775808,9223372036854775807]. - // This segment has been flushed, and the region is locked. - // However, writes may come between we lock regions. - - // TODO We can save the applied_index of every region, before the last time we flushCache. - // And we will persistRegion according to this applied_index, following the upper note. - storage->flushCache(tmt.getContext(), region_rowkey_range); - LOG_DEBUG(log, "segment of region {} flushed, [applied_index={}] [applied_term={}] [last_flushed_applied={}]", region_compact_info.first, region_compact_info.second.applied_index, region_compact_info.second.applied_term, last_flushed_applied); - } - else - { - Stopwatch watch2; - LOG_DEBUG(log, "extra segment of region {} to flush, region range:[{},{}], flushed segment range:[{},{}] [last_flushed_applied={}]", region_compact_info.first, region_rowkey_range.getStart().toDebugString(), region_rowkey_range.getEnd().toDebugString(), rowkey_range.getStart().toDebugString(), rowkey_range.getEnd().toDebugString(), last_flushed_applied); - storage->flushCache(tmt.getContext(), region_rowkey_range); - total_dm_flush_millis += watch2.elapsedSecondsFromLastTime(); - } - fiu_do_on(FailPoints::proactive_flush_between_persist_cache_and_region, return;); - { - persistRegion(*region_ptr, std::make_optional(®ion_task_lock), PersistRegionReason::ProactiveFlush, reason.c_str()); - // So proxy can get the current compact state of this region of TiFlash's side. - region_ptr->markCompactLog(); - region_ptr->cleanApproxMemCacheInfo(); - } - // Drop region task lock. - } - // Flush the origin cache again, in case writes has been written between handing regions. - storage->flushCache(tmt.getContext(), rowkey_range); - SYNC_FOR("after_proactiveFlushCacheAndRegion::loop_region"); - } - auto elapsed_coupled_flush = watch.elapsedMilliseconds(); - watch.restart(); - - fiu_do_on(FailPoints::proactive_flush_between_persist_regions, return;); - // forbid regions being removed. - for (const auto & region_compact_info : region_compact_indexes) - { - // Can truncated to flushed index, which is applied_index in this case. - // Region can be removed since we don't lock kvstore here. - if (!region_compact_info.second.skip_flush) - { - auto region_id = region_compact_info.first; - LOG_DEBUG(log, "skip notify compactlog region {}", region_id); - notifyCompactLog(region_id, region_compact_info.second.applied_index, region_compact_info.second.applied_term, is_background, false); - } - } - auto elapsed_notify_proxy = watch.elapsedMilliseconds(); - - LOG_DEBUG(log, "Finished proactive flush region range [{},{}] of {} regions. [couple_flush={}] [notify_proxy={}] [table_id={}] [keyspace_id={}] [is_background={}]", range.first.toDebugString(), range.second.toDebugString(), region_compact_indexes.size(), elapsed_coupled_flush, elapsed_notify_proxy, table_id, keyspace_id, is_background); -} - -/// The function will notify Proxy to schedule a CheckCompact task. When this task is handled, -/// When handling, it will try to update `applied_index` and `truncated_state`, and then persist. -/// The updated `truncated_state` will not exceed the recorded `max_compact_index`. -/// `max_compact_index` is updated by CompactLog, whether it is filtered. -/// Requirements: -/// 1. The caller will guarantee that delta cache has been flushed. -/// 1. Region cache being persisted before notifying. -/// The truncated_index is mono-increase since: -/// 1. Every non-filtered passive flush uses what is from the `CompactLog`, -/// and `entry_storage::first_index`/`compact_raft_log` will guard that. -/// 1. Every proactive flush uses the newest `applied_index`. -void KVStore::notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held) -{ - auto region = getRegion(region_id); - if (!region) - { - LOG_INFO(log, "region {} has been removed, ignore", region_id); - return; - } - - if (is_background) - { - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_bg).Increment(); - } - else - { - GET_METRIC(tiflash_storage_subtask_count, type_compact_log_region_fg).Increment(); - } - auto f = [&]() { - // We will notify even if `flush_state.applied_index` is greater than `compact_index`, - // since this greater `applied_index` may not trigger a compact log. - // We will maintain the biggest on Proxy's side. - getProxyHelper()->notifyCompactLog(region_id, compact_index, compact_term, compact_index); - }; - if (lock_held) - { - f(); - } - else - { - auto region_task_lock = region_manager.genRegionTaskLock(region_id); - f(); - } + // TODO We will enable this once TiKV supports. + UNUSED(tmt); + UNUSED(rowkey_range); + UNUSED(keyspace_id); + UNUSED(table_id); + UNUSED(is_background); } } // namespace DB diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index b5b0993da33..90b3e223df9 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -150,6 +150,7 @@ uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t try { auto & kvstore = server->tmt->getKVStore(); + // TODO truncated_index is always 0, since not support in Proxy. return kvstore->tryFlushRegionData(region_id, false, flush_pattern, *server->tmt, index, term, truncated_index, truncated_term); } catch (...) diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index e81d990eb87..3aeeaf26aea 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -19,8 +19,6 @@ namespace DB { namespace tests { - - TEST_F(RegionKVStoreTest, KVStorePassivePersistence) try { @@ -78,268 +76,5 @@ try } } CATCH - -std::tuple RegionKVStoreTest::prepareForProactiveFlushTest() -{ - auto & ctx = TiFlashTestEnv::getGlobalContext(); - // Allow enough large segment size. - ctx.getSettingsRef().dt_segment_limit_rows = 1000000; - ctx.getSettingsRef().dt_segment_limit_size = 1000000; - ctx.getSettingsRef().dt_segment_delta_cache_limit_rows = 0; - ctx.getSettingsRef().dt_segment_delta_cache_limit_size = 0; - UInt64 region_id = 1; - UInt64 region_id2 = 7; - TableID table_id; - KVStore & kvs = getKVS(); - ctx.getTMTContext().debugSetKVStore(kvstore); - MockRaftStoreProxy::FailCond cond; - { - initStorages(); - table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); - HandleID end_index = 100; - HandleID mid_index = 50; - auto start = RecordKVFormat::genKey(table_id, 0); - auto end = RecordKVFormat::genKey(table_id, end_index); - proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::make_pair(start.toString(), end.toString())); - - auto source_region = kvs.getRegion(region_id); - auto old_epoch = source_region->mutMeta().getMetaRegion().region_epoch(); - auto && [request, response] = MockRaftStoreProxy::composeBatchSplit( - {region_id, region_id2}, - {{RecordKVFormat::genKey(table_id, mid_index), RecordKVFormat::genKey(table_id, end_index)}, - {RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, mid_index)}}, - old_epoch); - auto && [index2, term2] = proxy_instance->adminCommand(region_id, std::move(request), std::move(response)); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); - - auto kvr1 = kvs.getRegion(region_id); - auto kvr2 = kvs.getRegion(region_id2); - ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); - ctx.getTMTContext().getRegionTable().updateRegion(*kvr2); - } - return std::make_tuple(table_id, region_id, region_id2); -} - -TEST_F(RegionKVStoreTest, ProactiveFlushConsistency) -try -{ - auto & ctx = TiFlashTestEnv::getGlobalContext(); - auto tp = prepareForProactiveFlushTest(); - // auto table_id = std::get<0>(tp); - auto region_id = std::get<1>(tp); - // auto region_id2 = std::get<2>(tp); - MockRaftStoreProxy::FailCond cond; - KVStore & kvs = getKVS(); - - std::shared_ptr> ai = std::make_shared>(); - DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); - ai->store(0b0000); - - { - // Newer passive and older proactive. - auto kvr1 = kvs.getRegion(region_id); - auto r1 = proxy_instance->getRegion(region_id); - uint64_t compact_index = 10; - auto && [request, response] = MockRaftStoreProxy::composeCompactLog(r1, compact_index); - auto && [index1, term] = proxy_instance->adminCommand(region_id, std::move(request), std::move(response), 11); - kvs.setRegionCompactLogConfig(0, 0, 0, 500); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index1); - UNUSED(term); - kvs.notifyCompactLog(region_id, 1, 5, false, false); - ASSERT_EQ(r1->getApply().truncated_state().index(), compact_index); - } - - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); -} -CATCH - -TEST_F(RegionKVStoreTest, ProactiveFlushLiveness) -try -{ - auto & ctx = TiFlashTestEnv::getGlobalContext(); - auto tp = prepareForProactiveFlushTest(); - auto table_id = std::get<0>(tp); - auto region_id = std::get<1>(tp); - auto region_id2 = std::get<2>(tp); - MockRaftStoreProxy::FailCond cond; - KVStore & kvs = getKVS(); - - std::shared_ptr> ai = std::make_shared>(); - DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); - { - // A fg flush and a bg flush will not deadlock. - DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); - DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); - ai->store(0b1011); - auto f1 = [&]() { - auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); - auto k1 = RecordKVFormat::genKey(table_id, 60, 111); - // Trigger a fg flush on region_id - auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); - auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2, std::make_optional(true)); - }; - std::thread t1(f1); - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - ai->store(0b1110); - // Force bg flush. - auto f2 = [&]() { - auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); - auto k1 = RecordKVFormat::genKey(table_id, 5, 111); - // Trigger a fg flush on region_id2 - auto [index, term] = proxy_instance->rawWrite(region_id2, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); - auto [index2, term2] = proxy_instance->rawWrite(region_id2, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index2, std::make_optional(false)); - }; - std::thread t2(f2); - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); - t1.join(); - t2.join(); - ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); - // We can't assert for region_id2, since bg flush may be be finished. - } - kvs.setRegionCompactLogConfig(0, 0, 0, 500); // Every notify will take effect. - { - // Two fg flush will not deadlock. - DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); - ai->store(0b1011); - auto f1 = [&]() { - auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); - auto k1 = RecordKVFormat::genKey(table_id, 60, 111); - // Trigger a fg flush on region_id - auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); - auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); - }; - auto f2 = [&]() { - auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); - auto k1 = RecordKVFormat::genKey(table_id, 5, 111); - // Trigger a fg flush on region_id2 - auto [index, term] = proxy_instance->rawWrite(region_id2, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); - auto [index2, term2] = proxy_instance->rawWrite(region_id2, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index2); - }; - std::thread t1(f1); - std::thread t2(f2); - std::this_thread::sleep_for(std::chrono::milliseconds(200)); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); - t1.join(); - t2.join(); - ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); - ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id2)->getLatestCommitIndex()); - } - { - // An obsolete notification triggered by another region's flush shall not override. - kvs.notifyCompactLog(region_id, 1, 5, true, false); - ASSERT_EQ(proxy_instance->getRegion(region_id)->getApply().truncated_state().index(), proxy_instance->getRegion(region_id)->getLatestCommitIndex()); - } - { - // Passive flush and fg proactive flush of the same region will not deadlock, - // since they must be executed by order in one thread. - // Passive flush and fg proactive flush will not deadlock. - ai->store(0b1011); // Force fg - DB::FailPointHelper::enableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); - auto f1 = [&]() { - auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); - auto k1 = RecordKVFormat::genKey(table_id, 60, 111); - // Trigger a fg flush on region_id - auto [index, term] = proxy_instance->rawWrite(region_id, {k1}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); - auto [index2, term2] = proxy_instance->rawWrite(region_id, {k1}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2, std::make_optional(true)); - }; - auto f2 = [&]() { - auto r2 = proxy_instance->getRegion(region_id2); - auto && [request, response] = MockRaftStoreProxy::composeCompactLog(r2, 555); - auto && [index2, term] = proxy_instance->adminCommand(region_id2, std::move(request), std::move(response), 600); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index2); - }; - std::thread t1(f1); - std::thread t2(f2); - std::this_thread::sleep_for(std::chrono::milliseconds(200)); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_passive_flush_before_persist_region); - t2.join(); - ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), 555); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::pause_proactive_flush_before_persist_region); - t1.join(); - ASSERT_EQ(proxy_instance->getRegion(region_id2)->getApply().truncated_state().index(), 555); - } - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); -} -CATCH - -TEST_F(RegionKVStoreTest, ProactiveFlushRecover1) -try -{ - auto & ctx = TiFlashTestEnv::getGlobalContext(); - std::shared_ptr> ai = std::make_shared>(); - // Safe to abort between flushCache and persistRegion. - DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_force_set_type, ai); - { - auto tp = prepareForProactiveFlushTest(); - auto table_id = std::get<0>(tp); - auto region_id = std::get<1>(tp); - auto region_id2 = std::get<2>(tp); - MockRaftStoreProxy::FailCond cond; - - DB::FailPointHelper::enableFailPoint(DB::FailPoints::proactive_flush_between_persist_cache_and_region); - KVStore & kvs = getKVS(); - auto && [value_write, value_default] = proxy_instance->generateTiKVKeyValue(111, 999); - auto k2 = RecordKVFormat::genKey(table_id, 5, 111); - // Will not trigger a fg flush on region_id2 - auto [index2, term2] = proxy_instance->rawWrite(region_id2, {k2, k2}, {value_default, value_write}, {WriteCmdType::Put, WriteCmdType::Put}, {ColumnFamilyType::Default, ColumnFamilyType::Write}); - - // Abort before persistRegion, but with DM flushed. - cond.type = MockRaftStoreProxy::FailCond::Type::BEFORE_PROXY_ADVANCE; - ai->store(0b1011); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id2, index2); - - // If reload here, the data is lost since we don't persistRegion. - // However, meta is not advanced either in KVStore or Proxy. - auto & kvs2 = reloadKVSFromDisk(); - auto kvr2 = kvs2.getRegion(region_id2); - auto r2 = proxy_instance->getRegion(region_id2); - ASSERT_EQ(kvr2->appliedIndex() + 1, index2); - ASSERT_EQ(r2->getLatestAppliedIndex() + 1, index2); - - cond.type = MockRaftStoreProxy::FailCond::Type::NORMAL; - ai->store(0b1010); - // No data lost. - proxy_instance->doApply(kvs2, ctx.getTMTContext(), cond, region_id2, index2); - auto [index22, term22] = proxy_instance->rawWrite(region_id2, {k2, k2}, {value_default, value_write}, {WriteCmdType::Put, WriteCmdType::Put}, {ColumnFamilyType::Default, ColumnFamilyType::Write}); - // There is no flush after write, so will throw when duplicate key. - EXPECT_THROW(proxy_instance->doApply(kvs2, ctx.getTMTContext(), cond, region_id2, index22), Exception); - - ai->store(0b1011); - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_between_persist_cache_and_region); - auto kvr1 = kvs2.getRegion(region_id); - auto r1 = proxy_instance->getRegion(region_id); - auto && [value_write1, value_default1] = proxy_instance->generateTiKVKeyValue(111, 999); - auto k1 = RecordKVFormat::genKey(table_id, 60, 111); - // Trigger a fg flush on region_id - auto [index1, term1] = proxy_instance->rawWrite(region_id, {k1, k1}, {value_default1, value_write1}, {WriteCmdType::Put, WriteCmdType::Put}, {ColumnFamilyType::Default, ColumnFamilyType::Write}); - proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index1); - - auto & kvs3 = reloadKVSFromDisk(); - { - auto kvr1 = kvs3.getRegion(region_id); - auto r1 = proxy_instance->getRegion(region_id); - ASSERT_EQ(kvr1->appliedIndex(), r1->getLatestAppliedIndex()); - auto kvr2 = kvs3.getRegion(region_id2); - auto r2 = proxy_instance->getRegion(region_id2); - ASSERT_EQ(kvr2->appliedIndex(), r2->getLatestAppliedIndex()); - } - } - - DB::FailPointHelper::disableFailPoint(DB::FailPoints::proactive_flush_force_set_type); -} -CATCH - } // namespace tests } // namespace DB \ No newline at end of file From c1c3ebc44ffe84f07cc2e6cb1a2e397922a45bd9 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 13:57:19 +0800 Subject: [PATCH 33/66] remove fps Signed-off-by: CalvinNeo --- dbms/src/Common/FailPoint.cpp | 3 --- dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp | 1 - dbms/src/Storages/Transaction/KVStore.cpp | 3 --- dbms/src/Storages/Transaction/tests/kvstore_helper.h | 2 -- 4 files changed, 9 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 81f83bb4c3b..6853855826a 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -100,8 +100,6 @@ namespace DB M(skip_seek_before_read_dmfile) \ M(exception_after_large_write_exceed) \ M(proactive_flush_force_set_type) \ - M(proactive_flush_between_persist_cache_and_region) \ - M(proactive_flush_between_persist_regions) \ M(exception_when_fetch_disagg_pages) #define APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) \ @@ -123,7 +121,6 @@ namespace DB M(pause_when_altering_dt_store) \ M(pause_after_copr_streams_acquired) \ M(pause_query_init) \ - M(pause_proactive_flush_before_persist_region) \ M(pause_passive_flush_before_persist_region) #define APPLY_FOR_RANDOM_FAILPOINTS(M) \ diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 8ce8717143a..86f80c4c84f 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -100,7 +100,6 @@ extern const char random_exception_after_dt_write_done[]; extern const char force_slow_page_storage_snapshot_release[]; extern const char exception_before_drop_segment[]; extern const char exception_after_drop_segment[]; -extern const char pause_proactive_flush_before_persist_region[]; extern const char proactive_flush_force_set_type[]; } // namespace FailPoints diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index b83ae36bbcd..cf8158058a8 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -46,10 +46,7 @@ extern const int TABLE_IS_DROPPED; namespace FailPoints { extern const char force_fail_in_flush_region_data[]; -extern const char pause_proactive_flush_before_persist_region[]; extern const char pause_passive_flush_before_persist_region[]; -extern const char proactive_flush_between_persist_cache_and_region[]; -extern const char proactive_flush_between_persist_regions[]; } // namespace FailPoints KVStore::KVStore(Context & context) diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index 7ea0152ff3a..21daccd61b1 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -54,10 +54,8 @@ namespace FailPoints { extern const char skip_check_segment_update[]; extern const char force_fail_in_flush_region_data[]; -extern const char pause_proactive_flush_before_persist_region[]; extern const char proactive_flush_force_set_type[]; extern const char pause_passive_flush_before_persist_region[]; -extern const char proactive_flush_between_persist_cache_and_region[]; } // namespace FailPoints namespace RegionBench From 084077f120e28f493ef9f3f4cf7b7a3a4eac6221 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 15:19:31 +0800 Subject: [PATCH 34/66] f Signed-off-by: CalvinNeo --- dbms/src/Common/FailPoint.cpp | 14 +++++++------- dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp | 8 ++++---- dbms/src/Storages/Transaction/KVStore.cpp | 8 ++++---- dbms/src/Storages/Transaction/KVStore.h | 4 ++-- dbms/src/Storages/Transaction/ProxyFFI.cpp | 2 +- 5 files changed, 18 insertions(+), 18 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 4630701ed33..efe9a841fd1 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -114,13 +114,13 @@ namespace DB M(pause_before_register_non_root_mpp_task) \ M(pause_before_make_non_root_mpp_task_active) -#define APPLY_FOR_PAUSEABLE_FAILPOINTS(M) \ - M(pause_when_reading_from_dt_stream) \ - M(pause_when_writing_to_dt_store) \ - M(pause_when_ingesting_to_dt_store) \ - M(pause_when_altering_dt_store) \ - M(pause_after_copr_streams_acquired) \ - M(pause_query_init) \ +#define APPLY_FOR_PAUSEABLE_FAILPOINTS(M) \ + M(pause_when_reading_from_dt_stream) \ + M(pause_when_writing_to_dt_store) \ + M(pause_when_ingesting_to_dt_store) \ + M(pause_when_altering_dt_store) \ + M(pause_after_copr_streams_acquired) \ + M(pause_query_init) \ M(pause_passive_flush_before_persist_region) #define APPLY_FOR_RANDOM_FAILPOINTS(M) \ diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index bb8a83b74fe..629c8d428ca 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -995,7 +995,7 @@ void DeltaMergeStore::readRaw( }); auto after_segment_read = [&](const DMContextPtr & dm_context_, const SegmentPtr & segment_) { - this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read); + this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read, InputType::NotRaft); }; size_t final_num_stream = enable_read_thread ? std::max(1, num_streams) @@ -1421,8 +1421,8 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto delta_cache_limit_rows = dm_context->delta_cache_limit_rows; auto delta_cache_limit_bytes = dm_context->delta_cache_limit_bytes; - - bool should_background_compact_log = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes); + // TODO(proactive flush) + // bool should_background_compact_log = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes); bool should_background_flush = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes) // && (delta_rows - delta_last_try_flush_rows >= delta_cache_limit_rows || delta_bytes - delta_last_try_flush_bytes >= delta_cache_limit_bytes); @@ -1533,7 +1533,7 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const try_add_background_task(BackgroundTask{TaskType::Flush, dm_context, segment}); } } - // TODO Will enable once TiKV supports. + // TODO(proactive flush) // if (should_background_compact_log) // { // try_add_background_task(BackgroundTask{TaskType::FlushDTAndKVStore, dm_context, segment}); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 223bb9c7b5a..a061a85bbc5 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -312,7 +312,7 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmdInner(const WriteCmdsView & cmds, /// This call is from Proxy's applying thread of this region, so: /// 1. No other thread can write from raft to this region even if we unlocked here. /// 2. If `proactiveFlushCacheAndRegion` causes a write stall, it will be forwarded to raft layer. - // TODO We will enable this once TiKV supports. + // TODO(proactive flush) // if (write_result) // { // auto & inner = write_result.value(); @@ -380,7 +380,7 @@ void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes, U gap); } -void KVStore::persistRegion(const Region & region, std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg) +void KVStore::persistRegion(const Region & region, std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg) const { RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); if (region_task_lock.has_value()) @@ -485,7 +485,7 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl can_flush = true; } auto gap_threshold = region_compact_log_gap.load(); - + auto last_compact_log_applied = curr_region.lastCompactLogApplied(); auto current_applied_gap = index > last_compact_log_applied ? index - last_compact_log_applied : 0; @@ -1008,7 +1008,7 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) { - // TODO We will enable this once TiKV supports. + // TODO(proactive flush) UNUSED(tmt); UNUSED(rowkey_range); UNUSED(keyspace_id); diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 0b9028f589b..ebddba7c4d0 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -285,9 +285,9 @@ class KVStore final : private boost::noncopyable /// It will not check if a flush will eventually succeed. /// In other words, `canFlushRegionDataImpl(flush_if_possible=true)` can return false. bool canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term, UInt64 truncated_index, UInt64 truncated_term); - bool forceFlushRegionDataImpl(Region & curr_region, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term); + bool forceFlushRegionDataImpl(Region & curr_region, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term) const; - void persistRegion(const Region & region, std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg); + void persistRegion(const Region & region, std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg) const; void releaseReadIndexWorkers(); void handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTaskLock &); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index eaec04fb436..1b7ea9feeb9 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -150,7 +150,7 @@ uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t try { auto & kvstore = server->tmt->getKVStore(); - // TODO truncated_index is always 0, since not support in Proxy. + // TODO(proactive flush) truncated_index is 0 for now. return kvstore->tryFlushRegionData(region_id, false, flush_pattern, *server->tmt, index, term, truncated_index, truncated_term); } catch (...) From 6317985d29b7757239f012ac354661f8fb344a5a Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 16:08:23 +0800 Subject: [PATCH 35/66] f Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 10 ---------- dbms/src/Storages/Transaction/KVStore.h | 3 ++- 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index a061a85bbc5..010cb236569 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -1005,14 +1005,4 @@ FileUsageStatistics KVStore::getFileUsageStatistics() const return region_persister->getFileUsageStatistics(); } - -void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background) -{ - // TODO(proactive flush) - UNUSED(tmt); - UNUSED(rowkey_range); - UNUSED(keyspace_id); - UNUSED(table_id); - UNUSED(is_background); -} } // namespace DB diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index ebddba7c4d0..7659bc642a4 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -213,7 +213,8 @@ class KVStore final : private boost::noncopyable FileUsageStatistics getFileUsageStatistics() const; - void proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background); + // TODO(proactive flush) + // void proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background); void notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held = true); #ifndef DBMS_PUBLIC_GTEST private: From 74d78ffeaa5964867c0ef90cc779616e7a7c0bf4 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 16:31:43 +0800 Subject: [PATCH 36/66] a Signed-off-by: CalvinNeo --- .../DeltaMerge/DeltaMergeStore_InternalBg.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index 033bb526dac..df1a51e4e79 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -400,7 +400,13 @@ bool DeltaMergeStore::handleBackgroundTask(bool heavy) type = ThreadType::BG_Flush; break; case TaskType::FlushDTAndKVStore: - triggerCompactLog(task.dm_context, task.segment->getRowKeyRange(), true); + { + // TODO(proactive flush) + // auto & tmt = task.dm_context->db_context.getTMTContext(); + // auto & kv_store = tmt.getKVStore(); + // kv_store->proactiveFlushCacheAndRegion(tmt, task.segment->getRowKeyRange(), keyspace_id, physical_table_id, true); + break; + } case TaskType::PlaceIndex: task.segment->placeDeltaIndex(*task.dm_context); break; @@ -920,13 +926,5 @@ UInt64 DeltaMergeStore::onSyncGc(Int64 limit, const GCOptions & gc_options) return gc_segments_num; } - -void DeltaMergeStore::triggerCompactLog(const DMContextPtr & dm_context, const RowKeyRange & range, bool is_background) const -{ - auto & tmt = dm_context->db_context.getTMTContext(); - auto & kv_store = tmt.getKVStore(); - - kv_store->proactiveFlushCacheAndRegion(tmt, range, keyspace_id, physical_table_id, is_background); -} } // namespace DM } // namespace DB From afe4c92f081d3f64d16800034cc1f2b29b6991db Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 16:38:50 +0800 Subject: [PATCH 37/66] fff Signed-off-by: CalvinNeo --- dbms/src/Common/TiFlashMetrics.h | 6 +----- dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp | 1 + 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 895c4670981..eaed59efab6 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -158,11 +158,7 @@ namespace DB F(type_seg_split_fg, {{"type", "seg_split_fg"}}, ExpBuckets{0.001, 2, 20}), \ F(type_seg_split_ingest, {{"type", "seg_split_ingest"}}, ExpBuckets{0.001, 2, 20}), \ F(type_seg_merge_bg_gc, {{"type", "seg_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_bg, {{"type", "compact_log_bg"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_fg, {{"type", "compact_log_fg"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_fg_dm, {{"type", "compact_log_fg_dm"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_compact_log_bg_dm, {{"type", "compact_log_bg_dm"}}, ExpBuckets{0.001, 2, 20})) \ + F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_storage_throughput_bytes, "Calculate the throughput of tasks of storage in bytes", Gauge, /**/ \ F(type_write, {"type", "write"}), /**/ \ F(type_ingest, {"type", "ingest"}), /**/ \ diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index df1a51e4e79..66cc99f14d9 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -405,6 +405,7 @@ bool DeltaMergeStore::handleBackgroundTask(bool heavy) // auto & tmt = task.dm_context->db_context.getTMTContext(); // auto & kv_store = tmt.getKVStore(); // kv_store->proactiveFlushCacheAndRegion(tmt, task.segment->getRowKeyRange(), keyspace_id, physical_table_id, true); + // task.segment->placeDeltaIndex(*task.dm_context); break; } case TaskType::PlaceIndex: From 32fac3fc88f4ea70e9e7c2101d7330ff995a5a81 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 16:53:01 +0800 Subject: [PATCH 38/66] add more logs Signed-off-by: CalvinNeo --- dbms/src/Storages/DeltaMerge/DeltaMergeStore.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 8286b25cab8..6e916a8d362 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -537,6 +537,9 @@ class DeltaMergeStore : private boost::noncopyable void waitForDeleteRange(const DMContextPtr & context, const SegmentPtr & segment); + /// Should be called after every write into DeltaMergeStore. + /// If the delta cache reaches the foreground flush limit, it will also trigger a KVStore flush of releated regions, + /// by returning a non-empty DM::WriteResult. // Deferencing `Iter` can get a pointer to a Segment. template DM::WriteResult checkSegmentsUpdateForKVStore(const DMContextPtr & context, Iter begin, Iter end, ThreadType thread_type, InputType input_type) @@ -733,9 +736,10 @@ class DeltaMergeStore : private boost::noncopyable * This may be called from multiple threads, e.g. at the foreground write moment, or in background threads. * A `thread_type` should be specified indicating the type of the thread calling this function. * Depend on the thread type, the "update" to do may be varied. + * + * It returns a bool which indicates whether a flush of KVStore is recommended. */ bool checkSegmentUpdate(const DMContextPtr & context, const SegmentPtr & segment, ThreadType thread_type, InputType input_type); - void triggerCompactLog(const DMContextPtr & dm_context, const RowKeyRange & range, bool is_background) const; #ifndef DBMS_PUBLIC_GTEST private: #else From 3cb8c76e3b9bddb088d309be1b60a5ed128b9352 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 4 Aug 2023 17:23:50 +0800 Subject: [PATCH 39/66] add more logs Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index 3aeeaf26aea..8eb94f27b9a 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -76,5 +76,8 @@ try } } CATCH + +// TODO(proactive flush) + } // namespace tests } // namespace DB \ No newline at end of file From 3d63b20153cc52e47faf0b42b16e3a2f05073c75 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 7 Aug 2023 13:13:53 +0800 Subject: [PATCH 40/66] a Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 1 + dbms/src/Storages/Transaction/Region.cpp | 7 ------- dbms/src/Storages/Transaction/RegionPersister.cpp | 10 ++++++++++ 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 010cb236569..c308e7f6831 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -397,6 +397,7 @@ void KVStore::persistRegion(const Region & region, std::optionalpersist(region); LOG_INFO(log, "After persisted {}, cache {} bytes", region.toString(false), region.dataSize()); } + switch (reason) { case PersistRegionReason::UselessAdminCommand: diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 66814402d0d..55893cca873 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -497,13 +497,6 @@ std::string Region::dataInfo() const void Region::markCompactLog() const { last_compact_log_time = Clock::now(); - uint64_t current_applied_index = 0; - if (last_compact_log_applied != 0) - { - uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; - GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); - } - last_compact_log_applied = current_applied_index; } Timepoint Region::lastCompactLogTime() const diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index ee548e83299..17747b3ec94 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -112,6 +113,15 @@ void RegionPersister::doPersist(RegionCacheWriteElement & region_write_buffer, c DB::WriteBatchWrapper wb{run_mode, getWriteBatchPrefix()}; wb.putPage(region_id, applied_index, read_buf, region_size); page_writer->write(std::move(wb), global_context.getWriteLimiter()); + + uint64_t current_applied_index = 0; + uint64_t last_compact_log_applied = region.lastCompactLogApplied(); + if (last_compact_log_applied != 0) + { + uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; + GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); + } + last_compact_log_applied = current_applied_index; } RegionPersister::RegionPersister(Context & global_context_, const RegionManager & region_manager_) From fd0a5f101c1017c5365a02b7037f1fea4c806710 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 7 Aug 2023 14:38:40 +0800 Subject: [PATCH 41/66] a Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/RegionPersister.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index 17747b3ec94..7b4f275a605 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -28,7 +29,6 @@ #include #include #include -#include #include #include From 7636776117f8b5244c36c29a2b261420be3debbd Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Mon, 7 Aug 2023 14:54:56 +0800 Subject: [PATCH 42/66] Update dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h Co-authored-by: jinhelin --- dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h b/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h index 9e87ec8763c..9b95d465594 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h @@ -52,7 +52,7 @@ struct RaftWriteResult return *this; } }; -typedef std::optional WriteResult; +using WriteResult = std::optional; static_assert(std::is_move_constructible_v); } // namespace DM } // namespace DB From c8d4f3797231af22e111499ac5d291dc8ed86da3 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 8 Aug 2023 12:41:51 +0800 Subject: [PATCH 43/66] f Signed-off-by: CalvinNeo --- dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp | 4 +++- dbms/src/Storages/Transaction/KVStore.cpp | 8 ++++++-- dbms/src/Storages/Transaction/RegionManager.h | 3 +-- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 629c8d428ca..206fae190dc 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1462,7 +1462,9 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const fiu_do_on(FailPoints::force_triggle_background_merge_delta, { should_background_merge_delta = true; }); fiu_do_on(FailPoints::proactive_flush_force_set_type, { - // | set bg bit | bg value bit | set fg bit | fg value bit| + // If bg/fg modify bit is set, we will perform background/foreground flush. + // Otherwise, it depends by the original logic. + // | bg modify bit | bg value bit | fg modify bit | fg value bit| if (auto v = FailPointHelper::getFailPointVal(FailPoints::proactive_flush_force_set_type); v) { auto set_kind = std::any_cast>>(v.value()); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index c308e7f6831..02d833a72c9 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -491,7 +491,8 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl auto current_applied_gap = index > last_compact_log_applied ? index - last_compact_log_applied : 0; // TODO We will use truncated_index once Proxy/TiKV supports. - if (index > last_compact_log_applied + gap_threshold) + // After restart, last_compact_log_applied is 0, we don't trigger immediately. + if (last_compact_log_applied && index > last_compact_log_applied + gap_threshold) { GET_METRIC(tiflash_raft_raft_events_count, type_flush_log_gap).Increment(1); can_flush = true; @@ -499,7 +500,10 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl GET_METRIC(tiflash_raft_raft_events_count, type_pre_exec_compact).Increment(1); // GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); - GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(current_applied_gap); + if (last_compact_log_applied) + { + GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(current_applied_gap); + } LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}, gap {}/{}", curr_region.toString(false), rows, size_bytes, current_applied_gap, gap_threshold); if (can_flush && flush_if_possible) diff --git a/dbms/src/Storages/Transaction/RegionManager.h b/dbms/src/Storages/Transaction/RegionManager.h index 09e2f6b1ba7..c375488a79b 100644 --- a/dbms/src/Storages/Transaction/RegionManager.h +++ b/dbms/src/Storages/Transaction/RegionManager.h @@ -26,8 +26,7 @@ class RegionTaskLock; struct RegionTaskCtrl : MutexLockWrap { - // TODO This lock may be changed back to simple mutex. - typedef std::recursive_mutex Mut; + using Mut = std::mutex; /// The life time of each RegionTaskElement element should be as long as RegionManager, just return const ref. struct RegionTaskElement : private boost::noncopyable { From 01abeb0d977f956a84dcf9825de2377585b086e8 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 8 Aug 2023 13:58:24 +0800 Subject: [PATCH 44/66] applied Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/Region.cpp | 5 +++++ dbms/src/Storages/Transaction/Region.h | 1 + 2 files changed, 6 insertions(+) diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 55893cca873..9d42dfb27f7 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -509,6 +509,11 @@ UInt64 Region::lastCompactLogApplied() const return last_compact_log_applied; } +void Region::setLastCompactLogApplied(UInt64 new_value) +{ + last_compact_log_applied = new_value; +} + Region::CommittedScanner Region::createCommittedScanner(bool use_lock, bool need_value) { return Region::CommittedScanner(this->shared_from_this(), use_lock, need_value); diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index a7470976945..f40fdf40be8 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -152,6 +152,7 @@ class Region : public std::enable_shared_from_this void markCompactLog() const; Timepoint lastCompactLogTime() const; UInt64 lastCompactLogApplied() const; + void setLastCompactLogApplied(UInt64 new_value); friend bool operator==(const Region & region1, const Region & region2) { From 97261a962ff84e792c1fa18342946d92c4faa8de Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 8 Aug 2023 14:00:32 +0800 Subject: [PATCH 45/66] applied 2 Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/RegionPersister.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index 7b4f275a605..55f6cca4288 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -121,7 +121,7 @@ void RegionPersister::doPersist(RegionCacheWriteElement & region_write_buffer, c uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); } - last_compact_log_applied = current_applied_index; + region.setLastCompactLogApplied(current_applied_index); } RegionPersister::RegionPersister(Context & global_context_, const RegionManager & region_manager_) From 2a36c953f94351326f26c4d9693fcde151439ba9 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 8 Aug 2023 14:04:27 +0800 Subject: [PATCH 46/66] applied 3 Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/RegionPersister.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index 55f6cca4288..c2b03c5735e 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -114,7 +114,7 @@ void RegionPersister::doPersist(RegionCacheWriteElement & region_write_buffer, c wb.putPage(region_id, applied_index, read_buf, region_size); page_writer->write(std::move(wb), global_context.getWriteLimiter()); - uint64_t current_applied_index = 0; + uint64_t current_applied_index = region.appliedIndex(); uint64_t last_compact_log_applied = region.lastCompactLogApplied(); if (last_compact_log_applied != 0) { From 05b5f437ad39424cba2a17dca207d3a257bbfce0 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 8 Aug 2023 14:27:35 +0800 Subject: [PATCH 47/66] applied 4 Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/Region.cpp | 10 ++++++++-- dbms/src/Storages/Transaction/Region.h | 3 ++- dbms/src/Storages/Transaction/RegionPersister.cpp | 9 +-------- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 9d42dfb27f7..b8ceb21851e 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -509,9 +509,15 @@ UInt64 Region::lastCompactLogApplied() const return last_compact_log_applied; } -void Region::setLastCompactLogApplied(UInt64 new_value) +void Region::updateLastCompactLogApplied() const { - last_compact_log_applied = new_value; + uint64_t current_applied_index = appliedIndex(); + if (last_compact_log_applied != 0) + { + uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; + GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); + } + last_compact_log_applied = current_applied_index; } Region::CommittedScanner Region::createCommittedScanner(bool use_lock, bool need_value) diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index f40fdf40be8..1681b85b5b9 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -152,7 +152,8 @@ class Region : public std::enable_shared_from_this void markCompactLog() const; Timepoint lastCompactLogTime() const; UInt64 lastCompactLogApplied() const; - void setLastCompactLogApplied(UInt64 new_value); + // Must hold region lock. + void updateLastCompactLogApplied() const; friend bool operator==(const Region & region1, const Region & region2) { diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index c2b03c5735e..4a5baad1f2d 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -114,14 +114,7 @@ void RegionPersister::doPersist(RegionCacheWriteElement & region_write_buffer, c wb.putPage(region_id, applied_index, read_buf, region_size); page_writer->write(std::move(wb), global_context.getWriteLimiter()); - uint64_t current_applied_index = region.appliedIndex(); - uint64_t last_compact_log_applied = region.lastCompactLogApplied(); - if (last_compact_log_applied != 0) - { - uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; - GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); - } - region.setLastCompactLogApplied(current_applied_index); + region.updateLastCompactLogApplied(); } RegionPersister::RegionPersister(Context & global_context_, const RegionManager & region_manager_) From 81b4b5af79bc0fc7981bfd1c21a9e89bd5fdc3c4 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 8 Aug 2023 15:40:46 +0800 Subject: [PATCH 48/66] apply 5 Signed-off-by: CalvinNeo --- dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 8 ++++---- dbms/src/Storages/Transaction/Region.cpp | 5 +++++ dbms/src/Storages/Transaction/Region.h | 1 + 4 files changed, 11 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 206fae190dc..c26adf6eaa8 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1384,7 +1384,7 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const fiu_do_on(FailPoints::skip_check_segment_update, { return should_trigger_foreground_kvstore_flush; }); if (segment->hasAbandoned()) - return should_trigger_foreground_kvstore_flush; + return false; const auto & delta = segment->getDelta(); size_t delta_saved_rows = delta->getRows(/* use_unsaved */ false); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 02d833a72c9..bb098c0db7c 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -497,13 +497,13 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl GET_METRIC(tiflash_raft_raft_events_count, type_flush_log_gap).Increment(1); can_flush = true; } + if (!last_compact_log_applied) + { + curr_region.setLastCompactLogApplied(last_compact_log_applied); + } GET_METRIC(tiflash_raft_raft_events_count, type_pre_exec_compact).Increment(1); // GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); - if (last_compact_log_applied) - { - GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(current_applied_gap); - } LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}, gap {}/{}", curr_region.toString(false), rows, size_bytes, current_applied_gap, gap_threshold); if (can_flush && flush_if_possible) diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index b8ceb21851e..56ddd637b0c 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -509,6 +509,11 @@ UInt64 Region::lastCompactLogApplied() const return last_compact_log_applied; } +void Region::setLastCompactLogApplied(UInt64 new_value) const +{ + last_compact_log_applied = new_value; +} + void Region::updateLastCompactLogApplied() const { uint64_t current_applied_index = appliedIndex(); diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index 1681b85b5b9..d20975116f8 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -152,6 +152,7 @@ class Region : public std::enable_shared_from_this void markCompactLog() const; Timepoint lastCompactLogTime() const; UInt64 lastCompactLogApplied() const; + void setLastCompactLogApplied(UInt64 new_value) const; // Must hold region lock. void updateLastCompactLogApplied() const; From 427e8c379153cdf19ab3364957ef171dccfec3f6 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 8 Aug 2023 16:21:51 +0800 Subject: [PATCH 49/66] metrics Signed-off-by: CalvinNeo --- metrics/grafana/tiflash_summary.json | 204 +++++++++++++++++++++++++-- 1 file changed, 193 insertions(+), 11 deletions(-) diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 432b53e249a..66582aad7dd 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1689824531144, + "iteration": 1691482557821, "links": [], "panels": [ { @@ -10537,6 +10537,188 @@ "yBucketNumber": null, "yBucketSize": null }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 44 + }, + "hiddenSeries": false, + "id": 235, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_raft_raft_events_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[$__rate_interval])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Events QPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:91", + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:92", + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "timeseries", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 44 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 234, + "legend": { + "show": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_raft_log_lag_count_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"applied_index\"}[1m])) by (le, type)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(delta(tiflash_raft_raft_log_lag_count_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"compact_index\"}[1m])) by (le, type)", + "hide": false, + "interval": "", + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Raft Log Gap Heatmap", + "tooltip": { + "show": true, + "showHistogram": true + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 0, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "cards": { "cardPadding": null, @@ -10561,7 +10743,7 @@ "h": 7, "w": 12, "x": 0, - "y": 44 + "y": 51 }, "heatmap": {}, "hideZeroBuckets": true, @@ -10631,7 +10813,7 @@ "h": 7, "w": 12, "x": 12, - "y": 44 + "y": 51 }, "heatmap": {}, "hideZeroBuckets": true, @@ -10695,7 +10877,7 @@ "h": 7, "w": 24, "x": 0, - "y": 51 + "y": 58 }, "height": "", "hiddenSeries": false, @@ -10809,7 +10991,7 @@ "h": 7, "w": 12, "x": 0, - "y": 58 + "y": 65 }, "heatmap": {}, "hideZeroBuckets": true, @@ -10878,7 +11060,7 @@ "h": 7, "w": 12, "x": 12, - "y": 58 + "y": 65 }, "heatmap": {}, "hideZeroBuckets": true, @@ -10948,7 +11130,7 @@ "h": 7, "w": 12, "x": 0, - "y": 65 + "y": 72 }, "heatmap": {}, "hideZeroBuckets": true, @@ -11018,7 +11200,7 @@ "h": 7, "w": 12, "x": 12, - "y": 65 + "y": 72 }, "heatmap": {}, "hideZeroBuckets": true, @@ -11088,7 +11270,7 @@ "h": 7, "w": 12, "x": 0, - "y": 72 + "y": 79 }, "heatmap": {}, "hideZeroBuckets": true, @@ -11154,7 +11336,7 @@ "h": 7, "w": 12, "x": 12, - "y": 72 + "y": 79 }, "hiddenSeries": false, "id": 91, @@ -15073,4 +15255,4 @@ "title": "Test-Cluster-TiFlash-Summary", "uid": "SVbh2xUWk", "version": 1 -} +} \ No newline at end of file From 623db32422eb488539b02d63ad13337092360203 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 8 Aug 2023 16:43:47 +0800 Subject: [PATCH 50/66] f Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index bb098c0db7c..41aa46a094a 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -499,7 +499,8 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl } if (!last_compact_log_applied) { - curr_region.setLastCompactLogApplied(last_compact_log_applied); + // If we just experienced a restart, we will set `last_compact_log_applied`. + curr_region.setLastCompactLogApplied(index); } GET_METRIC(tiflash_raft_raft_events_count, type_pre_exec_compact).Increment(1); From 4a514ee168a4e32968ba18c4b684705521d1c30e Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Wed, 9 Aug 2023 17:42:06 +0800 Subject: [PATCH 51/66] ffff Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 2 +- dbms/src/Storages/Transaction/Region.cpp | 1 + dbms/src/Storages/Transaction/TMTContext.cpp | 2 +- .../Storages/Transaction/tests/gtest_new_kvstore.cpp | 11 ++++++++++- dbms/src/Storages/Transaction/tests/kvstore_helper.h | 4 ++-- 5 files changed, 15 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 41aa46a094a..858356333d7 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -499,7 +499,7 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl } if (!last_compact_log_applied) { - // If we just experienced a restart, we will set `last_compact_log_applied`. + // If we just experienced a restart, we will set `last_compact_log_applied` to current applied_index. curr_region.setLastCompactLogApplied(index); } diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 56ddd637b0c..df7efde5288 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -416,6 +416,7 @@ RegionPtr Region::deserialize(ReadBuffer & buf, const TiFlashRaftProxyHelper * p auto region = std::make_shared(std::move(meta), proxy_helper); RegionData::deserialize(buf, region->data); + region->setLastCompactLogApplied(region->appliedIndex()); return region; } diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index d1f18d3eb33..18a386e6118 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -351,7 +351,7 @@ void TMTContext::reloadConfig(const Poco::Util::AbstractConfiguration & config) getKVStore()->setRegionCompactLogConfig(std::max(config.getUInt64(COMPACT_LOG_MIN_PERIOD, 120), 1), std::max(config.getUInt64(COMPACT_LOG_MIN_ROWS, 40 * 1024), 1), std::max(config.getUInt64(COMPACT_LOG_MIN_BYTES, 32 * 1024 * 1024), 1), - std::max(config.getUInt64(COMPACT_LOG_MIN_GAP, 500), 1)); + std::max(config.getUInt64(COMPACT_LOG_MIN_GAP, 200), 1)); { batch_read_index_timeout_ms = config.getUInt64(BATCH_READ_INDEX_TIMEOUT_MS, DEFAULT_BATCH_READ_INDEX_TIMEOUT_MS); wait_index_timeout_ms = config.getUInt64(WAIT_INDEX_TIMEOUT_MS, DEFAULT_WAIT_INDEX_TIMEOUT_MS); diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index 972e33a973c..6bd614a13e5 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -74,15 +74,24 @@ try tryPersistRegion(kvs, region_id); } { + MockRaftStoreProxy::FailCond cond; KVStore & kvs = reloadKVSFromDisk(); auto kvr1 = kvs.getRegion(region_id); auto r1 = proxy_instance->getRegion(region_id); + ASSERT_EQ(kvr1->lastCompactLogApplied(), 5); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index); ASSERT_EQ(kvr1->appliedIndex(), applied_index); ASSERT_EQ(kvr1->appliedIndex(), r1->getLatestCommitIndex() - 1); proxy_instance->replay(kvs, ctx.getTMTContext(), region_id, r1->getLatestCommitIndex()); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 1); ASSERT_EQ(kvr1->appliedIndex(), applied_index + 1); + + auto && [req, res] = MockRaftStoreProxy::composeCompactLog(r1, kvr1->appliedIndex()); + auto [indexc, termc] = proxy_instance->adminCommand(region_id, std::move(req), std::move(res), std::nullopt); + // Reject compact log. + kvs.setRegionCompactLogConfig(10000000, 10000000, 10000000, 10000000); + proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, indexc); + ASSERT_EQ(kvr1->lastCompactLogApplied(), indexc); } } @@ -105,7 +114,7 @@ try ASSERT_EQ(kvr1->appliedIndex(), applied_index); ASSERT_NE(kvr1->appliedIndex(), index); // The persisted applied_index is `applied_index`. - kvs.tryPersistRegion(region_id); + tryPersistRegion(kvs, region_id); } { KVStore & kvs = reloadKVSFromDisk(); diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index 6a3b1a2b1a1..573f666f670 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -112,7 +112,7 @@ class RegionKVStoreTest : public ::testing::Test auto & global_ctx = TiFlashTestEnv::getGlobalContext(); global_ctx.tryReleaseWriteNodePageStorageForTest(); global_ctx.initializeWriteNodePageStorageIfNeed(*path_pool); - kvstore = std::make_unique(global_ctx); + kvstore = std::make_shared(global_ctx); // only recreate kvstore and restore data from disk, don't recreate proxy instance kvstore->restore(*path_pool, proxy_helper.get()); return *kvstore; @@ -174,7 +174,7 @@ class RegionKVStoreTest : public ::testing::Test { if (auto region = kvs.getRegion(region_id); region) { - kvs.persistRegion(*region, std::nullopt, ""); + kvs.persistRegion(*region, std::nullopt, PersistRegionReason::Debug, ""); } } From 6bb12a4fb70a09c6daf381f78cc0f198d728e571 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Wed, 9 Aug 2023 19:15:20 +0800 Subject: [PATCH 52/66] remove all commented Signed-off-by: CalvinNeo --- dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp | 5 ----- .../Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp | 4 ---- dbms/src/Storages/Transaction/KVStore.cpp | 8 -------- dbms/src/Storages/Transaction/ProxyFFI.cpp | 1 - dbms/src/Storages/Transaction/RegionMeta.cpp | 5 +++++ dbms/src/Storages/Transaction/RegionMeta.h | 2 +- 6 files changed, 6 insertions(+), 19 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index c26adf6eaa8..d6c3d348ee2 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1422,7 +1422,6 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto delta_cache_limit_bytes = dm_context->delta_cache_limit_bytes; // TODO(proactive flush) - // bool should_background_compact_log = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes); bool should_background_flush = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes) // && (delta_rows - delta_last_try_flush_rows >= delta_cache_limit_rows || delta_bytes - delta_last_try_flush_bytes >= delta_cache_limit_bytes); @@ -1536,10 +1535,6 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const } } // TODO(proactive flush) - // if (should_background_compact_log) - // { - // try_add_background_task(BackgroundTask{TaskType::FlushDTAndKVStore, dm_context, segment}); - // } } // Need to check the latest delta (maybe updated after foreground flush). If it is updating by another thread, diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index 66cc99f14d9..dd420d3bbc7 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -402,10 +402,6 @@ bool DeltaMergeStore::handleBackgroundTask(bool heavy) case TaskType::FlushDTAndKVStore: { // TODO(proactive flush) - // auto & tmt = task.dm_context->db_context.getTMTContext(); - // auto & kv_store = tmt.getKVStore(); - // kv_store->proactiveFlushCacheAndRegion(tmt, task.segment->getRowKeyRange(), keyspace_id, physical_table_id, true); - // task.segment->placeDeltaIndex(*task.dm_context); break; } case TaskType::PlaceIndex: diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 858356333d7..3b1a21b2c5e 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -313,14 +313,6 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmdInner(const WriteCmdsView & cmds, /// 1. No other thread can write from raft to this region even if we unlocked here. /// 2. If `proactiveFlushCacheAndRegion` causes a write stall, it will be forwarded to raft layer. // TODO(proactive flush) - // if (write_result) - // { - // auto & inner = write_result.value(); - // for (auto it = inner.pending_flush_ranges.begin(); it != inner.pending_flush_ranges.end(); it++) - // { - // proactiveFlushCacheAndRegion(tmt, *it, inner.keyspace_id, inner.table_id, false); - // } - // } return res; } diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 1b7ea9feeb9..7a33a938a98 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -150,7 +150,6 @@ uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t try { auto & kvstore = server->tmt->getKVStore(); - // TODO(proactive flush) truncated_index is 0 for now. return kvstore->tryFlushRegionData(region_id, false, flush_pattern, *server->tmt, index, term, truncated_index, truncated_term); } catch (...) diff --git a/dbms/src/Storages/Transaction/RegionMeta.cpp b/dbms/src/Storages/Transaction/RegionMeta.cpp index 11e0f300b1c..4912d90a4a4 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.cpp +++ b/dbms/src/Storages/Transaction/RegionMeta.cpp @@ -471,4 +471,9 @@ const raft_serverpb::MergeState & RegionMeta::getMergeState() const std::lock_guard lock(mutex); return region_state.getMergeState(); } + +const RegionState & RegionMeta::getRegionState() const { + std::lock_guard lock(mutex); + return region_state; +} } // namespace DB diff --git a/dbms/src/Storages/Transaction/RegionMeta.h b/dbms/src/Storages/Transaction/RegionMeta.h index 7f7ecb53371..a6db236d9c5 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.h +++ b/dbms/src/Storages/Transaction/RegionMeta.h @@ -113,7 +113,7 @@ class RegionMeta metapb::Region cloneMetaRegion() const; const raft_serverpb::MergeState & getMergeState() const; raft_serverpb::MergeState cloneMergeState() const; - const RegionState & getRegionState() const { return region_state; }; + const RegionState & getRegionState() const; RegionMeta() = delete; From 1ebb2fe895c661a4f3ddc95ae4914eba280715da Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Thu, 10 Aug 2023 00:06:12 +0800 Subject: [PATCH 53/66] proxy Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 745272e78ab..edcc0a665ec 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 745272e78ab590befb41bed248dd9e06a5bd9d9f +Subproject commit edcc0a665ecb42ef1c2c07ebcf1714dca8e44d1c From 6bd64926bd53ffd420c2321a4a5bb7cfdaf358bb Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Thu, 10 Aug 2023 15:48:40 +0800 Subject: [PATCH 54/66] Update dbms/src/Storages/Transaction/RegionPersister.cpp Co-authored-by: JaySon --- dbms/src/Storages/Transaction/RegionPersister.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index 4a5baad1f2d..abca4971df1 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include #include From 14b74fb26ca6cc525a84bbd29b492f26b0ddd142 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Thu, 10 Aug 2023 15:48:54 +0800 Subject: [PATCH 55/66] Update dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp Co-authored-by: JaySon --- dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 034790e62ad..c3f0d2a0a26 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -1206,7 +1206,7 @@ TEST_F(RegionKVStoreTest, RegionRange) auto res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); ASSERT_EQ(res.size(), 3); auto res2 = region_index.findByRangeChecked(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); - ASSERT(std::holds_alternative(res2)); + ASSERT_TRUE(std::holds_alternative(res2)); region_index.add(makeRegion(4, RecordKVFormat::genKey(1, 1), RecordKVFormat::genKey(1, 4))); From 313fade9c338582c48da8d05091863f89b766751 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 11 Aug 2023 11:17:19 +0800 Subject: [PATCH 56/66] f Signed-off-by: CalvinNeo --- dbms/src/Debug/dbgTools.cpp | 3 +-- dbms/src/Storages/Transaction/RegionMeta.cpp | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Debug/dbgTools.cpp b/dbms/src/Debug/dbgTools.cpp index 57f54b4a373..6e89f6958ed 100644 --- a/dbms/src/Debug/dbgTools.cpp +++ b/dbms/src/Debug/dbgTools.cpp @@ -644,8 +644,7 @@ EngineStoreApplyRes applyWriteRaftCmd( UInt64 index, UInt64 term, TMTContext & tmt, - DM::WriteResult * write_result_ptr -) + DM::WriteResult * write_result_ptr) { std::vector keys; std::vector vals; diff --git a/dbms/src/Storages/Transaction/RegionMeta.cpp b/dbms/src/Storages/Transaction/RegionMeta.cpp index 4912d90a4a4..4fe8936ce7e 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.cpp +++ b/dbms/src/Storages/Transaction/RegionMeta.cpp @@ -472,7 +472,8 @@ const raft_serverpb::MergeState & RegionMeta::getMergeState() const return region_state.getMergeState(); } -const RegionState & RegionMeta::getRegionState() const { +const RegionState & RegionMeta::getRegionState() const +{ std::lock_guard lock(mutex); return region_state; } From f6240337efbf258da6a9d020c8f92a3359649084 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 11 Aug 2023 14:50:46 +0800 Subject: [PATCH 57/66] addr cmt Signed-off-by: CalvinNeo --- .../Storages/DeltaMerge/DeltaMergeInterfaces.h | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h b/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h index 9b95d465594..d3e0a07d66a 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeInterfaces.h @@ -37,20 +37,9 @@ struct RaftWriteResult DISALLOW_COPY(RaftWriteResult); - RaftWriteResult(RaftWriteResult && other) - { - pending_flush_ranges = std::move(other.pending_flush_ranges); - keyspace_id = other.keyspace_id; - table_id = other.table_id; - } - - RaftWriteResult & operator=(RaftWriteResult && other) - { - pending_flush_ranges = std::move(other.pending_flush_ranges); - keyspace_id = other.keyspace_id; - table_id = other.table_id; - return *this; - } + RaftWriteResult(RaftWriteResult && other) = default; + + RaftWriteResult & operator=(RaftWriteResult && other) = default; }; using WriteResult = std::optional; static_assert(std::is_move_constructible_v); From a340c741a577ba5bf1a3844fd97b8fd1068513cc Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 11 Aug 2023 17:15:35 +0800 Subject: [PATCH 58/66] fix some tests Signed-off-by: CalvinNeo --- dbms/src/Common/TiFlashMetrics.h | 3 ++- dbms/src/Storages/Transaction/KVStore.cpp | 1 + dbms/src/Storages/Transaction/Region.cpp | 5 +---- dbms/src/Storages/Transaction/RegionData.cpp | 4 ++-- dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp | 2 +- .../Transaction/tests/gtest_kvstore_fast_add_peer.cpp | 1 + dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index eaed59efab6..ec99eab341b 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -265,7 +265,8 @@ namespace DB F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ M(tiflash_raft_raft_log_lag_count, "Bucketed histogram raft index lag", Histogram, \ F(type_compact_index, {{"type", "compact_index"}}, EqualWidthBuckets{0, 200, 5}), \ - F(type_applied_index, {{"type", "applied_index"}}, EqualWidthBuckets{0, 200, 5})) \ + F(type_applied_index, {{"type", "applied_index"}}, EqualWidthBuckets{0, 200, 5}), \ + F(type_unflushed_applied_index, {{"type", "unflushed_applied_index"}}, EqualWidthBuckets{0, 200, 5})) \ M(tiflash_raft_raft_events_count, "Raft event counter", Counter, \ F(type_pre_exec_compact, {{"type", "pre_exec_compact"}}), \ F(type_flush_apply_snapshot, {{"type", "flush_apply_snapshot"}}), \ diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 7361dc277b2..cb13ca0379b 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -452,6 +452,7 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl else { GET_METRIC(tiflash_raft_region_flush_size, type_unflushed).Observe(size_bytes); + GET_METRIC(tiflash_raft_raft_log_lag_count, type_unflushed_applied_index).Observe(current_applied_gap); } return can_flush; } diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index d4b5fc5ad12..37195b4626f 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -57,10 +57,7 @@ std::optional Region::readDataByWriteIt(const RegionData::Co } catch (DB::Exception & e) { - e.addMessage(fmt::format("(region id {}, applied_index:{}, applied_term:{})", - meta.regionId(), - appliedIndex(), - appliedIndexTerm())); + e.addMessage(fmt::format("(applied_term: {})", appliedIndexTerm())); throw; } } diff --git a/dbms/src/Storages/Transaction/RegionData.cpp b/dbms/src/Storages/Transaction/RegionData.cpp index 6470d2c5fcb..f3d2078cb43 100644 --- a/dbms/src/Storages/Transaction/RegionData.cpp +++ b/dbms/src/Storages/Transaction/RegionData.cpp @@ -177,13 +177,13 @@ std::optional RegionData::readDataByWriteIt(const ConstWrite } if (!hard_error) { - orphan_key_debug_msg = fmt::format("{}, snapshot_index: {}, {}, orphan key size {}", + orphan_key_debug_msg = fmt::format("orphan_info: ({}, snapshot_index: {}, {}, orphan key size {})", hard_error ? "" : ", not orphan key", orphan_keys_info.snapshot_index.has_value() ? std::to_string(orphan_keys_info.snapshot_index.value()) : "", orphan_keys_info.removed_remained_keys.contains(*key) ? "duplicated write" : "missing default", orphan_keys_info.remainedKeyCount()); } - throw Exception(fmt::format("Raw TiDB PK: {}, Prewrite ts: {} can not found in default cf for key: {}, region_id: {}, applied: {}{}", + throw Exception(fmt::format("Raw TiDB PK: {}, Prewrite ts: {} can not found in default cf for key: {}, region_id: {}, applied_index: {}{}", pk.toDebugString(), decoded_val.prewrite_ts, key->toDebugString(), diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index c3f0d2a0a26..42833283ab9 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -679,7 +679,7 @@ TEST_F(RegionKVStoreTest, Writes) } catch (Exception & e) { - ASSERT_EQ(e.message(), "Raw TiDB PK: 800000000000091D, Prewrite ts: 2333 can not found in default cf for key: 7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE: (region id 1, applied_index:5, applied_term:5)"); + ASSERT_EQ(e.message(), "Raw TiDB PK: 800000000000091D, Prewrite ts: 2333 can not found in default cf for key: 7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE, region_id: 1, applied_index: 5: (applied_term: 5)"); ASSERT_EQ(kvs.getRegion(1)->dataInfo(), "[write 1 lock 1 ]"); kvs.getRegion(1)->tryCompactionFilter(1000); } diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp index a63fd701be9..c8112e880be 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp @@ -196,6 +196,7 @@ try // Write some data, and persist meta. auto [index, term] = proxy_instance->normalWrite(region_id, {34}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + kvs.setRegionCompactLogConfig(0, 0, 0, 0); persistAfterWrite(global_context, kvs, proxy_instance, page_storage, region_id, index); auto s3_client = S3::ClientFactory::instance().sharedTiFlashClient(); diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index 326e7551cba..2c8fae7775d 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -91,7 +91,7 @@ try // Reject compact log. kvs.setRegionCompactLogConfig(10000000, 10000000, 10000000, 10000000); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, indexc); - ASSERT_EQ(kvr1->lastCompactLogApplied(), indexc); + ASSERT_EQ(kvr1->lastCompactLogApplied(), 5); } } From 363fd5de738b01275a609424d36e906e00d9df8c Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 11 Aug 2023 21:53:09 +0800 Subject: [PATCH 59/66] add debug Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/Region.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 37195b4626f..36e0b65a4b9 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -524,6 +524,7 @@ void Region::updateLastCompactLogApplied() const if (last_compact_log_applied != 0) { uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; + LOG_DEBUG(log, "!!!!! updateLastCompactLogApplied region_id: {} gap {} current_applied_index {} last_compact_log_applied {}", id(), gap, current_applied_index, last_compact_log_applied); GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); } last_compact_log_applied = current_applied_index; From 8321f6926e5e9906ef4840eea5e97840c0425686 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Mon, 14 Aug 2023 12:38:47 +0800 Subject: [PATCH 60/66] Update dbms/src/Storages/DeltaMerge/DeltaMergeStore.h Co-authored-by: jinhelin --- dbms/src/Storages/DeltaMerge/DeltaMergeStore.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 3c0e42ef43c..ad4d62a457c 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -552,7 +552,7 @@ class DeltaMergeStore : private boost::noncopyable std::vector ranges; if (thread_type != ThreadType::Write) return result; - for (Iter it = begin; it != end; it++) + for (auto it = begin; it != end; ++it) { if (checkSegmentUpdate(context, *it, thread_type, input_type)) { From f7c25fce354a613e3b9cd1e5fdbe896088332ef8 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 14 Aug 2023 13:34:17 +0800 Subject: [PATCH 61/66] addr cmt Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/tests/kvstore_helper.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index 573f666f670..6796df02915 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -112,7 +112,7 @@ class RegionKVStoreTest : public ::testing::Test auto & global_ctx = TiFlashTestEnv::getGlobalContext(); global_ctx.tryReleaseWriteNodePageStorageForTest(); global_ctx.initializeWriteNodePageStorageIfNeed(*path_pool); - kvstore = std::make_shared(global_ctx); + kvstore = std::make_unique(global_ctx); // only recreate kvstore and restore data from disk, don't recreate proxy instance kvstore->restore(*path_pool, proxy_helper.get()); return *kvstore; @@ -204,7 +204,7 @@ class RegionKVStoreTest : public ::testing::Test std::string test_path; std::unique_ptr path_pool; - std::shared_ptr kvstore; + std::unique_ptr kvstore; std::unique_ptr proxy_instance; std::unique_ptr proxy_helper; From 65bd0a322d53ec20c0d3badc86d8300c40940689 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 14 Aug 2023 13:36:27 +0800 Subject: [PATCH 62/66] remove deubg msgs Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/Region.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 36e0b65a4b9..37195b4626f 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -524,7 +524,6 @@ void Region::updateLastCompactLogApplied() const if (last_compact_log_applied != 0) { uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; - LOG_DEBUG(log, "!!!!! updateLastCompactLogApplied region_id: {} gap {} current_applied_index {} last_compact_log_applied {}", id(), gap, current_applied_index, last_compact_log_applied); GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); } last_compact_log_applied = current_applied_index; From 704959fe2fb5a33280a636d6b885ced7bad3b5a4 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 14 Aug 2023 14:27:06 +0800 Subject: [PATCH 63/66] fix error Signed-off-by: CalvinNeo --- dbms/src/Storages/DeltaMerge/Segment.cpp | 1 - dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp | 1 - dbms/src/Storages/Transaction/tests/kvstore_helper.h | 4 ++-- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index cf04f01c4a5..f4ee6fa95e2 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -51,7 +51,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index 2c8fae7775d..500d2b64814 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -664,7 +664,6 @@ try region_id = 2; initStorages(); KVStore & kvs = getKVS(); - // ctx.getTMTContext().debugSetKVStore(kvstore); table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); auto start = RecordKVFormat::genKey(table_id, 0); auto end = RecordKVFormat::genKey(table_id, 10); diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index 6796df02915..573f666f670 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -112,7 +112,7 @@ class RegionKVStoreTest : public ::testing::Test auto & global_ctx = TiFlashTestEnv::getGlobalContext(); global_ctx.tryReleaseWriteNodePageStorageForTest(); global_ctx.initializeWriteNodePageStorageIfNeed(*path_pool); - kvstore = std::make_unique(global_ctx); + kvstore = std::make_shared(global_ctx); // only recreate kvstore and restore data from disk, don't recreate proxy instance kvstore->restore(*path_pool, proxy_helper.get()); return *kvstore; @@ -204,7 +204,7 @@ class RegionKVStoreTest : public ::testing::Test std::string test_path; std::unique_ptr path_pool; - std::unique_ptr kvstore; + std::shared_ptr kvstore; std::unique_ptr proxy_instance; std::unique_ptr proxy_helper; From d1743a69475b904914dbf57e1c4938e34d81f286 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 15 Aug 2023 15:24:10 +0800 Subject: [PATCH 64/66] fmt Signed-off-by: CalvinNeo --- dbms/src/Common/FailPoint.cpp | 13 +- dbms/src/Common/TiFlashMetrics.h | 44 +- dbms/src/Debug/MockRaftStoreProxy.cpp | 162 +- dbms/src/Debug/MockRaftStoreProxy.h | 99 +- dbms/src/Debug/dbgTools.cpp | 129 +- dbms/src/Debug/dbgTools.h | 55 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 441 +++-- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 237 +-- .../DeltaMerge/DeltaMergeStore_Ingest.cpp | 194 +- .../DeltaMerge/DeltaMergeStore_InternalBg.cpp | 127 +- dbms/src/Storages/DeltaMerge/RowKeyRange.h | 153 +- .../tests/gtest_dm_delta_merge_store.cpp | 1693 +++++++++-------- dbms/src/Storages/StorageDeltaMerge.cpp | 240 ++- dbms/src/Storages/StorageDeltaMerge.h | 70 +- .../Storages/Transaction/ApplySnapshot.cpp | 156 +- dbms/src/Storages/Transaction/KVStore.cpp | 299 ++- dbms/src/Storages/Transaction/KVStore.h | 102 +- .../Storages/Transaction/PartitionStreams.cpp | 148 +- dbms/src/Storages/Transaction/ProxyFFI.cpp | 135 +- dbms/src/Storages/Transaction/ProxyFFI.h | 30 +- .../Storages/Transaction/ReadIndexWorker.cpp | 94 +- dbms/src/Storages/Transaction/Region.cpp | 236 ++- dbms/src/Storages/Transaction/Region.h | 28 +- .../Storages/Transaction/RegionCFDataBase.cpp | 18 +- dbms/src/Storages/Transaction/RegionData.cpp | 57 +- dbms/src/Storages/Transaction/RegionManager.h | 10 +- dbms/src/Storages/Transaction/RegionMeta.cpp | 44 +- dbms/src/Storages/Transaction/RegionMeta.h | 5 +- .../Storages/Transaction/RegionPersister.cpp | 139 +- dbms/src/Storages/Transaction/RegionTable.h | 41 +- .../Transaction/RegionsRangeIndex.cpp | 7 +- dbms/src/Storages/Transaction/TMTContext.cpp | 88 +- .../Transaction/tests/gtest_kvstore.cpp | 603 +++--- .../tests/gtest_kvstore_fast_add_peer.cpp | 22 +- .../Transaction/tests/gtest_new_kvstore.cpp | 285 ++- .../tests/gtest_proactive_flush.cpp | 14 +- .../tests/gtest_read_index_worker.cpp | 136 +- .../Transaction/tests/kvstore_helper.h | 45 +- 38 files changed, 3937 insertions(+), 2462 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index efe9a841fd1..e82ef6f520d 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -269,8 +269,7 @@ void FailPointHelper::enableFailPoint(const String & fail_point_name, std::optio throw Exception(fmt::format("Cannot find fail point {}", fail_point_name), ErrorCodes::FAIL_POINT_ERROR); } -std::optional -FailPointHelper::getFailPointVal(const String & fail_point_name) +std::optional FailPointHelper::getFailPointVal(const String & fail_point_name) { if (auto iter = fail_point_val.find(fail_point_name); iter != fail_point_val.end()) { @@ -313,7 +312,10 @@ void FailPointHelper::initRandomFailPoints(Poco::Util::LayeredConfiguration & co for (const auto & string_token : string_tokens) { Poco::StringTokenizer pair_tokens(string_token, "-"); - RUNTIME_ASSERT((pair_tokens.count() == 2), log, "RandomFailPoints config should be FailPointA-RatioA,FailPointB-RatioB,... format"); + RUNTIME_ASSERT( + (pair_tokens.count() == 2), + log, + "RandomFailPoints config should be FailPointA-RatioA,FailPointB-RatioB,... format"); double rate = atof(pair_tokens[1].c_str()); //NOLINT(cert-err34-c): check conversion error manually RUNTIME_ASSERT((0 <= rate && rate <= 1.0), log, "RandomFailPoint trigger rate should in [0,1], while {}", rate); enableRandomFailPoint(pair_tokens[0], rate); @@ -331,7 +333,10 @@ void FailPointHelper::disableRandomFailPoints(Poco::Util::LayeredConfiguration & for (const auto & string_token : string_tokens) { Poco::StringTokenizer pair_tokens(string_token, "-"); - RUNTIME_ASSERT((pair_tokens.count() == 2), log, "RandomFailPoints config should be FailPointA-RatioA,FailPointB-RatioB,... format"); + RUNTIME_ASSERT( + (pair_tokens.count() == 2), + log, + "RandomFailPoints config should be FailPointA-RatioA,FailPointB-RatioB,... format"); disableFailPoint(pair_tokens[0]); } LOG_INFO(log, "Disable RandomFailPoints: {}", random_fail_point_cfg); diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 35bc19428c5..f5add6cf08d 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -390,25 +390,31 @@ namespace DB Histogram, \ F(type_decode, {{"type", "decode"}}, ExpBuckets{0.0005, 2, 20}), \ F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ - M(tiflash_raft_raft_log_lag_count, "Bucketed histogram raft index lag", Histogram, \ - F(type_compact_index, {{"type", "compact_index"}}, EqualWidthBuckets{0, 200, 5}), \ - F(type_applied_index, {{"type", "applied_index"}}, EqualWidthBuckets{0, 200, 5}), \ - F(type_unflushed_applied_index, {{"type", "unflushed_applied_index"}}, EqualWidthBuckets{0, 200, 5})) \ - M(tiflash_raft_raft_events_count, "Raft event counter", Counter, \ - F(type_pre_exec_compact, {{"type", "pre_exec_compact"}}), \ - F(type_flush_apply_snapshot, {{"type", "flush_apply_snapshot"}}), \ - F(type_flush_ingest_sst, {{"type", "flush_ingest_sst"}}), \ - F(type_flush_useless_admin, {{"type", "flush_useless_admin"}}), \ - F(type_flush_useful_admin, {{"type", "flush_useful_admin"}}), \ - F(type_flush_passive, {{"type", "flush_passive"}}), \ - F(type_flush_proactive, {{"type", "flush_proactive"}}), \ - F(type_flush_log_gap, {{"type", "flush_log_gap"}}), \ - F(type_flush_size, {{"type", "flush_size"}}), \ - F(type_flush_rowcount, {{"type", "flush_rowcount"}}), \ - F(type_exec_compact, {{"type", "exec_compact"}})) \ - M(tiflash_raft_region_flush_size, "Bucketed histogram of region flushed size", Histogram, \ - F(type_flushed, {{"type", "flushed"}}, ExpBuckets{32, 2, 16}), \ - F(type_unflushed, {{"type", "unflushed"}}, ExpBuckets{32, 2, 16})) \ + M(tiflash_raft_raft_log_lag_count, \ + "Bucketed histogram raft index lag", \ + Histogram, \ + F(type_compact_index, {{"type", "compact_index"}}, EqualWidthBuckets{0, 200, 5}), \ + F(type_applied_index, {{"type", "applied_index"}}, EqualWidthBuckets{0, 200, 5}), \ + F(type_unflushed_applied_index, {{"type", "unflushed_applied_index"}}, EqualWidthBuckets{0, 200, 5})) \ + M(tiflash_raft_raft_events_count, \ + "Raft event counter", \ + Counter, \ + F(type_pre_exec_compact, {{"type", "pre_exec_compact"}}), \ + F(type_flush_apply_snapshot, {{"type", "flush_apply_snapshot"}}), \ + F(type_flush_ingest_sst, {{"type", "flush_ingest_sst"}}), \ + F(type_flush_useless_admin, {{"type", "flush_useless_admin"}}), \ + F(type_flush_useful_admin, {{"type", "flush_useful_admin"}}), \ + F(type_flush_passive, {{"type", "flush_passive"}}), \ + F(type_flush_proactive, {{"type", "flush_proactive"}}), \ + F(type_flush_log_gap, {{"type", "flush_log_gap"}}), \ + F(type_flush_size, {{"type", "flush_size"}}), \ + F(type_flush_rowcount, {{"type", "flush_rowcount"}}), \ + F(type_exec_compact, {{"type", "exec_compact"}})) \ + M(tiflash_raft_region_flush_size, \ + "Bucketed histogram of region flushed size", \ + Histogram, \ + F(type_flushed, {{"type", "flushed"}}, ExpBuckets{32, 2, 16}), \ + F(type_unflushed, {{"type", "unflushed"}}, ExpBuckets{32, 2, 16})) \ /* required by DBaaS */ \ M(tiflash_server_info, \ "Indicate the tiflash server info, and the value is the start timestamp (s).", \ diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 024de51c78a..06cd1400f39 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -69,8 +69,7 @@ RawRustPtr fn_make_read_index_task(RaftStoreProxyPtr ptr, BaseBuffView view) return RawRustPtr{task, static_cast(RawObjType::MockReadIndexTask)}; } -RawRustPtr fn_make_async_waker(void (*wake_fn)(RawVoidPtr), - RawCppPtr data) +RawRustPtr fn_make_async_waker(void (*wake_fn)(RawVoidPtr), RawCppPtr data) { auto * p = new MockAsyncWaker{std::make_shared()}; p->data->data = data; @@ -115,12 +114,21 @@ void fn_gc_rust_ptr(RawVoidPtr ptr, RawRustPtrType type_) } } -void fn_handle_batch_read_index(RaftStoreProxyPtr, CppStrVecView, RawVoidPtr, uint64_t, void (*)(RawVoidPtr, BaseBuffView, uint64_t)) +void fn_handle_batch_read_index( + RaftStoreProxyPtr, + CppStrVecView, + RawVoidPtr, + uint64_t, + void (*)(RawVoidPtr, BaseBuffView, uint64_t)) { throw Exception("`fn_handle_batch_read_index` is deprecated"); } -KVGetStatus fn_get_region_local_state(RaftStoreProxyPtr ptr, uint64_t region_id, RawVoidPtr data, RawCppStringPtr * error_msg) +KVGetStatus fn_get_region_local_state( + RaftStoreProxyPtr ptr, + uint64_t region_id, + RawVoidPtr data, + RawCppStringPtr * error_msg) { if (!ptr.inner) { @@ -140,7 +148,12 @@ KVGetStatus fn_get_region_local_state(RaftStoreProxyPtr ptr, uint64_t region_id, return KVGetStatus::NotFound; } -void fn_notify_compact_log(RaftStoreProxyPtr ptr, uint64_t region_id, uint64_t compact_index, uint64_t compact_term, uint64_t applied_index) +void fn_notify_compact_log( + RaftStoreProxyPtr ptr, + uint64_t region_id, + uint64_t compact_index, + uint64_t compact_term, + uint64_t applied_index) { UNUSED(applied_index); // Update flushed applied_index and truncated state. @@ -154,9 +167,7 @@ void fn_notify_compact_log(RaftStoreProxyPtr ptr, uint64_t region_id, uint64_t c } } -RaftstoreVer fn_get_cluster_raftstore_version(RaftStoreProxyPtr ptr, - uint8_t, - int64_t) +RaftstoreVer fn_get_cluster_raftstore_version(RaftStoreProxyPtr ptr, uint8_t, int64_t) { auto & x = as_ref(ptr); return x.cluster_ver; @@ -272,13 +283,21 @@ UniversalWriteBatch MockProxyRegion::persistMeta() auto region_local_state = this->state.SerializeAsString(); MemoryWriteBuffer buf(0, region_local_state.size()); buf.write(region_local_state.data(), region_local_state.size()); - wb.putPage(UniversalPageId(region_key.data(), region_key.size()), 0, buf.tryGetReadBuffer(), region_local_state.size()); + wb.putPage( + UniversalPageId(region_key.data(), region_key.size()), + 0, + buf.tryGetReadBuffer(), + region_local_state.size()); auto apply_key = UniversalPageIdFormat::toRaftApplyStateKeyInKVEngine(this->id); auto raft_apply_state = this->apply.SerializeAsString(); MemoryWriteBuffer buf2(0, raft_apply_state.size()); buf2.write(raft_apply_state.data(), raft_apply_state.size()); - wb.putPage(UniversalPageId(apply_key.data(), apply_key.size()), 0, buf2.tryGetReadBuffer(), raft_apply_state.size()); + wb.putPage( + UniversalPageId(apply_key.data(), apply_key.size()), + 0, + buf2.tryGetReadBuffer(), + raft_apply_state.size()); raft_serverpb::RegionLocalState restored_region_state; raft_serverpb::RaftApplyState restored_apply_state; @@ -381,8 +400,8 @@ void MockRaftStoreProxy::init(size_t region_num) std::unique_ptr MockRaftStoreProxy::generateProxyHelper() { - auto proxy_helper = std::make_unique(MockRaftStoreProxy::SetRaftStoreProxyFFIHelper( - RaftStoreProxyPtr{this})); + auto proxy_helper = std::make_unique( + MockRaftStoreProxy::SetRaftStoreProxyFFIHelper(RaftStoreProxyPtr{this})); // Bind ffi to MockSSTReader. proxy_helper->sst_reader_interfaces = make_mock_sst_reader_interface(); return proxy_helper; @@ -470,10 +489,7 @@ void MockRaftStoreProxy::debugAddRegions( } } -void MockRaftStoreProxy::loadRegionFromKVStore( - KVStore & kvs, - TMTContext & tmt, - UInt64 region_id) +void MockRaftStoreProxy::loadRegionFromKVStore(KVStore & kvs, TMTContext & tmt, UInt64 region_id) { UNUSED(tmt); auto kvr = kvs.getRegion(region_id); @@ -499,7 +515,12 @@ void MockRaftStoreProxy::loadRegionFromKVStore( r->apply.set_commit_term(commit_term); } } - LOG_INFO(log, "loadRegionFromKVStore [region_id={}] region_state {} apply_state {}", region_id, r->state.DebugString(), r->apply.DebugString()); + LOG_INFO( + log, + "loadRegionFromKVStore [region_id={}] region_state {} apply_state {}", + region_id, + r->state.DebugString(), + r->apply.DebugString()); } std::tuple MockRaftStoreProxy::normalWrite( @@ -534,14 +555,14 @@ std::tuple MockRaftStoreProxy::normalWrite( new_keys.emplace_back(RecordKVFormat::genKey(table_id, keys[i], 1)); } } - region->commands[index] = { - term, - MockProxyRegion::RawWrite{ - new_keys, - vals, - cmd_types, - cmd_cf, - }}; + region->commands[index] + = {term, + MockProxyRegion::RawWrite{ + new_keys, + vals, + cmd_types, + cmd_cf, + }}; } return std::make_tuple(index, term); } @@ -566,14 +587,14 @@ std::tuple MockRaftStoreProxy::rawWrite( // The new entry is committed on Proxy's side. region->updateCommitIndex(index); // We record them, as persisted raft log, for potential recovery. - region->commands[index] = { - term, - MockProxyRegion::RawWrite{ - keys, - vals, - cmd_types, - cmd_cf, - }}; + region->commands[index] + = {term, + MockProxyRegion::RawWrite{ + keys, + vals, + cmd_types, + cmd_cf, + }}; } return std::make_tuple(index, term); } @@ -597,17 +618,19 @@ std::tuple MockRaftStoreProxy::adminCommand( // The new entry is committed on Proxy's side. region->updateCommitIndex(index); // We record them, as persisted raft log, for potential recovery. - region->commands[index] = { - term, - MockProxyRegion::AdminCommand{ - request, - response, - }}; + region->commands[index] + = {term, + MockProxyRegion::AdminCommand{ + request, + response, + }}; } return std::make_tuple(index, term); } -std::tuple MockRaftStoreProxy::composeCompactLog(MockProxyRegionPtr region, UInt64 compact_index) +std::tuple MockRaftStoreProxy::composeCompactLog( + MockProxyRegionPtr region, + UInt64 compact_index) { raft_cmdpb::AdminRequest request; raft_cmdpb::AdminResponse response; @@ -621,7 +644,10 @@ std::tuple MockRaftStorePro return std::make_tuple(request, response); } -std::tuple MockRaftStoreProxy::composeChangePeer(metapb::Region && meta, std::vector peer_ids, bool is_v2) +std::tuple MockRaftStoreProxy::composeChangePeer( + metapb::Region && meta, + std::vector peer_ids, + bool is_v2) { raft_cmdpb::AdminRequest request; raft_cmdpb::AdminResponse response; @@ -642,7 +668,9 @@ std::tuple MockRaftStorePro return std::make_tuple(request, response); } -std::tuple MockRaftStoreProxy::composePrepareMerge(metapb::Region && target, UInt64 min_index) +std::tuple MockRaftStoreProxy::composePrepareMerge( + metapb::Region && target, + UInt64 min_index) { raft_cmdpb::AdminRequest request; raft_cmdpb::AdminResponse response; @@ -653,7 +681,9 @@ std::tuple MockRaftStorePro return std::make_tuple(request, response); } -std::tuple MockRaftStoreProxy::composeCommitMerge(metapb::Region && source, UInt64 commit) +std::tuple MockRaftStoreProxy::composeCommitMerge( + metapb::Region && source, + UInt64 commit) { raft_cmdpb::AdminRequest request; raft_cmdpb::AdminResponse response; @@ -674,7 +704,10 @@ std::tuple MockRaftStorePro return std::make_tuple(request, response); } -std::tuple MockRaftStoreProxy::composeBatchSplit(std::vector && region_ids, std::vector> && ranges, metapb::RegionEpoch old_epoch) +std::tuple MockRaftStoreProxy::composeBatchSplit( + std::vector && region_ids, + std::vector> && ranges, + metapb::RegionEpoch old_epoch) { RUNTIME_CHECK_MSG(region_ids.size() == ranges.size(), "error composeBatchSplit input"); auto n = region_ids.size(); @@ -774,7 +807,15 @@ void MockRaftStoreProxy::doApply( { if (cmd.admin().cmd_type() == raft_cmdpb::AdminCmdType::CompactLog) { - auto res = kvs.tryFlushRegionData(region_id, false, true, tmt, index, term, region->getApply().truncated_state().index(), region->getApply().truncated_state().term()); + auto res = kvs.tryFlushRegionData( + region_id, + false, + true, + tmt, + index, + term, + region->getApply().truncated_state().index(), + region->getApply().truncated_state().term()); auto compact_index = cmd.admin().request.compact_log().compact_index(); auto compact_term = cmd.admin().request.compact_log().compact_term(); if (!res) @@ -787,7 +828,13 @@ void MockRaftStoreProxy::doApply( LOG_DEBUG(log, "mock pre exec success, update to {},{}", compact_index, compact_term); } } - kvs.handleAdminRaftCmd(std::move(cmd.admin().request), std::move(cmd.admin().response), region_id, index, term, tmt); + kvs.handleAdminRaftCmd( + std::move(cmd.admin().request), + std::move(cmd.admin().response), + region_id, + index, + term, + tmt); } if (cond.type == MockRaftStoreProxy::FailCond::Type::BEFORE_KVSTORE_ADVANCE) @@ -828,11 +875,7 @@ void MockRaftStoreProxy::doApply( region->persistAppliedIndex(); } -void MockRaftStoreProxy::replay( - KVStore & kvs, - TMTContext & tmt, - uint64_t region_id, - uint64_t to) +void MockRaftStoreProxy::replay(KVStore & kvs, TMTContext & tmt, uint64_t region_id, uint64_t to) { auto region = getRegion(region_id); assert(region != nullptr); @@ -920,7 +963,8 @@ RegionPtr MockRaftStoreProxy::snapshot( term = region->getLatestCommitTerm(); } - auto new_kv_region = kvs.genRegionPtr(old_kv_region->cloneMetaRegion(), old_kv_region->mutMeta().peerId(), index, term); + auto new_kv_region + = kvs.genRegionPtr(old_kv_region->cloneMetaRegion(), old_kv_region->mutMeta().peerId(), index, term); // The new entry is committed on Proxy's side. region->updateCommitIndex(index); @@ -934,13 +978,7 @@ RegionPtr MockRaftStoreProxy::snapshot( } } SSTViewVec snaps{ssts.data(), ssts.size()}; - auto ingest_ids = kvs.preHandleSnapshotToFiles( - new_kv_region, - snaps, - index, - term, - deadline_index, - tmt); + auto ingest_ids = kvs.preHandleSnapshotToFiles(new_kv_region, snaps, index, term, deadline_index, tmt); auto rg = RegionPtrWithSnapshotFiles{new_kv_region, std::move(ingest_ids)}; if (cancel_after_prehandle) @@ -957,11 +995,7 @@ RegionPtr MockRaftStoreProxy::snapshot( return kvs.getRegion(region_id); } -TableID MockRaftStoreProxy::bootstrapTable( - Context & ctx, - KVStore & kvs, - TMTContext & tmt, - bool drop_at_first) +TableID MockRaftStoreProxy::bootstrapTable(Context & ctx, KVStore & kvs, TMTContext & tmt, bool drop_at_first) { UNUSED(kvs); ColumnsDescription columns; diff --git a/dbms/src/Debug/MockRaftStoreProxy.h b/dbms/src/Debug/MockRaftStoreProxy.h index dfe0a1bb9db..1ce51be7c9a 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.h +++ b/dbms/src/Debug/MockRaftStoreProxy.h @@ -54,10 +54,7 @@ struct MockProxyRegion : MutexLockWrap { raft_cmdpb::AdminRequest request; raft_cmdpb::AdminResponse response; - raft_cmdpb::AdminCmdType cmd_type() const - { - return request.cmd_type(); - } + raft_cmdpb::AdminCmdType cmd_type() const { return request.cmd_type(); } }; struct CachedCommand @@ -65,25 +62,13 @@ struct MockProxyRegion : MutexLockWrap uint64_t term; std::variant inner; - bool has_admin_request() const - { - return std::holds_alternative(inner); - } + bool has_admin_request() const { return std::holds_alternative(inner); } - bool has_raw_write_request() const - { - return std::holds_alternative(inner); - } + bool has_raw_write_request() const { return std::holds_alternative(inner); } - AdminCommand & admin() - { - return std::get(inner); - } + AdminCommand & admin() { return std::get(inner); } - RawWrite & raw_write() - { - return std::get(inner); - } + RawWrite & raw_write() { return std::get(inner); } }; const uint64_t id; @@ -98,14 +83,8 @@ struct MockAsyncNotifier { RawCppPtr data; // notifier void (*wake_fn)(RawVoidPtr); - void wake() const - { - wake_fn(data.ptr); - } - ~MockAsyncNotifier() - { - GcRawCppPtr(data.ptr, data.type); - } + void wake() const { wake_fn(data.ptr); } + ~MockAsyncNotifier() { GcRawCppPtr(data.ptr, data.type); } }; struct MockAsyncWaker @@ -199,11 +178,7 @@ struct MockRaftStoreProxy : MutexLockWrap /// Must be called if: /// 1. Applying snapshot which needs table schema /// 2. Doing row2col. - TableID bootstrapTable( - Context & ctx, - KVStore & kvs, - TMTContext & tmt, - bool drop_at_first = true); + TableID bootstrapTable(Context & ctx, KVStore & kvs, TMTContext & tmt, bool drop_at_first = true); /// Manually add a region. void debugAddRegions( @@ -212,10 +187,7 @@ struct MockRaftStoreProxy : MutexLockWrap std::vector region_ids, std::vector> && ranges); - void loadRegionFromKVStore( - KVStore & kvs, - TMTContext & tmt, - UInt64 region_id); + void loadRegionFromKVStore(KVStore & kvs, TMTContext & tmt, UInt64 region_id); /// We assume that we generate one command, and immediately commit. /// Normal write to a region. @@ -235,14 +207,30 @@ struct MockRaftStoreProxy : MutexLockWrap std::optional forced_index = std::nullopt); - std::tuple adminCommand(UInt64 region_id, raft_cmdpb::AdminRequest &&, raft_cmdpb::AdminResponse &&, std::optional forced_index = std::nullopt); + std::tuple adminCommand( + UInt64 region_id, + raft_cmdpb::AdminRequest &&, + raft_cmdpb::AdminResponse &&, + std::optional forced_index = std::nullopt); - static std::tuple composeCompactLog(MockProxyRegionPtr region, UInt64 compact_index); - static std::tuple composeChangePeer(metapb::Region && meta, std::vector peer_ids, bool is_v2 = true); - static std::tuple composePrepareMerge(metapb::Region && target, UInt64 min_index); - static std::tuple composeCommitMerge(metapb::Region && source, UInt64 commit); + static std::tuple composeCompactLog( + MockProxyRegionPtr region, + UInt64 compact_index); + static std::tuple composeChangePeer( + metapb::Region && meta, + std::vector peer_ids, + bool is_v2 = true); + static std::tuple composePrepareMerge( + metapb::Region && target, + UInt64 min_index); + static std::tuple composeCommitMerge( + metapb::Region && source, + UInt64 commit); static std::tuple composeRollbackMerge(UInt64 commit); - static std::tuple composeBatchSplit(std::vector && region_ids, std::vector> && ranges, metapb::RegionEpoch old_epoch); + static std::tuple composeBatchSplit( + std::vector && region_ids, + std::vector> && ranges, + metapb::RegionEpoch old_epoch); struct Cf { @@ -255,10 +243,7 @@ struct MockRaftStoreProxy : MutexLockWrap void insert(HandleID key, std::string val); void insert_raw(std::string key, std::string val); - ColumnFamilyType cf_type() const - { - return type; - } + ColumnFamilyType cf_type() const { return type; } // Only use this after all sst_files is generated. // vector::push_back can cause destruction of std::string, @@ -293,11 +278,7 @@ struct MockRaftStoreProxy : MutexLockWrap uint64_t index, std::optional check_proactive_flush = std::nullopt); - void replay( - KVStore & kvs, - TMTContext & tmt, - uint64_t region_id, - uint64_t to); + void replay(KVStore & kvs, TMTContext & tmt, uint64_t region_id, uint64_t to); void clear() { @@ -333,7 +314,8 @@ enum class RawObjType : uint32_t MockAsyncWaker, }; -struct GCMonitor : MutexLockWrap +struct GCMonitor + : MutexLockWrap , public ext::Singleton { void add(RawObjType type, int64_t diff); @@ -353,11 +335,12 @@ std::vector> regionRangeToEncodeKeys(Types & // RegionRangeKeys::RegionRange is not copy-constructible, however, initialize_list need copy construction. // So we have to so this way, rather than create a composeXXX that accepts a vector of RegionRangeKeys::RegionRange. std::vector> ranges_str; - ([&] { - auto & x = args; - ranges_str.emplace_back(std::make_pair(x.first.toString(), x.second.toString())); - }(), - ...); + ( + [&] { + auto & x = args; + ranges_str.emplace_back(std::make_pair(x.first.toString(), x.second.toString())); + }(), + ...); return ranges_str; } diff --git a/dbms/src/Debug/dbgTools.cpp b/dbms/src/Debug/dbgTools.cpp index 6e89f6958ed..f6a96cc4954 100644 --- a/dbms/src/Debug/dbgTools.cpp +++ b/dbms/src/Debug/dbgTools.cpp @@ -44,7 +44,12 @@ namespace RegionBench { using TiDB::ColumnInfo; -RegionPtr createRegion(TableID table_id, RegionID region_id, const HandleID & start, const HandleID & end, std::optional index_) +RegionPtr createRegion( + TableID table_id, + RegionID region_id, + const HandleID & start, + const HandleID & end, + std::optional index_) { metapb::Region region; metapb::Peer peer; @@ -64,7 +69,12 @@ RegionPtr createRegion(TableID table_id, RegionID region_id, const HandleID & st return std::make_shared(std::move(region_meta)); } -Regions createRegions(TableID table_id, size_t region_num, size_t key_num_each_region, HandleID handle_begin, RegionID new_region_id_begin) +Regions createRegions( + TableID table_id, + size_t region_num, + size_t key_num_each_region, + HandleID handle_begin, + RegionID new_region_id_begin) { Regions regions; for (RegionID region_id = new_region_id_begin; region_id < static_cast(new_region_id_begin + region_num); @@ -114,7 +124,14 @@ void setupDelRequest(raft_cmdpb::Request * req, const std::string & cf, const Ti del->set_key(key.getStr()); } -void addRequestsToRaftCmd(raft_cmdpb::RaftCmdRequest & request, const TiKVKey & key, const TiKVValue & value, UInt64 prewrite_ts, UInt64 commit_ts, bool del, const String pk) +void addRequestsToRaftCmd( + raft_cmdpb::RaftCmdRequest & request, + const TiKVKey & key, + const TiKVValue & value, + UInt64 prewrite_ts, + UInt64 commit_ts, + bool del, + const String pk) { TiKVKey commit_key = RecordKVFormat::appendTs(key, commit_ts); const TiKVKey & lock_key = key; @@ -176,7 +193,9 @@ T convertNumber(const Field & field) case Field::Types::Decimal256: return static_cast(field.get>()); default: - throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to number", ErrorCodes::LOGICAL_ERROR); + throw Exception( + String("Unable to convert field type ") + field.getTypeName() + " to number", + ErrorCodes::LOGICAL_ERROR); } } @@ -199,7 +218,9 @@ Field convertDecimal(const ColumnInfo & column_info, const Field & field) case Field::Types::Decimal256: return column_info.getDecimalValue(field.get().toString(column_info.decimal)); default: - throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to number", ErrorCodes::LOGICAL_ERROR); + throw Exception( + String("Unable to convert field type ") + field.getTypeName() + " to number", + ErrorCodes::LOGICAL_ERROR); } } @@ -213,7 +234,9 @@ Field convertEnum(const ColumnInfo & column_info, const Field & field) case Field::Types::String: return static_cast(column_info.getEnumIndex(field.get())); default: - throw Exception(String("Unable to convert field type ") + field.getTypeName() + " to Enum", ErrorCodes::LOGICAL_ERROR); + throw Exception( + String("Unable to convert field type ") + field.getTypeName() + " to Enum", + ErrorCodes::LOGICAL_ERROR); } } @@ -269,9 +292,10 @@ Field convertField(const ColumnInfo & column_info, const Field & field) void encodeRow(const TiDB::TableInfo & table_info, const std::vector & fields, WriteBuffer & ss) { if (table_info.columns.size() < fields.size() + table_info.pk_is_handle) - throw Exception("Encoding row has less columns than encode values [num_columns=" + DB::toString(table_info.columns.size()) - + "] [num_fields=" + DB::toString(fields.size()) + "] . ", - ErrorCodes::LOGICAL_ERROR); + throw Exception( + "Encoding row has less columns than encode values [num_columns=" + DB::toString(table_info.columns.size()) + + "] [num_fields=" + DB::toString(fields.size()) + "] . ", + ErrorCodes::LOGICAL_ERROR); std::vector flatten_fields; std::unordered_set pk_column_names; @@ -296,7 +320,8 @@ void encodeRow(const TiDB::TableInfo & table_info, const std::vector & fi static bool row_format_flip = false; // Ping-pong encoding using row format V1/V2. - (row_format_flip = !row_format_flip) ? encodeRowV1(table_info, flatten_fields, ss) : encodeRowV2(table_info, flatten_fields, ss); + (row_format_flip = !row_format_flip) ? encodeRowV1(table_info, flatten_fields, ss) + : encodeRowV2(table_info, flatten_fields, ss); } void insert( // @@ -408,7 +433,16 @@ struct BatchCtrl HandleID handle_begin; bool del; - BatchCtrl(Int64 concurrent_id_, Int64 flush_num_, Int64 batch_num_, UInt64 min_strlen_, UInt64 max_strlen_, Context * context_, RegionPtr region_, HandleID handle_begin_, bool del_) + BatchCtrl( + Int64 concurrent_id_, + Int64 flush_num_, + Int64 batch_num_, + UInt64 min_strlen_, + UInt64 max_strlen_, + Context * context_, + RegionPtr region_, + HandleID handle_begin_, + bool del_) : concurrent_id(concurrent_id_) , flush_num(flush_num_) , batch_num(batch_num_) @@ -432,11 +466,15 @@ struct BatchCtrl switch (flag) { case TiDB::CodecFlagJson: - throw Exception("Not implented yet: BatchCtrl::encodeDatum, TiDB::CodecFlagJson", ErrorCodes::LOGICAL_ERROR); + throw Exception( + "Not implented yet: BatchCtrl::encodeDatum, TiDB::CodecFlagJson", + ErrorCodes::LOGICAL_ERROR); case TiDB::CodecFlagMax: throw Exception("Not implented yet: BatchCtrl::encodeDatum, TiDB::CodecFlagMax", ErrorCodes::LOGICAL_ERROR); case TiDB::CodecFlagDuration: - throw Exception("Not implented yet: BatchCtrl::encodeDatum, TiDB::CodecFlagDuration", ErrorCodes::LOGICAL_ERROR); + throw Exception( + "Not implented yet: BatchCtrl::encodeDatum, TiDB::CodecFlagDuration", + ErrorCodes::LOGICAL_ERROR); case TiDB::CodecFlagNil: return; case TiDB::CodecFlagBytes: @@ -475,7 +513,10 @@ struct BatchCtrl } }; -void batchInsert(const TiDB::TableInfo & table_info, std::unique_ptr batch_ctrl, std::function fn_gen_magic_num) +void batchInsert( + const TiDB::TableInfo & table_info, + std::unique_ptr batch_ctrl, + std::function fn_gen_magic_num) { RegionPtr & region = batch_ctrl->region; @@ -508,14 +549,22 @@ void batchInsert(const TiDB::TableInfo & table_info, std::unique_ptr } } -void concurrentBatchInsert(const TiDB::TableInfo & table_info, Int64 concurrent_num, Int64 flush_num, Int64 batch_num, UInt64 min_strlen, UInt64 max_strlen, Context & context) +void concurrentBatchInsert( + const TiDB::TableInfo & table_info, + Int64 concurrent_num, + Int64 flush_num, + Int64 batch_num, + UInt64 min_strlen, + UInt64 max_strlen, + Context & context) { TMTContext & tmt = context.getTMTContext(); RegionID curr_max_region_id(InvalidRegionID); HandleID curr_max_handle_id = 0; tmt.getKVStore()->traverseRegions([&](const RegionID region_id, const RegionPtr & region) { - curr_max_region_id = (curr_max_region_id == InvalidRegionID) ? region_id : std::max(curr_max_region_id, region_id); + curr_max_region_id + = (curr_max_region_id == InvalidRegionID) ? region_id : std::max(curr_max_region_id, region_id); const auto range = region->getRange(); curr_max_handle_id = std::max(RecordKVFormat::getHandle(*range->rawKeys().second), curr_max_handle_id); }); @@ -523,16 +572,18 @@ void concurrentBatchInsert(const TiDB::TableInfo & table_info, Int64 concurrent_ Int64 key_num_each_region = flush_num * batch_num; HandleID handle_begin = curr_max_handle_id; - Regions regions = createRegions(table_info.id, concurrent_num, key_num_each_region, handle_begin, curr_max_region_id + 1); + Regions regions + = createRegions(table_info.id, concurrent_num, key_num_each_region, handle_begin, curr_max_region_id + 1); for (const RegionPtr & region : regions) tmt.getKVStore()->onSnapshot(region, nullptr, 0, tmt); std::list threads; for (Int64 i = 0; i < concurrent_num; i++, handle_begin += key_num_each_region) { - auto batch_ptr - = std::make_unique(i, flush_num, batch_num, min_strlen, max_strlen, &context, regions[i], handle_begin, false); - threads.push_back(std::thread(&batchInsert, table_info, std::move(batch_ptr), [](Int64 index) -> Int64 { return index; })); + auto batch_ptr = std::make_unique< + BatchCtrl>(i, flush_num, batch_num, min_strlen, max_strlen, &context, regions[i], handle_begin, false); + threads.push_back( + std::thread(&batchInsert, table_info, std::move(batch_ptr), [](Int64 index) -> Int64 { return index; })); } for (auto & thread : threads) { @@ -575,7 +626,8 @@ Int64 concurrentRangeOperate( continue; Int64 batch_num = handle_end - handle_begin; tol += batch_num; - auto batch_ptr = std::make_unique(-1, 1, batch_num, 1, 1, &context, region, handle_begin.handle_id, del); + auto batch_ptr + = std::make_unique(-1, 1, batch_num, 1, 1, &context, region, handle_begin.handle_id, del); threads.push_back(std::thread(&batchInsert, table_info, std::move(batch_ptr), [=](Int64 index) -> Int64 { std::ignore = index; return magic_num; @@ -588,7 +640,11 @@ Int64 concurrentRangeOperate( return tol; } -TableID getTableID(Context & context, const std::string & database_name, const std::string & table_name, const std::string & partition_id) +TableID getTableID( + Context & context, + const std::string & database_name, + const std::string & table_name, + const std::string & partition_id) { try { @@ -674,13 +730,20 @@ EngineStoreApplyRes applyWriteRaftCmd( cmd_cf.push_back(NameToCF(req.delete_().cf())); break; default: - throw Exception(fmt::format("Unsupport raft cmd {}", raft_cmdpb::CmdType_Name(type)), ErrorCodes::LOGICAL_ERROR); + throw Exception( + fmt::format("Unsupport raft cmd {}", raft_cmdpb::CmdType_Name(type)), + ErrorCodes::LOGICAL_ERROR); } } if (write_result_ptr) { return kvstore.handleWriteRaftCmdInner( - WriteCmdsView{.keys = keys.data(), .vals = vals.data(), .cmd_types = cmd_types.data(), .cmd_cf = cmd_cf.data(), .len = keys.size()}, + WriteCmdsView{ + .keys = keys.data(), + .vals = vals.data(), + .cmd_types = cmd_types.data(), + .cmd_cf = cmd_cf.data(), + .len = keys.size()}, region_id, index, term, @@ -691,7 +754,12 @@ EngineStoreApplyRes applyWriteRaftCmd( { DM::WriteResult write_result; return kvstore.handleWriteRaftCmdInner( - WriteCmdsView{.keys = keys.data(), .vals = vals.data(), .cmd_types = cmd_types.data(), .cmd_cf = cmd_cf.data(), .len = keys.size()}, + WriteCmdsView{ + .keys = keys.data(), + .vals = vals.data(), + .cmd_types = cmd_types.data(), + .cmd_cf = cmd_cf.data(), + .len = keys.size()}, region_id, index, term, @@ -724,7 +792,10 @@ std::optional mappedDatabaseWithOptional(Context & context, const String return SchemaNameMapper().mapDatabaseName(*db_info); } -std::optional mappedTableWithOptional(Context & context, const String & database_name, const String & table_name) +std::optional mappedTableWithOptional( + Context & context, + const String & database_name, + const String & table_name) { auto mapped_db = mappedDatabaseWithOptional(context, database_name); @@ -740,7 +811,11 @@ std::optional mappedTableWithOptional(Context & context, const St return std::make_pair(storage->getDatabaseName(), storage->getTableName()); } -QualifiedName mappedTable(Context & context, const String & database_name, const String & table_name, bool include_tombstone) +QualifiedName mappedTable( + Context & context, + const String & database_name, + const String & table_name, + bool include_tombstone) { auto mapped_db = mappedDatabase(context, database_name); diff --git a/dbms/src/Debug/dbgTools.h b/dbms/src/Debug/dbgTools.h index c4997e2cf04..f4426df9dba 100644 --- a/dbms/src/Debug/dbgTools.h +++ b/dbms/src/Debug/dbgTools.h @@ -47,7 +47,12 @@ RegionPtr createRegion( const HandleID & end, std::optional index = std::nullopt); -Regions createRegions(TableID table_id, size_t region_num, size_t key_num_each_region, HandleID handle_begin, RegionID new_region_id_begin); +Regions createRegions( + TableID table_id, + size_t region_num, + size_t key_num_each_region, + HandleID handle_begin, + RegionID new_region_id_begin); RegionPtr createRegion( const TiDB::TableInfo & table_info, @@ -57,11 +62,32 @@ RegionPtr createRegion( void encodeRow(const TiDB::TableInfo & table_info, const std::vector & fields, WriteBuffer & ss); -void insert(const TiDB::TableInfo & table_info, RegionID region_id, HandleID handle_id, ASTs::const_iterator begin, ASTs::const_iterator end, Context & context, const std::optional> & tso_del = {}); - -void addRequestsToRaftCmd(raft_cmdpb::RaftCmdRequest & request, const TiKVKey & key, const TiKVValue & value, UInt64 prewrite_ts, UInt64 commit_ts, bool del, const String pk = "pk"); - -void concurrentBatchInsert(const TiDB::TableInfo & table_info, Int64 concurrent_num, Int64 flush_num, Int64 batch_num, UInt64 min_strlen, UInt64 max_strlen, Context & context); +void insert( + const TiDB::TableInfo & table_info, + RegionID region_id, + HandleID handle_id, + ASTs::const_iterator begin, + ASTs::const_iterator end, + Context & context, + const std::optional> & tso_del = {}); + +void addRequestsToRaftCmd( + raft_cmdpb::RaftCmdRequest & request, + const TiKVKey & key, + const TiKVValue & value, + UInt64 prewrite_ts, + UInt64 commit_ts, + bool del, + const String pk = "pk"); + +void concurrentBatchInsert( + const TiDB::TableInfo & table_info, + Int64 concurrent_num, + Int64 flush_num, + Int64 batch_num, + UInt64 min_strlen, + UInt64 max_strlen, + Context & context); void remove(const TiDB::TableInfo & table_info, RegionID region_id, HandleID handle_id, Context & context); @@ -75,7 +101,11 @@ Int64 concurrentRangeOperate( Field convertField(const TiDB::ColumnInfo & column_info, const Field & field); -TableID getTableID(Context & context, const std::string & database_name, const std::string & table_name, const std::string & partition_id); +TableID getTableID( + Context & context, + const std::string & database_name, + const std::string & table_name, + const std::string & partition_id); const TiDB::TableInfo & getTableInfo(Context & context, const String & database_name, const String & table_name); @@ -94,7 +124,14 @@ namespace DB using QualifiedName = std::pair; String mappedDatabase(Context & context, const String & database_name); std::optional mappedDatabaseWithOptional(Context & context, const String & database_name); -std::optional mappedTableWithOptional(Context & context, const String & database_name, const String & table_name); -QualifiedName mappedTable(Context & context, const String & database_name, const String & table_name, bool include_tombstone = false); +std::optional mappedTableWithOptional( + Context & context, + const String & database_name, + const String & table_name); +QualifiedName mappedTable( + Context & context, + const String & database_name, + const String & table_name, + bool include_tombstone = false); } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index d6c3d348ee2..b3090dd976e 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -109,7 +109,11 @@ namespace DM // MergeDeltaTaskPool // ================================================ -std::pair DeltaMergeStore::MergeDeltaTaskPool::tryAddTask(const BackgroundTask & task, const ThreadType & whom, const size_t max_task_num, const LoggerPtr & log_) +std::pair DeltaMergeStore::MergeDeltaTaskPool::tryAddTask( + const BackgroundTask & task, + const ThreadType & whom, + const size_t max_task_num, + const LoggerPtr & log_) { std::scoped_lock lock(mutex); if (light_tasks.size() + heavy_tasks.size() >= max_task_num) @@ -159,7 +163,11 @@ DeltaMergeStore::BackgroundTask DeltaMergeStore::MergeDeltaTaskPool::nextTask(bo auto task = tasks.front(); tasks.pop(); - LOG_DEBUG(log_, "Segment task pop from background task pool, segment={} task={}", task.segment->simpleInfo(), magic_enum::enum_name(task.type)); + LOG_DEBUG( + log_, + "Segment task pop from background task pool, segment={} task={}", + task.segment->simpleInfo(), + magic_enum::enum_name(task.type)); return task; } @@ -189,23 +197,26 @@ ColumnDefinesPtr generateStoreColumns(const ColumnDefines & table_columns, bool } } // namespace -DeltaMergeStore::Settings DeltaMergeStore::EMPTY_SETTINGS = DeltaMergeStore::Settings{.not_compress_columns = NotCompress{}}; - -DeltaMergeStore::DeltaMergeStore(Context & db_context, - bool data_path_contains_database_name, - const String & db_name_, - const String & table_name_, - KeyspaceID keyspace_id_, - TableID physical_table_id_, - bool has_replica, - const ColumnDefines & columns, - const ColumnDefine & handle, - bool is_common_handle_, - size_t rowkey_column_size_, - const Settings & settings_, - ThreadPool * thread_pool) +DeltaMergeStore::Settings DeltaMergeStore::EMPTY_SETTINGS + = DeltaMergeStore::Settings{.not_compress_columns = NotCompress{}}; + +DeltaMergeStore::DeltaMergeStore( + Context & db_context, + bool data_path_contains_database_name, + const String & db_name_, + const String & table_name_, + KeyspaceID keyspace_id_, + TableID physical_table_id_, + bool has_replica, + const ColumnDefines & columns, + const ColumnDefine & handle, + bool is_common_handle_, + size_t rowkey_column_size_, + const Settings & settings_, + ThreadPool * thread_pool) : global_context(db_context.getGlobalContext()) - , path_pool(std::make_shared(global_context.getPathPool().withTable(db_name_, table_name_, data_path_contains_database_name))) + , path_pool(std::make_shared( + global_context.getPathPool().withTable(db_name_, table_name_, data_path_contains_database_name))) , settings(settings_) , db_name(db_name_) , table_name(table_name_) @@ -225,11 +236,8 @@ DeltaMergeStore::DeltaMergeStore(Context & db_context, LOG_INFO(log, "Restore DeltaMerge Store start"); - storage_pool = std::make_shared(global_context, - keyspace_id, - ns_id, - *path_pool, - db_name_ + "." + table_name_); + storage_pool + = std::make_shared(global_context, keyspace_id, ns_id, *path_pool, db_name_ + "." + table_name_); // Restore existing dm files. // Should be done before any background task setup. @@ -435,24 +443,29 @@ void DeltaMergeStore::shutdown() LOG_TRACE(log, "Shutdown DeltaMerge end"); } -DMContextPtr DeltaMergeStore::newDMContext(const Context & db_context, const DB::Settings & db_settings, const String & tracing_id, ScanContextPtr scan_context_) +DMContextPtr DeltaMergeStore::newDMContext( + const Context & db_context, + const DB::Settings & db_settings, + const String & tracing_id, + ScanContextPtr scan_context_) { std::shared_lock lock(read_write_mutex); // Here we use global context from db_context, instead of db_context directly. // Because db_context could be a temporary object and won't last long enough during the query process. // Like the context created by InterpreterSelectWithUnionQuery. - auto * ctx = new DMContext(db_context.getGlobalContext(), - path_pool, - storage_pool, - latest_gc_safe_point.load(std::memory_order_acquire), - keyspace_id, - physical_table_id, - is_common_handle, - rowkey_column_size, - db_settings, - scan_context_, - tracing_id); + auto * ctx = new DMContext( + db_context.getGlobalContext(), + path_pool, + storage_pool, + latest_gc_safe_point.load(std::memory_order_acquire), + keyspace_id, + physical_table_id, + is_common_handle, + rowkey_column_size, + db_settings, + scan_context_, + tracing_id); return DMContextPtr(ctx); } @@ -481,18 +494,22 @@ inline Block getSubBlock(const Block & block, size_t offset, size_t limit) // Add an extra handle column if the `handle_define` is used as the primary key // TODO: consider merging it into `RegionBlockReader`? -Block DeltaMergeStore::addExtraColumnIfNeed(const Context & db_context, const ColumnDefine & handle_define, Block && block) +Block DeltaMergeStore::addExtraColumnIfNeed( + const Context & db_context, + const ColumnDefine & handle_define, + Block && block) { if (pkIsHandle(handle_define)) { if (!EXTRA_HANDLE_COLUMN_INT_TYPE->equals(*handle_define.type)) { auto handle_pos = getPosByColumnId(block, handle_define.id); - addColumnToBlock(block, // - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_INT_TYPE, - EXTRA_HANDLE_COLUMN_INT_TYPE->createColumn()); + addColumnToBlock( + block, // + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_INT_TYPE, + EXTRA_HANDLE_COLUMN_INT_TYPE->createColumn()); // Fill the new handle column with data in column[handle_pos] by applying cast. DefaultExecutable(FunctionToInt64::create(db_context)).execute(block, {handle_pos}, block.columns() - 1); } @@ -503,11 +520,12 @@ Block DeltaMergeStore::addExtraColumnIfNeed(const Context & db_context, const Co auto pk_col_with_name = getByColumnId(block, handle_define.id); auto pk_column = pk_col_with_name.column; ColumnPtr handle_column = pk_column->cloneResized(pk_column->size()); - addColumnToBlock(block, // - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_INT_TYPE, - handle_column); + addColumnToBlock( + block, // + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_INT_TYPE, + handle_column); } } return std::move(block); @@ -556,7 +574,8 @@ DM::WriteResult DeltaMergeStore::write(const Context & db_context, const DB::Set while (true) { // Find the segment according to current start_key - auto [segment, is_empty] = getSegmentByStartKey(start_key, /*create_if_empty*/ true, /*throw_if_notfound*/ true); + auto [segment, is_empty] + = getSegmentByStartKey(start_key, /*create_if_empty*/ true, /*throw_if_notfound*/ true); FAIL_POINT_PAUSE(FailPoints::pause_when_writing_to_dt_store); @@ -570,20 +589,23 @@ DM::WriteResult DeltaMergeStore::write(const Context & db_context, const DB::Set // The [offset, rows - offset] can be exceeding the Segment's rowkey_range. Cut the range // to fit the segment. auto [cur_offset, cur_limit] = rowkey_range.getPosRange(handle_column, offset, rows - offset); - RUNTIME_CHECK_MSG(cur_offset == offset && cur_limit != 0, - "invalid cur_offset or cur_limit. is_common_handle={} start_key={} cur_offset={} cur_limit={} rows={} offset={} rowkey_range={}", - is_common_handle, - start_key.toRowKeyValue().toString(), - cur_offset, - cur_limit, - rows, - offset, - rowkey_range.toDebugString()); + RUNTIME_CHECK_MSG( + cur_offset == offset && cur_limit != 0, + "invalid cur_offset or cur_limit. is_common_handle={} start_key={} cur_offset={} cur_limit={} rows={} " + "offset={} rowkey_range={}", + is_common_handle, + start_key.toRowKeyValue().toString(), + cur_offset, + cur_limit, + rows, + offset, + rowkey_range.toDebugString()); limit = cur_limit; auto alloc_bytes = block.bytes(offset, limit); - bool is_small = limit < dm_context->delta_cache_limit_rows / 4 && alloc_bytes < dm_context->delta_cache_limit_bytes / 4; + bool is_small = limit < dm_context->delta_cache_limit_rows / 4 + && alloc_bytes < dm_context->delta_cache_limit_bytes / 4; // For small column files, data is appended to MemTableSet, then flushed later. // For large column files, data is directly written to PageStorage, while the ColumnFile entry is appended to MemTableSet. if (is_small) @@ -636,14 +658,24 @@ DM::WriteResult DeltaMergeStore::write(const Context & db_context, const DB::Set fiu_do_on(FailPoints::random_exception_after_dt_write_done, { static int num_call = 0; if (num_call++ % 10 == 7) - throw Exception("Fail point random_exception_after_dt_write_done is triggered.", ErrorCodes::FAIL_POINT_ERROR); + throw Exception( + "Fail point random_exception_after_dt_write_done is triggered.", + ErrorCodes::FAIL_POINT_ERROR); }); // TODO: Update the tracing_id before checkSegmentsUpdateForKVStore - return checkSegmentsUpdateForKVStore(dm_context, updated_segments.begin(), updated_segments.end(), ThreadType::Write, InputType::RaftLog); + return checkSegmentsUpdateForKVStore( + dm_context, + updated_segments.begin(), + updated_segments.end(), + ThreadType::Write, + InputType::RaftLog); } -void DeltaMergeStore::deleteRange(const Context & db_context, const DB::Settings & db_settings, const RowKeyRange & delete_range) +void DeltaMergeStore::deleteRange( + const Context & db_context, + const DB::Settings & db_settings, + const RowKeyRange & delete_range) { LOG_INFO(log, "Table delete range, range={}", delete_range.toDebugString()); @@ -665,7 +697,8 @@ void DeltaMergeStore::deleteRange(const Context & db_context, const DB::Settings // Keep trying until succeeded. while (true) { - auto [segment, is_empty] = getSegmentByStartKey(cur_range.getStart(), /*create_if_empty*/ true, /*throw_if_notfound*/ true); + auto [segment, is_empty] + = getSegmentByStartKey(cur_range.getStart(), /*create_if_empty*/ true, /*throw_if_notfound*/ true); waitForDeleteRange(dm_context, segment); if (segment->hasAbandoned()) @@ -709,7 +742,8 @@ bool DeltaMergeStore::flushCache(const DMContextPtr & dm_context, const RowKeyRa // Keep trying until succeeded if needed. while (true) { - auto [segment, is_empty] = getSegmentByStartKey(cur_range.getStart(), /*create_if_empty*/ false, /*throw_if_notfound*/ false); + auto [segment, is_empty] + = getSegmentByStartKey(cur_range.getStart(), /*create_if_empty*/ false, /*throw_if_notfound*/ false); RUNTIME_CHECK(segment != nullptr || is_empty, cur_range.getStart().toDebugString()); // No segment need to flush. if (unlikely(is_empty)) @@ -768,21 +802,28 @@ bool DeltaMergeStore::mergeDeltaAll(const Context & context) return all_succ; } -std::optional DeltaMergeStore::mergeDeltaBySegment(const Context & context, const RowKeyValue & start_key) +std::optional DeltaMergeStore::mergeDeltaBySegment( + const Context & context, + const RowKeyValue & start_key) { LOG_INFO(log, "Table mergeDeltaBySegment, start={}", start_key.toDebugString()); SYNC_FOR("before_DeltaMergeStore::mergeDeltaBySegment"); updateGCSafePoint(); - auto dm_context = newDMContext(context, context.getSettingsRef(), - /*tracing_id*/ fmt::format("mergeDeltaBySegment_{}", latest_gc_safe_point.load(std::memory_order_relaxed))); + auto dm_context = newDMContext( + context, + context.getSettingsRef(), + /*tracing_id*/ fmt::format("mergeDeltaBySegment_{}", latest_gc_safe_point.load(std::memory_order_relaxed))); size_t sleep_ms = 50; while (true) { - auto [segment, is_empty] = getSegmentByStartKey(start_key.toRowKeyValueRef(), /*create_if_empty*/ false, /*throw_if_notfound*/ false); + auto [segment, is_empty] = getSegmentByStartKey( + start_key.toRowKeyValueRef(), + /*create_if_empty*/ false, + /*throw_if_notfound*/ false); if (unlikely(segment == nullptr)) { return std::nullopt; @@ -797,7 +838,11 @@ std::optional DeltaMergeStore::mergeDeltaBySegment(const Contex if (unlikely(*segment_end.value <= *start_key.value)) { // The next start key must be > current start key - LOG_ERROR(log, "Assert new_segment.end {} > start {} failed", segment_end.toDebugString(), start_key.toDebugString()); + LOG_ERROR( + log, + "Assert new_segment.end {} > start {} failed", + segment_end.toDebugString(), + start_key.toDebugString()); throw Exception("Assert segment range failed", ErrorCodes::LOGICAL_ERROR); } return new_segment->getRowKeyRange(); @@ -831,7 +876,8 @@ void DeltaMergeStore::compact(const Context & db_context, const RowKeyRange & ra // Keep trying until succeeded. while (true) { - auto [segment, is_empty] = getSegmentByStartKey(cur_range.getStart(), /*create_if_empty*/ false, /*throw_if_notfound*/ false); + auto [segment, is_empty] + = getSegmentByStartKey(cur_range.getStart(), /*create_if_empty*/ false, /*throw_if_notfound*/ false); if (segment == nullptr) { return; @@ -852,13 +898,14 @@ void DeltaMergeStore::compact(const Context & db_context, const RowKeyRange & ra // Read data without mvcc filtering. // just for debug // readRaw is called under 'selraw xxxx' -BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, - const DB::Settings & db_settings, - const ColumnDefines & columns_to_read, - size_t num_streams, - bool keep_order, - const SegmentIdSet & read_segments, - size_t extra_table_id_index) +BlockInputStreams DeltaMergeStore::readRaw( + const Context & db_context, + const DB::Settings & db_settings, + const ColumnDefines & columns_to_read, + size_t num_streams, + bool keep_order, + const SegmentIdSet & read_segments, + size_t extra_table_id_index) { SegmentReadTasks tasks; @@ -877,7 +924,8 @@ BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, if (unlikely(!segment_snap)) throw Exception("Failed to get segment snap", ErrorCodes::LOGICAL_ERROR); - tasks.push_back(std::make_shared(segment, segment_snap, RowKeyRanges{segment->getRowKeyRange()})); + tasks.push_back( + std::make_shared(segment, segment_snap, RowKeyRanges{segment->getRowKeyRange()})); } } } @@ -938,10 +986,8 @@ BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, DEFAULT_BLOCK_SIZE, /* read_mode */ ReadMode::Raw, req_info); - stream = std::make_shared( - stream, - extra_table_id_index, - physical_table_id); + stream + = std::make_shared(stream, extra_table_id_index, physical_table_id); } res.push_back(stream); } @@ -979,7 +1025,8 @@ void DeltaMergeStore::readRaw( if (unlikely(!segment_snap)) throw Exception("Failed to get segment snap", ErrorCodes::LOGICAL_ERROR); - tasks.push_back(std::make_shared(segment, segment_snap, RowKeyRanges{segment->getRowKeyRange()})); + tasks.push_back( + std::make_shared(segment, segment_snap, RowKeyRanges{segment->getRowKeyRange()})); } } } @@ -997,9 +1044,8 @@ void DeltaMergeStore::readRaw( auto after_segment_read = [&](const DMContextPtr & dm_context_, const SegmentPtr & segment_) { this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read, InputType::NotRaft); }; - size_t final_num_stream = enable_read_thread - ? std::max(1, num_streams) - : std::max(1, std::min(num_streams, tasks.size())); + size_t final_num_stream + = enable_read_thread ? std::max(1, num_streams) : std::max(1, std::min(num_streams, tasks.size())); String req_info; if (db_context.getDAGContext() != nullptr && db_context.getDAGContext()->isMPPTask()) req_info = db_context.getDAGContext()->getMPPTaskId().toString(); @@ -1022,13 +1068,12 @@ void DeltaMergeStore::readRaw( { for (size_t i = 0; i < final_num_stream; ++i) { - group_builder.addConcurrency( - std::make_unique( - exec_context, - read_task_pool, - columns_to_read, - extra_table_id_index, - req_info)); + group_builder.addConcurrency(std::make_unique( + exec_context, + read_task_pool, + columns_to_read, + extra_table_id_index, + req_info)); } } else @@ -1071,32 +1116,38 @@ static ReadMode getReadModeImpl(const Context & db_context, bool is_fast_scan, b return ReadMode::Normal; } -ReadMode DeltaMergeStore::getReadMode(const Context & db_context, bool is_fast_scan, bool keep_order, const PushDownFilterPtr & filter) +ReadMode DeltaMergeStore::getReadMode( + const Context & db_context, + bool is_fast_scan, + bool keep_order, + const PushDownFilterPtr & filter) { auto read_mode = getReadModeImpl(db_context, is_fast_scan, keep_order); - RUNTIME_CHECK_MSG(!filter || !filter->before_where || read_mode == ReadMode::Bitmap, - "Push down filters needs bitmap, push down filters is empty: {}, read mode: {}", - filter == nullptr || filter->before_where == nullptr, - magic_enum::enum_name(read_mode)); + RUNTIME_CHECK_MSG( + !filter || !filter->before_where || read_mode == ReadMode::Bitmap, + "Push down filters needs bitmap, push down filters is empty: {}, read mode: {}", + filter == nullptr || filter->before_where == nullptr, + magic_enum::enum_name(read_mode)); return read_mode; } -BlockInputStreams DeltaMergeStore::read(const Context & db_context, - const DB::Settings & db_settings, - const ColumnDefines & columns_to_read, - const RowKeyRanges & sorted_ranges, - size_t num_streams, - UInt64 max_version, - const PushDownFilterPtr & filter, - const RuntimeFilteList & runtime_filter_list, - const int rf_max_wait_time_ms, - const String & tracing_id, - bool keep_order, - bool is_fast_scan, - size_t expected_block_size, - const SegmentIdSet & read_segments, - size_t extra_table_id_index, - ScanContextPtr scan_context) +BlockInputStreams DeltaMergeStore::read( + const Context & db_context, + const DB::Settings & db_settings, + const ColumnDefines & columns_to_read, + const RowKeyRanges & sorted_ranges, + size_t num_streams, + UInt64 max_version, + const PushDownFilterPtr & filter, + const RuntimeFilteList & runtime_filter_list, + const int rf_max_wait_time_ms, + const String & tracing_id, + bool keep_order, + bool is_fast_scan, + size_t expected_block_size, + const SegmentIdSet & read_segments, + size_t extra_table_id_index, + ScanContextPtr scan_context) { // Use the id from MPP/Coprocessor level as tracing_id auto dm_context = newDMContext(db_context, db_settings, tracing_id, scan_context); @@ -1106,16 +1157,23 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, // SegmentReadTaskScheduler and SegmentReadTaskPool use table_id + segment id as unique ID when read thread is enabled. // 'try_split_task' can result in several read tasks with the same id that can cause some trouble. // Also, too many read tasks of a segment with different small ranges is not good for data sharing cache. - SegmentReadTasks tasks = getReadTasksByRanges(*dm_context, sorted_ranges, num_streams, read_segments, /*try_split_task =*/!enable_read_thread); + SegmentReadTasks tasks = getReadTasksByRanges( + *dm_context, + sorted_ranges, + num_streams, + read_segments, + /*try_split_task =*/!enable_read_thread); auto log_tracing_id = getLogTracingId(*dm_context); auto tracing_logger = log->getChild(log_tracing_id); - LOG_INFO(tracing_logger, - "Read create segment snapshot done, keep_order={} dt_enable_read_thread={} enable_read_thread={} is_fast_scan={} is_push_down_filter_empty={}", - keep_order, - db_context.getSettingsRef().dt_enable_read_thread, - enable_read_thread, - is_fast_scan, - filter == nullptr || filter->before_where == nullptr); + LOG_INFO( + tracing_logger, + "Read create segment snapshot done, keep_order={} dt_enable_read_thread={} enable_read_thread={} " + "is_fast_scan={} is_push_down_filter_empty={}", + keep_order, + db_context.getSettingsRef().dt_enable_read_thread, + enable_read_thread, + is_fast_scan, + filter == nullptr || filter->before_where == nullptr); auto after_segment_read = [&](const DMContextPtr & dm_context_, const SegmentPtr & segment_) { // TODO: Update the tracing_id before checkSegmentUpdate? @@ -1166,10 +1224,8 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, expected_block_size, /* read_mode = */ is_fast_scan ? ReadMode::Fast : ReadMode::Normal, log_tracing_id); - stream = std::make_shared( - stream, - extra_table_id_index, - physical_table_id); + stream + = std::make_shared(stream, extra_table_id_index, physical_table_id); } res.push_back(stream); } @@ -1206,14 +1262,20 @@ void DeltaMergeStore::read( // SegmentReadTaskScheduler and SegmentReadTaskPool use table_id + segment id as unique ID when read thread is enabled. // 'try_split_task' can result in several read tasks with the same id that can cause some trouble. // Also, too many read tasks of a segment with different small ranges is not good for data sharing cache. - SegmentReadTasks tasks = getReadTasksByRanges(*dm_context, sorted_ranges, num_streams, read_segments, /*try_split_task =*/!enable_read_thread); + SegmentReadTasks tasks = getReadTasksByRanges( + *dm_context, + sorted_ranges, + num_streams, + read_segments, + /*try_split_task =*/!enable_read_thread); auto log_tracing_id = getLogTracingId(*dm_context); auto tracing_logger = log->getChild(log_tracing_id); - LOG_DEBUG(tracing_logger, - "Read create segment snapshot done, keep_order={} dt_enable_read_thread={} enable_read_thread={}", - keep_order, - db_context.getSettingsRef().dt_enable_read_thread, - enable_read_thread); + LOG_DEBUG( + tracing_logger, + "Read create segment snapshot done, keep_order={} dt_enable_read_thread={} enable_read_thread={}", + keep_order, + db_context.getSettingsRef().dt_enable_read_thread, + enable_read_thread); auto after_segment_read = [&](const DMContextPtr & dm_context_, const SegmentPtr & segment_) { // TODO: Update the tracing_id before checkSegmentUpdate? @@ -1221,9 +1283,8 @@ void DeltaMergeStore::read( }; GET_METRIC(tiflash_storage_read_tasks_count).Increment(tasks.size()); - size_t final_num_stream = enable_read_thread - ? std::max(1, num_streams) - : std::max(1, std::min(num_streams, tasks.size())); + size_t final_num_stream + = enable_read_thread ? std::max(1, num_streams) : std::max(1, std::min(num_streams, tasks.size())); auto read_mode = getReadMode(db_context, is_fast_scan, keep_order, filter); auto read_task_pool = std::make_shared( physical_table_id, @@ -1244,15 +1305,14 @@ void DeltaMergeStore::read( { for (size_t i = 0; i < final_num_stream; ++i) { - group_builder.addConcurrency( - std::make_unique( - exec_context, - read_task_pool, - filter && filter->extra_cast ? *filter->columns_after_cast : columns_to_read, - extra_table_id_index, - log_tracing_id, - runtime_filter_list, - rf_max_wait_time_ms)); + group_builder.addConcurrency(std::make_unique( + exec_context, + read_task_pool, + filter && filter->extra_cast ? *filter->columns_after_cast : columns_to_read, + extra_table_id_index, + log_tracing_id, + runtime_filter_list, + rf_max_wait_time_ms)); } } else @@ -1284,8 +1344,7 @@ void DeltaMergeStore::read( LOG_DEBUG(tracing_logger, "Read create PipelineExec done"); } -Remote::DisaggPhysicalTableReadSnapshotPtr -DeltaMergeStore::writeNodeBuildRemoteReadSnapshot( +Remote::DisaggPhysicalTableReadSnapshotPtr DeltaMergeStore::writeNodeBuildRemoteReadSnapshot( const Context & db_context, const DB::Settings & db_settings, const RowKeyRanges & sorted_ranges, @@ -1302,11 +1361,14 @@ DeltaMergeStore::writeNodeBuildRemoteReadSnapshot( // could fetch the data segment by segment with these snapshots later. // `try_split_task` is false because we need to ensure only one segment task // for one segment. - SegmentReadTasks tasks = getReadTasksByRanges(*dm_context, sorted_ranges, num_streams, read_segments, /* try_split_task */ false); + SegmentReadTasks tasks + = getReadTasksByRanges(*dm_context, sorted_ranges, num_streams, read_segments, /* try_split_task */ false); GET_METRIC(tiflash_disaggregated_read_tasks_count).Increment(tasks.size()); LOG_DEBUG(tracing_logger, "Read create segment snapshot done"); - return std::make_unique(KeyspaceTableID{keyspace_id, physical_table_id}, std::move(tasks)); + return std::make_unique( + KeyspaceTableID{keyspace_id, physical_table_id}, + std::move(tasks)); } size_t forceMergeDeltaRows(const DMContextPtr & dm_context) @@ -1335,7 +1397,8 @@ void DeltaMergeStore::waitForWrite(const DMContextPtr & dm_context, const Segmen // FIXME: checkSegmentUpdate will also count write stalls at each call. Stopwatch watch; - SCOPE_EXIT({ GET_METRIC(tiflash_storage_write_stall_duration_seconds, type_write).Observe(watch.elapsedSeconds()); }); + SCOPE_EXIT( + { GET_METRIC(tiflash_storage_write_stall_duration_seconds, type_write).Observe(watch.elapsedSeconds()); }); size_t segment_bytes = segment->getEstimatedBytes(); // The speed of delta merge in a very bad situation we assume. It should be a very conservative value. @@ -1378,7 +1441,11 @@ void DeltaMergeStore::waitForDeleteRange(const DB::DM::DMContextPtr &, const DB: // TODO: maybe we should wait, if there are too many delete ranges? } -bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const SegmentPtr & segment, ThreadType thread_type, InputType input_type) +bool DeltaMergeStore::checkSegmentUpdate( + const DMContextPtr & dm_context, + const SegmentPtr & segment, + ThreadType thread_type, + InputType input_type) { bool should_trigger_foreground_kvstore_flush = false; fiu_do_on(FailPoints::skip_check_segment_update, { return should_trigger_foreground_kvstore_flush; }); @@ -1422,20 +1489,24 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto delta_cache_limit_bytes = dm_context->delta_cache_limit_bytes; // TODO(proactive flush) - bool should_background_flush = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes) // + bool should_background_flush + = (unsaved_rows >= delta_cache_limit_rows || unsaved_bytes >= delta_cache_limit_bytes) // && (delta_rows - delta_last_try_flush_rows >= delta_cache_limit_rows || delta_bytes - delta_last_try_flush_bytes >= delta_cache_limit_bytes); - bool should_foreground_flush = unsaved_rows >= delta_cache_limit_rows * 3 || unsaved_bytes >= delta_cache_limit_bytes * 3; + bool should_foreground_flush + = unsaved_rows >= delta_cache_limit_rows * 3 || unsaved_bytes >= delta_cache_limit_bytes * 3; /// For write thread, we want to avoid foreground flush to block the process of apply raft command. /// So we increase the threshold of foreground flush for write thread. if (thread_type == ThreadType::Write) { - should_foreground_flush = unsaved_rows >= delta_cache_limit_rows * 10 || unsaved_bytes >= delta_cache_limit_bytes * 10; + should_foreground_flush + = unsaved_rows >= delta_cache_limit_rows * 10 || unsaved_bytes >= delta_cache_limit_bytes * 10; } - bool should_background_merge_delta = ((delta_check_rows >= delta_limit_rows || delta_check_bytes >= delta_limit_bytes) // - && (delta_rows - delta_last_try_merge_delta_rows >= delta_cache_limit_rows - || delta_bytes - delta_last_try_merge_delta_bytes >= delta_cache_limit_bytes)); + bool should_background_merge_delta + = ((delta_check_rows >= delta_limit_rows || delta_check_bytes >= delta_limit_bytes) // + && (delta_rows - delta_last_try_merge_delta_rows >= delta_cache_limit_rows + || delta_bytes - delta_last_try_merge_delta_bytes >= delta_cache_limit_bytes)); bool should_foreground_merge_delta_by_rows_or_bytes = delta_check_rows >= forceMergeDeltaRows(dm_context) || delta_check_bytes >= forceMergeDeltaBytes(dm_context); bool should_foreground_merge_delta_by_deletes = delta_deletes >= forceMergeDeltaDeletes(dm_context); @@ -1451,7 +1522,8 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const || (segment_rows >= segment_limit_rows * 3 || segment_bytes >= segment_limit_bytes * 3); // Don't do compact on starting up. - bool should_compact = (thread_type != ThreadType::Init) && std::max(static_cast(column_file_count) - delta_last_try_compact_column_files, 0) >= 15; + bool should_compact = (thread_type != ThreadType::Init) + && std::max(static_cast(column_file_count) - delta_last_try_compact_column_files, 0) >= 15; // Don't do background place index if we limit DeltaIndex cache. bool should_place_delta_index = !dm_context->db_context.isDeltaIndexLimited() @@ -1479,7 +1551,11 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const if (shutdown_called.load(std::memory_order_relaxed)) return; - auto [added, heavy] = background_tasks.tryAddTask(task, thread_type, std::max(id_to_segment.size() * 2, background_pool.getNumberOfThreads() * 3), log); + auto [added, heavy] = background_tasks.tryAddTask( + task, + thread_type, + std::max(id_to_segment.size() * 2, background_pool.getNumberOfThreads() * 3), + log); // Prevent too many tasks. if (!added) return; @@ -1504,13 +1580,19 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const { // FIXME: We'd better count write stall duration for per-write, instead of per-call, // in order to produce a meaningful value. - GET_METRIC(tiflash_storage_write_stall_duration_seconds, type_flush).Observe(watch.elapsedSeconds()); + GET_METRIC(tiflash_storage_write_stall_duration_seconds, type_flush) + .Observe(watch.elapsedSeconds()); } }); delta_last_try_flush_rows = delta_rows; delta_last_try_flush_bytes = delta_bytes; - LOG_DEBUG(log, "Foreground flush cache in checkSegmentUpdate, thread={} segment={} input_type={}", thread_type, segment->info(), magic_enum::enum_name(input_type)); + LOG_DEBUG( + log, + "Foreground flush cache in checkSegmentUpdate, thread={} segment={} input_type={}", + thread_type, + segment->info(), + magic_enum::enum_name(input_type)); segment->flushCache(*dm_context); if (input_type == InputType::RaftLog) { @@ -1545,7 +1627,8 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto try_fg_merge_delta = [&]() -> SegmentPtr { // If the table is already dropped, don't trigger foreground merge delta when executing `remove region peer`, // or the raft-log apply threads may be blocked. - if ((should_foreground_merge_delta_by_rows_or_bytes || should_foreground_merge_delta_by_deletes) && replica_exist.load()) + if ((should_foreground_merge_delta_by_rows_or_bytes || should_foreground_merge_delta_by_deletes) + && replica_exist.load()) { delta_last_try_merge_delta_rows = delta_rows; @@ -1556,9 +1639,11 @@ bool DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const // FIXME: We'd better count write stall duration for per-write, instead of per-call, // in order to produce a meaningful value. if (should_foreground_merge_delta_by_rows_or_bytes) - GET_METRIC(tiflash_storage_write_stall_duration_seconds, type_delta_merge_by_write).Observe(watch.elapsedSeconds()); + GET_METRIC(tiflash_storage_write_stall_duration_seconds, type_delta_merge_by_write) + .Observe(watch.elapsedSeconds()); if (should_foreground_merge_delta_by_deletes) - GET_METRIC(tiflash_storage_write_stall_duration_seconds, type_delta_merge_by_delete_range).Observe(watch.elapsedSeconds()); + GET_METRIC(tiflash_storage_write_stall_duration_seconds, type_delta_merge_by_delete_range) + .Observe(watch.elapsedSeconds()); }); return segmentMergeDelta(*dm_context, segment, MergeDeltaReason::ForegroundWrite); @@ -1700,11 +1785,11 @@ void DeltaMergeStore::check(const Context & /*db_context*/) throw Exception(fmt::format("Segment [{}] is expected to have id [{}]", segment_id, next_segment_id)); } if (compare(last_end.data, last_end.size, range.getStart().data, range.getStart().size) != 0) - throw Exception( - fmt::format("Segment [{}:{}] is expected to have the same start edge value like the end edge value in {}", - segment_id, - range.toDebugString(), - last_range.toDebugString())); + throw Exception(fmt::format( + "Segment [{}:{}] is expected to have the same start edge value like the end edge value in {}", + segment_id, + range.toDebugString(), + last_range.toDebugString())); last_range = range; last_end = last_range.getEnd(); @@ -1901,13 +1986,17 @@ SegmentReadTasks DeltaMergeStore::getReadTasksByRanges( auto range_it = sorted_ranges.begin(); auto seg_it = segments.upper_bound(range_it->getStart()); - RUNTIME_CHECK_MSG(seg_it != segments.end(), "Failed to locate segment begin with start in range: {}", range_it->toDebugString()); + RUNTIME_CHECK_MSG( + seg_it != segments.end(), + "Failed to locate segment begin with start in range: {}", + range_it->toDebugString()); while (range_it != sorted_ranges.end() && seg_it != segments.end()) { const auto & req_range = *range_it; const auto & seg_range = seg_it->second->getRowKeyRange(); - if (req_range.intersect(seg_range) && (read_segments.empty() || read_segments.count(seg_it->second->segmentId()))) + if (req_range.intersect(seg_range) + && (read_segments.empty() || read_segments.count(seg_it->second->segmentId()))) { if (tasks.empty() || tasks.back()->segment != seg_it->second) { @@ -1959,7 +2048,8 @@ SegmentReadTasks DeltaMergeStore::getReadTasksByRanges( auto tracing_logger = log->getChild(getLogTracingId(dm_context)); LOG_INFO( tracing_logger, - "Segment read tasks build done, cost={}ms sorted_ranges={} n_tasks_before_split={} n_tasks_final={} n_ranges_final={}", + "Segment read tasks build done, cost={}ms sorted_ranges={} n_tasks_before_split={} n_tasks_final={} " + "n_ranges_final={}", watch.elapsedMilliseconds(), sorted_ranges.size(), tasks_before_split, @@ -1985,10 +2075,14 @@ std::pair DeltaMergeStore::getSegmentByStartKeyInner(const Row { std::shared_lock lock(read_write_mutex); auto seg_it = segments.upper_bound(start_key); - return seg_it != segments.end() ? std::pair{seg_it->second, segments.empty()} : std::pair{nullptr, segments.empty()}; + return seg_it != segments.end() ? std::pair{seg_it->second, segments.empty()} + : std::pair{nullptr, segments.empty()}; } -std::pair DeltaMergeStore::getSegmentByStartKey(const RowKeyValueRef & start_key, bool create_if_empty, bool throw_if_notfound) +std::pair DeltaMergeStore::getSegmentByStartKey( + const RowKeyValueRef & start_key, + bool create_if_empty, + bool throw_if_notfound) { SegmentPtr seg; bool is_empty = false; @@ -2010,7 +2104,10 @@ std::pair DeltaMergeStore::getSegmentByStartKey(const RowKeyVa if (throw_if_notfound) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to locate segment begin with start in range: {}", start_key.toDebugString()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Failed to locate segment begin with start in range: {}", + start_key.toDebugString()); } else { diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index ad4d62a457c..5642315febd 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -250,36 +250,35 @@ class DeltaMergeStore : private boost::noncopyable // first element of return value means whether task is added or not // second element of return value means whether task is heavy or not - std::pair tryAddTask(const BackgroundTask & task, const ThreadType & whom, size_t max_task_num, const LoggerPtr & log_); + std::pair tryAddTask( + const BackgroundTask & task, + const ThreadType & whom, + size_t max_task_num, + const LoggerPtr & log_); BackgroundTask nextTask(bool is_heavy, const LoggerPtr & log_); }; - DeltaMergeStore(Context & db_context, // - bool data_path_contains_database_name, - const String & db_name, - const String & table_name_, - KeyspaceID keyspace_id_, - TableID physical_table_id_, - bool has_replica, - const ColumnDefines & columns, - const ColumnDefine & handle, - bool is_common_handle_, - size_t rowkey_column_size_, - const Settings & settings_ = EMPTY_SETTINGS, - ThreadPool * thread_pool = nullptr); + DeltaMergeStore( + Context & db_context, // + bool data_path_contains_database_name, + const String & db_name, + const String & table_name_, + KeyspaceID keyspace_id_, + TableID physical_table_id_, + bool has_replica, + const ColumnDefines & columns, + const ColumnDefine & handle, + bool is_common_handle_, + size_t rowkey_column_size_, + const Settings & settings_ = EMPTY_SETTINGS, + ThreadPool * thread_pool = nullptr); ~DeltaMergeStore(); void setUpBackgroundTask(const DMContextPtr & dm_context); - const String & getDatabaseName() const - { - return db_name; - } - const String & getTableName() const - { - return table_name; - } + const String & getDatabaseName() const { return db_name; } + const String & getTableName() const { return table_name; } void rename(String new_path, String new_database_name, String new_table_name); @@ -301,26 +300,29 @@ class DeltaMergeStore : private boost::noncopyable void preIngestFile(const String & parent_path, PageIdU64 file_id, size_t file_size); void removePreIngestFile(PageIdU64 file_id, bool throw_on_not_exist); - void cleanPreIngestFiles(const Context & db_context, - const DB::Settings & db_settings, - const std::vector & external_files); + void cleanPreIngestFiles( + const Context & db_context, + const DB::Settings & db_settings, + const std::vector & external_files); /// You must ensure external files are ordered and do not overlap. Otherwise exceptions will be thrown. /// You must ensure all of the external files are contained by the range. Otherwise exceptions will be thrown. /// Return the 'ingested bytes'. - UInt64 ingestFiles(const DMContextPtr & dm_context, // - const RowKeyRange & range, - const std::vector & external_files, - bool clear_data_in_range); + UInt64 ingestFiles( + const DMContextPtr & dm_context, // + const RowKeyRange & range, + const std::vector & external_files, + bool clear_data_in_range); /// You must ensure external files are ordered and do not overlap. Otherwise exceptions will be thrown. /// You must ensure all of the external files are contained by the range. Otherwise exceptions will be thrown. /// Return the 'ingtested bytes'. - UInt64 ingestFiles(const Context & db_context, // - const DB::Settings & db_settings, - const RowKeyRange & range, - const std::vector & external_files, - bool clear_data_in_range) + UInt64 ingestFiles( + const Context & db_context, // + const DB::Settings & db_settings, + const RowKeyRange & range, + const std::vector & external_files, + bool clear_data_in_range) { auto dm_context = newDMContext(db_context, db_settings); return ingestFiles(dm_context, range, external_files, clear_data_in_range); @@ -337,27 +339,30 @@ class DeltaMergeStore : private boost::noncopyable const RowKeyRange & ingest_range, const SegmentPtr & segment_to_ingest); - void ingestSegmentsFromCheckpointInfo(const DMContextPtr & dm_context, - const DM::RowKeyRange & range, - CheckpointInfoPtr checkpoint_info); + void ingestSegmentsFromCheckpointInfo( + const DMContextPtr & dm_context, + const DM::RowKeyRange & range, + CheckpointInfoPtr checkpoint_info); - void ingestSegmentsFromCheckpointInfo(const Context & db_context, - const DB::Settings & db_settings, - const DM::RowKeyRange & range, - CheckpointInfoPtr checkpoint_info) + void ingestSegmentsFromCheckpointInfo( + const Context & db_context, + const DB::Settings & db_settings, + const DM::RowKeyRange & range, + CheckpointInfoPtr checkpoint_info) { auto dm_context = newDMContext(db_context, db_settings); return ingestSegmentsFromCheckpointInfo(dm_context, range, checkpoint_info); } /// Read all rows without MVCC filtering - BlockInputStreams readRaw(const Context & db_context, - const DB::Settings & db_settings, - const ColumnDefines & columns_to_read, - size_t num_streams, - bool keep_order, - const SegmentIdSet & read_segments = {}, - size_t extra_table_id_index = InvalidColumnID); + BlockInputStreams readRaw( + const Context & db_context, + const DB::Settings & db_settings, + const ColumnDefines & columns_to_read, + size_t num_streams, + bool keep_order, + const SegmentIdSet & read_segments = {}, + size_t extra_table_id_index = InvalidColumnID); /// Read all rows without MVCC filtering void readRaw( @@ -375,22 +380,23 @@ class DeltaMergeStore : private boost::noncopyable /// when is_fast_scan == false, we will read rows with MVCC filtering, del mark !=0 filter and sorted merge. /// when is_fast_scan == true, we will read rows without MVCC and sorted merge. /// `sorted_ranges` should be already sorted and merged. - BlockInputStreams read(const Context & db_context, - const DB::Settings & db_settings, - const ColumnDefines & columns_to_read, - const RowKeyRanges & sorted_ranges, - size_t num_streams, - UInt64 max_version, - const PushDownFilterPtr & filter, - const RuntimeFilteList & runtime_filter_list, - const int rf_max_wait_time_ms, - const String & tracing_id, - bool keep_order, - bool is_fast_scan = false, - size_t expected_block_size = DEFAULT_BLOCK_SIZE, - const SegmentIdSet & read_segments = {}, - size_t extra_table_id_index = InvalidColumnID, - ScanContextPtr scan_context = nullptr); + BlockInputStreams read( + const Context & db_context, + const DB::Settings & db_settings, + const ColumnDefines & columns_to_read, + const RowKeyRanges & sorted_ranges, + size_t num_streams, + UInt64 max_version, + const PushDownFilterPtr & filter, + const RuntimeFilteList & runtime_filter_list, + const int rf_max_wait_time_ms, + const String & tracing_id, + bool keep_order, + bool is_fast_scan = false, + size_t expected_block_size = DEFAULT_BLOCK_SIZE, + const SegmentIdSet & read_segments = {}, + size_t extra_table_id_index = InvalidColumnID, + ScanContextPtr scan_context = nullptr); /// Read rows in two modes: @@ -417,8 +423,7 @@ class DeltaMergeStore : private boost::noncopyable size_t extra_table_id_index = InvalidColumnID, ScanContextPtr scan_context = nullptr); - Remote::DisaggPhysicalTableReadSnapshotPtr - writeNodeBuildRemoteReadSnapshot( + Remote::DisaggPhysicalTableReadSnapshotPtr writeNodeBuildRemoteReadSnapshot( const Context & db_context, const DB::Settings & db_settings, const RowKeyRanges & sorted_ranges, @@ -462,7 +467,12 @@ class DeltaMergeStore : private boost::noncopyable * Try to merge delta in the current thread as the GC operation. * This function may be blocking, and should be called in the GC background thread. */ - SegmentPtr gcTrySegmentMergeDelta(const DMContextPtr & dm_context, const SegmentPtr & segment, const SegmentPtr & prev_segment, const SegmentPtr & next_segment, DB::Timestamp gc_safe_point); + SegmentPtr gcTrySegmentMergeDelta( + const DMContextPtr & dm_context, + const SegmentPtr & segment, + const SegmentPtr & prev_segment, + const SegmentPtr & next_segment, + DB::Timestamp gc_safe_point); /** * Starting from the given base segment, find continuous segments that could be merged. @@ -481,23 +491,11 @@ class DeltaMergeStore : private boost::noncopyable std::shared_lock lock(read_write_mutex); return store_columns; } - const ColumnDefines & getTableColumns() const - { - return original_table_columns; - } - const ColumnDefine & getHandle() const - { - return original_table_handle_define; - } + const ColumnDefines & getTableColumns() const { return original_table_columns; } + const ColumnDefine & getHandle() const { return original_table_handle_define; } BlockPtr getHeader() const; - const Settings & getSettings() const - { - return settings; - } - DataTypePtr getPKDataType() const - { - return original_table_handle_define.type; - } + const Settings & getSettings() const { return settings; } + DataTypePtr getPKDataType() const { return original_table_handle_define.type; } SortDescription getPrimarySortDescription() const; void check(const Context & db_context); @@ -505,16 +503,14 @@ class DeltaMergeStore : private boost::noncopyable StoreStats getStoreStats(); SegmentsStats getSegmentsStats(); - bool isCommonHandle() const - { - return is_common_handle; - } - size_t getRowKeyColumnSize() const - { - return rowkey_column_size; - } + bool isCommonHandle() const { return is_common_handle; } + size_t getRowKeyColumnSize() const { return rowkey_column_size; } - static ReadMode getReadMode(const Context & db_context, bool is_fast_scan, bool keep_order, const PushDownFilterPtr & filter); + static ReadMode getReadMode( + const Context & db_context, + bool is_fast_scan, + bool keep_order, + const PushDownFilterPtr & filter); public: /// Methods mainly used by region split. @@ -526,12 +522,13 @@ class DeltaMergeStore : private boost::noncopyable private: #endif - DMContextPtr newDMContext(const Context & db_context, const DB::Settings & db_settings, const String & tracing_id = "", ScanContextPtr scan_context = nullptr); + DMContextPtr newDMContext( + const Context & db_context, + const DB::Settings & db_settings, + const String & tracing_id = "", + ScanContextPtr scan_context = nullptr); - static bool pkIsHandle(const ColumnDefine & handle_define) - { - return handle_define.id != EXTRA_HANDLE_COLUMN_ID; - } + static bool pkIsHandle(const ColumnDefine & handle_define) { return handle_define.id != EXTRA_HANDLE_COLUMN_ID; } /// Try to stall the writing. It will suspend the current thread if flow control is necessary. /// There are roughly two flow control mechanisms: @@ -546,7 +543,12 @@ class DeltaMergeStore : private boost::noncopyable /// by returning a non-empty DM::WriteResult. // Deferencing `Iter` can get a pointer to a Segment. template - DM::WriteResult checkSegmentsUpdateForKVStore(const DMContextPtr & context, Iter begin, Iter end, ThreadType thread_type, InputType input_type) + DM::WriteResult checkSegmentsUpdateForKVStore( + const DMContextPtr & context, + Iter begin, + Iter end, + ThreadType thread_type, + InputType input_type) { DM::WriteResult result = std::nullopt; std::vector ranges; @@ -562,10 +564,7 @@ class DeltaMergeStore : private boost::noncopyable // TODO We can try merge ranges here. if (!ranges.empty()) { - result = RaftWriteResult{ - std::move(ranges), - keyspace_id, - physical_table_id}; + result = RaftWriteResult{std::move(ranges), keyspace_id, physical_table_id}; } return result; } @@ -608,7 +607,12 @@ class DeltaMergeStore : private boost::noncopyable * * When `opt_split_at` is not specified, this function will try to find a mid point for splitting, and may lead to failures. */ - SegmentPair segmentSplit(DMContext & dm_context, const SegmentPtr & segment, SegmentSplitReason reason, std::optional opt_split_at = std::nullopt, SegmentSplitMode opt_split_mode = SegmentSplitMode::Auto); + SegmentPair segmentSplit( + DMContext & dm_context, + const SegmentPtr & segment, + SegmentSplitReason reason, + std::optional opt_split_at = std::nullopt, + SegmentSplitMode opt_split_mode = SegmentSplitMode::Auto); enum class SegmentMergeReason { @@ -621,7 +625,10 @@ class DeltaMergeStore : private boost::noncopyable * Fail if given segments are not continuous or not valid. * After merging, all specified segments will be abandoned (with `segment->hasAbandoned() == true`). */ - SegmentPtr segmentMerge(DMContext & dm_context, const std::vector & ordered_segments, SegmentMergeReason reason); + SegmentPtr segmentMerge( + DMContext & dm_context, + const std::vector & ordered_segments, + SegmentMergeReason reason); enum class MergeDeltaReason { @@ -725,11 +732,12 @@ class DeltaMergeStore : private boost::noncopyable void restoreStableFilesFromLocal() const; void removeLocalStableFilesIfDisagg() const; - SegmentReadTasks getReadTasksByRanges(DMContext & dm_context, - const RowKeyRanges & sorted_ranges, - size_t expected_tasks_count = 1, - const SegmentIdSet & read_segments = {}, - bool try_split_task = true); + SegmentReadTasks getReadTasksByRanges( + DMContext & dm_context, + const RowKeyRanges & sorted_ranges, + size_t expected_tasks_count = 1, + const SegmentIdSet & read_segments = {}, + bool try_split_task = true); private: /** @@ -743,7 +751,11 @@ class DeltaMergeStore : private boost::noncopyable * * It returns a bool which indicates whether a flush of KVStore is recommended. */ - bool checkSegmentUpdate(const DMContextPtr & context, const SegmentPtr & segment, ThreadType thread_type, InputType input_type); + bool checkSegmentUpdate( + const DMContextPtr & context, + const SegmentPtr & segment, + ThreadType thread_type, + InputType input_type); #ifndef DBMS_PUBLIC_GTEST private: #else @@ -753,7 +765,10 @@ class DeltaMergeStore : private boost::noncopyable String getLogTracingId(const DMContext & dm_ctx); // Returns segment that contains start_key and whether 'segments' is empty. std::pair getSegmentByStartKeyInner(const RowKeyValueRef & start_key); - std::pair getSegmentByStartKey(const RowKeyValueRef & start_key, bool create_if_empty, bool throw_if_notfound); + std::pair getSegmentByStartKey( + const RowKeyValueRef & start_key, + bool create_if_empty, + bool throw_if_notfound); void createFirstSegment(DM::DMContext & dm_context, PageStorageRunMode page_storage_run_mode); Context & global_context; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp index 1baaed6b13d..1550ea57c24 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp @@ -74,8 +74,7 @@ void DeltaMergeStore::preIngestFile(const String & parent_path, const PageIdU64 return; auto delegator = path_pool->getStableDiskDelegator(); - if (auto remote_data_store = global_context.getSharedContextDisagg()->remote_data_store; - !remote_data_store) + if (auto remote_data_store = global_context.getSharedContextDisagg()->remote_data_store; !remote_data_store) { delegator.addDTFile(file_id, file_size, parent_path); } @@ -91,8 +90,7 @@ void DeltaMergeStore::removePreIngestFile(PageIdU64 file_id, bool throw_on_not_e return; auto delegator = path_pool->getStableDiskDelegator(); - if (auto remote_data_store = global_context.getSharedContextDisagg()->remote_data_store; - !remote_data_store) + if (auto remote_data_store = global_context.getSharedContextDisagg()->remote_data_store; !remote_data_store) { delegator.removeDTFile(file_id, throw_on_not_exist); } @@ -103,9 +101,10 @@ void DeltaMergeStore::removePreIngestFile(PageIdU64 file_id, bool throw_on_not_e } -void DeltaMergeStore::cleanPreIngestFiles(const Context & db_context, - const DB::Settings & db_settings, - const std::vector & external_files) +void DeltaMergeStore::cleanPreIngestFiles( + const Context & db_context, + const DB::Settings & db_settings, + const std::vector & external_files) { auto dm_context = newDMContext(db_context, db_settings); auto delegate = dm_context->path_pool->getStableDiskDelegator(); @@ -145,7 +144,8 @@ Segments DeltaMergeStore::ingestDTFilesUsingColumnFile( // Keep trying until succeeded. while (true) { - auto [segment, is_empty] = getSegmentByStartKey(cur_range.getStart(), /*create_if_empty*/ true, /*throw_if_notfound*/ true); + auto [segment, is_empty] + = getSegmentByStartKey(cur_range.getStart(), /*create_if_empty*/ true, /*throw_if_notfound*/ true); FAIL_POINT_PAUSE(FailPoints::pause_when_ingesting_to_dt_store); waitForWrite(dm_context, segment); @@ -169,7 +169,8 @@ Segments DeltaMergeStore::ingestDTFilesUsingColumnFile( const auto & file_parent_path = file->parentPath(); auto page_id = storage_pool->newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); - auto ref_file = DMFile::restore(file_provider, file_id, page_id, file_parent_path, DMFile::ReadMetaMode::all()); + auto ref_file + = DMFile::restore(file_provider, file_id, page_id, file_parent_path, DMFile::ReadMetaMode::all()); data_files.emplace_back(std::move(ref_file)); wbs.data.putRefPage(page_id, file->pageId()); } @@ -178,7 +179,8 @@ Segments DeltaMergeStore::ingestDTFilesUsingColumnFile( // they are visible for readers immediately, who require file_ids to be found in PageStorage. wbs.writeLogAndData(); - bool ingest_success = segment->ingestDataToDelta(*dm_context, range.shrink(segment_range), data_files, clear_data_in_range); + bool ingest_success + = segment->ingestDataToDelta(*dm_context, range.shrink(segment_range), data_files, clear_data_in_range); fiu_do_on(FailPoints::force_set_segment_ingest_packs_fail, { ingest_success = false; }); if (ingest_success) { @@ -221,15 +223,9 @@ Segments DeltaMergeStore::ingestDTFilesUsingSplit( bool clear_data_in_range) { { - RUNTIME_CHECK( - files.size() == external_files.size(), - files.size(), - external_files.size()); + RUNTIME_CHECK(files.size() == external_files.size(), files.size(), external_files.size()); for (size_t i = 0; i < files.size(); ++i) - RUNTIME_CHECK( - files[i]->pageId() == external_files[i].id, - files[i]->pageId(), - external_files[i].toString()); + RUNTIME_CHECK(files[i]->pageId() == external_files[i].id, files[i]->pageId(), external_files[i].toString()); } std::set updated_segments; @@ -246,17 +242,22 @@ Segments DeltaMergeStore::ingestDTFilesUsingSplit( while (!remaining_delete_range.none()) { - auto [segment, is_empty] = getSegmentByStartKey(remaining_delete_range.getStart(), /*create_if_empty*/ true, /*throw_if_notfound*/ true); + auto [segment, is_empty] = getSegmentByStartKey( + remaining_delete_range.getStart(), + /*create_if_empty*/ true, + /*throw_if_notfound*/ true); const auto delete_range = remaining_delete_range.shrink(segment->getRowKeyRange()); RUNTIME_CHECK( - !delete_range.none(), // as remaining_delete_range is not none, we expect the shrinked range to be not none. + !delete_range + .none(), // as remaining_delete_range is not none, we expect the shrinked range to be not none. delete_range.toDebugString(), segment->simpleInfo(), remaining_delete_range.toDebugString()); LOG_DEBUG( log, - "Table ingest using split - delete range phase - Try to delete range in segment, delete_range={} segment={} remaining_delete_range={} updated_segments_n={}", + "Table ingest using split - delete range phase - Try to delete range in segment, delete_range={} " + "segment={} remaining_delete_range={} updated_segments_n={}", delete_range.toDebugString(), segment->simpleInfo(), remaining_delete_range.toDebugString(), @@ -321,7 +322,8 @@ Segments DeltaMergeStore::ingestDTFilesUsingSplit( { LOG_WARNING( log, - "Table ingest using split - split ingest phase - Unexpected empty DMFile, skipped. ingest_range={} file_idx={} file={}", + "Table ingest using split - split ingest phase - Unexpected empty DMFile, skipped. ingest_range={} " + "file_idx={} file={}", ingest_range.toDebugString(), file_idx, files[file_idx]->path()); @@ -339,7 +341,10 @@ Segments DeltaMergeStore::ingestDTFilesUsingSplit( auto file_ingest_range = external_files[file_idx].range.shrink(ingest_range); while (!file_ingest_range.none()) // This DMFile has remaining data to ingest { - auto [segment, is_empty] = getSegmentByStartKey(file_ingest_range.getStart(), /*create_if_empty*/ true, /*throw_if_notfound*/ true); + auto [segment, is_empty] = getSegmentByStartKey( + file_ingest_range.getStart(), + /*create_if_empty*/ true, + /*throw_if_notfound*/ true); if (segment->hasAbandoned()) continue; // retry with current range and file @@ -361,14 +366,20 @@ Segments DeltaMergeStore::ingestDTFilesUsingSplit( LOG_INFO( log, - "Table ingest using split - split ingest phase - Try to ingest file into segment, file_idx={} file_id=dmf_{} file_ingest_range={} segment={} segment_ingest_range={}", + "Table ingest using split - split ingest phase - Try to ingest file into segment, file_idx={} " + "file_id=dmf_{} file_ingest_range={} segment={} segment_ingest_range={}", file_idx, files[file_idx]->fileId(), file_ingest_range.toDebugString(), segment->simpleInfo(), segment_ingest_range.toDebugString()); - const bool succeeded = ingestDTFileIntoSegmentUsingSplit(*dm_context, segment, segment_ingest_range, files[file_idx], clear_data_in_range); + const bool succeeded = ingestDTFileIntoSegmentUsingSplit( + *dm_context, + segment, + segment_ingest_range, + files[file_idx], + clear_data_in_range); if (succeeded) { updated_segments.insert(segment); @@ -389,9 +400,7 @@ Segments DeltaMergeStore::ingestDTFilesUsingSplit( "Table ingest using split - split ingest phase - finished, updated_segments_n={}", updated_segments.size()); - return std::vector( - updated_segments.begin(), - updated_segments.end()); + return std::vector(updated_segments.begin(), updated_segments.end()); } /** @@ -407,9 +416,7 @@ bool DeltaMergeStore::ingestDTFileIntoSegmentUsingSplit( const auto & segment_range = segment->getRowKeyRange(); // The ingest_range must fall in segment's range. - RUNTIME_CHECK( - !ingest_range.none(), - ingest_range.toDebugString()); + RUNTIME_CHECK(!ingest_range.none(), ingest_range.toDebugString()); RUNTIME_CHECK( compare(segment_range.getStart(), ingest_range.getStart()) <= 0, segment_range.toDebugString(), @@ -479,7 +486,12 @@ bool DeltaMergeStore::ingestDTFileIntoSegmentUsingSplit( * │--------------- Segment ------│--------│ * │-------- Ingest Range --------│ */ - const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.end, SegmentSplitMode::Logical); + const auto [left, right] = segmentSplit( + dm_context, + segment, + SegmentSplitReason::ForIngest, + ingest_range.end, + SegmentSplitMode::Logical); if (left == nullptr || right == nullptr) { // Split failed, likely caused by snapshot failed. @@ -501,7 +513,12 @@ bool DeltaMergeStore::ingestDTFileIntoSegmentUsingSplit( * │--------│------ Segment ---------------│ * │-------- Ingest Range --------│ */ - const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.start, SegmentSplitMode::Logical); + const auto [left, right] = segmentSplit( + dm_context, + segment, + SegmentSplitReason::ForIngest, + ingest_range.start, + SegmentSplitMode::Logical); if (left == nullptr || right == nullptr) { // Split failed, likely caused by snapshot failed. @@ -521,7 +538,12 @@ bool DeltaMergeStore::ingestDTFileIntoSegmentUsingSplit( * │---│----------- Segment ---------------│ * │-------- Ingest Range --------│ */ - const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.start, SegmentSplitMode::Logical); + const auto [left, right] = segmentSplit( + dm_context, + segment, + SegmentSplitReason::ForIngest, + ingest_range.start, + SegmentSplitMode::Logical); if (left == nullptr || right == nullptr) { // Split failed, likely caused by snapshot failed. @@ -562,9 +584,7 @@ UInt64 DeltaMergeStore::ingestFiles( // We require A <= B. for (const auto & ext_file : external_files) { - RUNTIME_CHECK( - !ext_file.range.none(), - ext_file.toString()); + RUNTIME_CHECK(!ext_file.range.none(), ext_file.toString()); RUNTIME_CHECK( compare(last_end.toRowKeyValueRef(), ext_file.range.getStart()) <= 0, last_end.toDebugString(), @@ -578,7 +598,8 @@ UInt64 DeltaMergeStore::ingestFiles( for (const auto & ext_file : external_files) { RUNTIME_CHECK_MSG( - compare(range.getStart(), ext_file.range.getStart()) <= 0 && compare(range.getEnd(), ext_file.range.getEnd()) >= 0, + compare(range.getStart(), ext_file.range.getStart()) <= 0 + && compare(range.getEnd(), ext_file.range.getEnd()) >= 0, "Detected illegal region boundary: range={} file_range={} keyspace={} table_id={}. " "TiFlash will exit to prevent data inconsistency. " "If you accept data inconsistency and want to continue the service, " @@ -624,8 +645,13 @@ UInt64 DeltaMergeStore::ingestFiles( } else { - Remote::DMFileOID oid{.store_id = store_id, .keyspace_id = dm_context->keyspace_id, .table_id = dm_context->physical_table_id, .file_id = external_file.id}; - file = remote_data_store->prepareDMFile(oid, external_file.id)->restore(DMFile::ReadMetaMode::memoryAndDiskSize()); + Remote::DMFileOID oid{ + .store_id = store_id, + .keyspace_id = dm_context->keyspace_id, + .table_id = dm_context->physical_table_id, + .file_id = external_file.id}; + file = remote_data_store->prepareDMFile(oid, external_file.id) + ->restore(DMFile::ReadMetaMode::memoryAndDiskSize()); } rows += file->getRows(); bytes += file->getBytes(); @@ -662,7 +688,8 @@ UInt64 DeltaMergeStore::ingestFiles( }; LOG_INFO( log, - "Table ingest files - begin, use_split_replace={} files={} rows={} bytes={} bytes_on_disk={} range={} clear={}", + "Table ingest files - begin, use_split_replace={} files={} rows={} bytes={} bytes_on_disk={} range={} " + "clear={}", use_split_replace, get_ingest_files(), rows, @@ -688,7 +715,11 @@ UInt64 DeltaMergeStore::ingestFiles( } else { - Remote::DMFileOID oid{.store_id = store_id, .keyspace_id = dm_context->keyspace_id, .table_id = dm_context->physical_table_id, .file_id = file->fileId()}; + Remote::DMFileOID oid{ + .store_id = store_id, + .keyspace_id = dm_context->keyspace_id, + .table_id = dm_context->physical_table_id, + .file_id = file->fileId()}; PS::V3::CheckpointLocation loc{ .data_file_id = std::make_shared(S3::S3Filename::fromDMFileOID(oid).toFullKey()), .offset_in_file = 0, @@ -789,17 +820,22 @@ std::vector DeltaMergeStore::ingestSegmentsUsingSplit( while (!remaining_delete_range.none()) { - auto [segment, is_empty] = getSegmentByStartKey(remaining_delete_range.getStart(), /*create_if_empty*/ true, /*throw_if_notfound*/ true); + auto [segment, is_empty] = getSegmentByStartKey( + remaining_delete_range.getStart(), + /*create_if_empty*/ true, + /*throw_if_notfound*/ true); const auto delete_range = remaining_delete_range.shrink(segment->getRowKeyRange()); RUNTIME_CHECK( - !delete_range.none(), // as remaining_delete_range is not none, we expect the shrinked range to be not none. + !delete_range + .none(), // as remaining_delete_range is not none, we expect the shrinked range to be not none. delete_range.toDebugString(), segment->simpleInfo(), remaining_delete_range.toDebugString()); LOG_DEBUG( log, - "Table ingest checkpoint using split - delete range phase - Try to delete range in segment, delete_range={} segment={} remaining_delete_range={} updated_segments_n={}", + "Table ingest checkpoint using split - delete range phase - Try to delete range in segment, " + "delete_range={} segment={} remaining_delete_range={} updated_segments_n={}", delete_range.toDebugString(), segment->simpleInfo(), remaining_delete_range.toDebugString(), @@ -863,7 +899,8 @@ std::vector DeltaMergeStore::ingestSegmentsUsingSplit( { LOG_INFO( log, - "Table ingest checkpoint using split - split ingest phase - Meet empty Segment, skipped. ingest_range={} segment_idx={}", + "Table ingest checkpoint using split - split ingest phase - Meet empty Segment, skipped. " + "ingest_range={} segment_idx={}", ingest_range.toDebugString(), segment_idx); continue; @@ -880,7 +917,10 @@ std::vector DeltaMergeStore::ingestSegmentsUsingSplit( auto file_ingest_range = target_segments[segment_idx]->getRowKeyRange(); while (!file_ingest_range.none()) // This DMFile has remaining data to ingest { - auto [segment, is_empty] = getSegmentByStartKey(file_ingest_range.getStart(), /*create_if_empty*/ true, /*throw_if_notfound*/ true); + auto [segment, is_empty] = getSegmentByStartKey( + file_ingest_range.getStart(), + /*create_if_empty*/ true, + /*throw_if_notfound*/ true); if (segment->hasAbandoned()) continue; // retry with current range and file @@ -897,14 +937,19 @@ std::vector DeltaMergeStore::ingestSegmentsUsingSplit( LOG_INFO( log, - "Table ingest checkpoint using split - split ingest phase - Try to ingest file into segment, segment_idx={} segment_id={} segment_ingest_range={} segment={} segment_ingest_range={}", + "Table ingest checkpoint using split - split ingest phase - Try to ingest file into segment, " + "segment_idx={} segment_id={} segment_ingest_range={} segment={} segment_ingest_range={}", segment_idx, target_segments[segment_idx]->segmentId(), file_ingest_range.toDebugString(), segment->simpleInfo(), segment_ingest_range.toDebugString()); - const bool succeeded = ingestSegmentDataIntoSegmentUsingSplit(*dm_context, segment, segment_ingest_range, target_segments[segment_idx]); + const bool succeeded = ingestSegmentDataIntoSegmentUsingSplit( + *dm_context, + segment, + segment_ingest_range, + target_segments[segment_idx]); if (succeeded) { updated_segments.insert(segment); @@ -925,9 +970,7 @@ std::vector DeltaMergeStore::ingestSegmentsUsingSplit( "Table ingest checkpoint using split - split ingest phase - finished, updated_segments_n={}", updated_segments.size()); - return std::vector( - updated_segments.begin(), - updated_segments.end()); + return std::vector(updated_segments.begin(), updated_segments.end()); } bool DeltaMergeStore::ingestSegmentDataIntoSegmentUsingSplit( @@ -939,9 +982,7 @@ bool DeltaMergeStore::ingestSegmentDataIntoSegmentUsingSplit( const auto & segment_range = segment->getRowKeyRange(); // The ingest_range must fall in segment's range. - RUNTIME_CHECK( - !ingest_range.none(), - ingest_range.toDebugString()); + RUNTIME_CHECK(!ingest_range.none(), ingest_range.toDebugString()); RUNTIME_CHECK( compare(segment_range.getStart(), ingest_range.getStart()) <= 0, segment_range.toDebugString(), @@ -974,7 +1015,8 @@ bool DeltaMergeStore::ingestSegmentDataIntoSegmentUsingSplit( wbs.writeLogAndData(); RUNTIME_CHECK(in_memory_files.empty()); RUNTIME_CHECK(dm_files.size() == 1); - const auto new_segment_or_null = segmentDangerouslyReplaceDataFromCheckpoint(dm_context, segment, dm_files[0], column_file_persisteds); + const auto new_segment_or_null + = segmentDangerouslyReplaceDataFromCheckpoint(dm_context, segment, dm_files[0], column_file_persisteds); const bool succeeded = new_segment_or_null != nullptr; if (!succeeded) { @@ -993,7 +1035,12 @@ bool DeltaMergeStore::ingestSegmentDataIntoSegmentUsingSplit( * │--------------- Segment ------│--------│ * │-------- Ingest Range --------│ */ - const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.end, SegmentSplitMode::Logical); + const auto [left, right] = segmentSplit( + dm_context, + segment, + SegmentSplitReason::ForIngest, + ingest_range.end, + SegmentSplitMode::Logical); if (left == nullptr || right == nullptr) { // Split failed, likely caused by snapshot failed. @@ -1015,7 +1062,12 @@ bool DeltaMergeStore::ingestSegmentDataIntoSegmentUsingSplit( * │--------│------ Segment ---------------│ * │-------- Ingest Range --------│ */ - const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.start, SegmentSplitMode::Logical); + const auto [left, right] = segmentSplit( + dm_context, + segment, + SegmentSplitReason::ForIngest, + ingest_range.start, + SegmentSplitMode::Logical); if (left == nullptr || right == nullptr) { std::this_thread::sleep_for(std::chrono::milliseconds(15)); @@ -1033,7 +1085,12 @@ bool DeltaMergeStore::ingestSegmentDataIntoSegmentUsingSplit( * │---│----------- Segment ---------------│ * │-------- Ingest Range --------│ */ - const auto [left, right] = segmentSplit(dm_context, segment, SegmentSplitReason::ForIngest, ingest_range.start, SegmentSplitMode::Logical); + const auto [left, right] = segmentSplit( + dm_context, + segment, + SegmentSplitReason::ForIngest, + ingest_range.start, + SegmentSplitMode::Logical); if (left == nullptr || right == nullptr) { std::this_thread::sleep_for(std::chrono::milliseconds(15)); @@ -1056,11 +1113,19 @@ void DeltaMergeStore::ingestSegmentsFromCheckpointInfo( if (unlikely(range.none())) { - LOG_INFO(log, "Ingest checkpoint from remote meet empty range, ignore, store_id={} region_id={}", checkpoint_info->remote_store_id, checkpoint_info->region_id); + LOG_INFO( + log, + "Ingest checkpoint from remote meet empty range, ignore, store_id={} region_id={}", + checkpoint_info->remote_store_id, + checkpoint_info->region_id); return; } - LOG_INFO(log, "Ingest checkpoint from remote, store_id={} region_id={}", checkpoint_info->remote_store_id, checkpoint_info->region_id); + LOG_INFO( + log, + "Ingest checkpoint from remote, store_id={} region_id={}", + checkpoint_info->remote_store_id, + checkpoint_info->region_id); auto segment_meta_infos = Segment::readAllSegmentsMetaInfoInRange(*dm_context, range, checkpoint_info); LOG_INFO(log, "Ingest checkpoint segments num {}", segment_meta_infos.size()); WriteBatches wbs{*dm_context->storage_pool}; @@ -1081,7 +1146,12 @@ void DeltaMergeStore::ingestSegmentsFromCheckpointInfo( wbs.writeLogAndData(); auto updated_segments = ingestSegmentsUsingSplit(dm_context, range, restored_segments); - LOG_INFO(log, "Ingest checkpoint from remote done, store_id={} region_id={} n_segments={}", checkpoint_info->remote_store_id, checkpoint_info->region_id, restored_segments.size()); + LOG_INFO( + log, + "Ingest checkpoint from remote done, store_id={} region_id={} n_segments={}", + checkpoint_info->remote_store_id, + checkpoint_info->region_id, + restored_segments.size()); for (auto & segment : restored_segments) { diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index dd420d3bbc7..8b6f4cdbb49 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -108,7 +108,9 @@ class LocalDMFileGcRemover final , logger(std::move(log)) {} - void operator()(const ExternalPageCallbacks::PathAndIdsVec & path_and_ids_vec, const std::set & valid_ids) + void operator()( + const ExternalPageCallbacks::PathAndIdsVec & path_and_ids_vec, + const std::set & valid_ids) { // If the StoragePathPool is invalid or shutdown flag is set, // meaning we call `remover` after shutdowning or dropping the table, @@ -140,9 +142,10 @@ class LocalDMFileGcRemover final { // just ignore } - LOG_INFO(logger, - "GC try remove useless DM file, but file not found and may have been removed, dmfile={}", - DMFile::getPathByStatus(path, id, DMFile::Status::READABLE)); + LOG_INFO( + logger, + "GC try remove useless DM file, but file not found and may have been removed, dmfile={}", + DMFile::getPathByStatus(path, id, DMFile::Status::READABLE)); continue; // next file } else if (dmfile->canGC()) @@ -167,7 +170,11 @@ class LocalDMFileGcRemover final if (err_msg.empty()) LOG_INFO(logger, "GC removed useless DM file, dmfile={}", dmfile->path()); else - LOG_INFO(logger, "GC try remove useless DM file, but error happen, dmfile={} err_msg={}", dmfile->path(), err_msg); + LOG_INFO( + logger, + "GC try remove useless DM file, but error happen, dmfile={} err_msg={}", + dmfile->path(), + err_msg); continue; // next file } } @@ -227,7 +234,9 @@ class S3DMFileGcRemover final , logger(std::move(log)) {} - void operator()(const ExternalPageCallbacks::PathAndIdsVec & path_and_ids_vec, const std::set & valid_ids) + void operator()( + const ExternalPageCallbacks::PathAndIdsVec & path_and_ids_vec, + const std::set & valid_ids) { // If the StoragePathPool is invalid or shutdown flag is set, // meaning we call `remover` after shutdowning or dropping the table, @@ -260,8 +269,10 @@ void DeltaMergeStore::setUpBackgroundTask(const DMContextPtr & dm_context) callbacks.prefix = storage_pool->getNamespaceID(); if (auto data_store = dm_context->db_context.getSharedContextDisagg()->remote_data_store; !data_store) { - callbacks.scanner = LocalDMFileGcScanner(std::weak_ptr(path_pool), global_context.getFileProvider()); - callbacks.remover = LocalDMFileGcRemover(std::weak_ptr(path_pool), global_context.getFileProvider(), log); + callbacks.scanner + = LocalDMFileGcScanner(std::weak_ptr(path_pool), global_context.getFileProvider()); + callbacks.remover + = LocalDMFileGcRemover(std::weak_ptr(path_pool), global_context.getFileProvider(), log); } else { @@ -292,7 +303,9 @@ void DeltaMergeStore::setUpBackgroundTask(const DMContextPtr & dm_context) blockable_background_pool_handle->wake(); } -std::vector DeltaMergeStore::getMergeableSegments(const DMContextPtr & context, const SegmentPtr & baseSegment) +std::vector DeltaMergeStore::getMergeableSegments( + const DMContextPtr & context, + const SegmentPtr & baseSegment) { // Last segment cannot be merged. if (baseSegment->getRowKeyRange().isEndInfinite()) @@ -464,7 +477,11 @@ static std::string toString(MergeDeltaReason type) // Returns true if it needs gc. // This is for optimization purpose, does not mean to be accurate. -bool shouldCompactStableWithTooManyInvalidVersion(const SegmentPtr & seg, DB::Timestamp gc_safepoint, double ratio_threshold, const LoggerPtr & log) +bool shouldCompactStableWithTooManyInvalidVersion( + const SegmentPtr & seg, + DB::Timestamp gc_safepoint, + double ratio_threshold, + const LoggerPtr & log) { // Always GC. if (ratio_threshold < 1.0) @@ -484,7 +501,13 @@ bool shouldCompactStableWithTooManyInvalidVersion(const SegmentPtr & seg, DB::Ti return false; } -bool shouldCompactDeltaWithStable(const DMContext & context, const SegmentPtr & segment, const SegmentSnapshotPtr & snap, const RowKeyRange & segment_range, double invalid_data_ratio_threshold, const LoggerPtr & log) +bool shouldCompactDeltaWithStable( + const DMContext & context, + const SegmentPtr & segment, + const SegmentSnapshotPtr & snap, + const RowKeyRange & segment_range, + double invalid_data_ratio_threshold, + const LoggerPtr & log) { auto actual_delete_range = snap->delta->getSquashDeleteRange().shrink(segment_range); if (actual_delete_range.none()) @@ -502,7 +525,8 @@ bool shouldCompactDeltaWithStable(const DMContext & context, const SegmentPtr & // because before write apply snapshot file, it will write a delete range first, and will meet the following gc criteria. // But the cost should be really minor because merge delta on an empty segment should be very fast. // What's more, we can ignore this kind of delete range in future to avoid this extra gc. - auto check_result = (delete_rows >= stable_rows * invalid_data_ratio_threshold) || (delete_bytes >= stable_bytes * invalid_data_ratio_threshold); + auto check_result = (delete_rows >= stable_rows * invalid_data_ratio_threshold) + || (delete_bytes >= stable_bytes * invalid_data_ratio_threshold); LOG_TRACE( log, @@ -536,13 +560,14 @@ std::unordered_set getDMFileIDs(const SegmentPtr & seg) return file_ids; } -bool shouldCompactStableWithTooMuchDataOutOfSegmentRange(const DMContext & context, // - const SegmentPtr & seg, - const SegmentSnapshotPtr & snap, - const SegmentPtr & prev_seg, - const SegmentPtr & next_seg, - double invalid_data_ratio_threshold, - const LoggerPtr & log) +bool shouldCompactStableWithTooMuchDataOutOfSegmentRange( + const DMContext & context, // + const SegmentPtr & seg, + const SegmentSnapshotPtr & snap, + const SegmentPtr & prev_seg, + const SegmentPtr & next_seg, + double invalid_data_ratio_threshold, + const LoggerPtr & log) { if (snap->stable->getDMFilesPacks() == 0) { @@ -558,9 +583,11 @@ bool shouldCompactStableWithTooMuchDataOutOfSegmentRange(const DMContext & conte if (at_least_result.first_pack_intersection == RSResult::All // && at_least_result.last_pack_intersection == RSResult::All) { - LOG_TRACE(log, "GC - shouldCompactStableWithTooMuchDataOutOfSegmentRange permanently skipped segment " - "because all packs in DTFiles are fully contained by the segment range, segment={}", - seg->info()); + LOG_TRACE( + log, + "GC - shouldCompactStableWithTooMuchDataOutOfSegmentRange permanently skipped segment " + "because all packs in DTFiles are fully contained by the segment range, segment={}", + seg->info()); seg->setValidDataRatioChecked(); return false; } @@ -603,9 +630,7 @@ bool shouldCompactStableWithTooMuchDataOutOfSegmentRange(const DMContext & conte fmt_buf.joinStr( dt_files.begin(), dt_files.end(), - [](const DMFilePtr & dt_file, FmtBuffer & fb) { - fb.fmtAppend("{}", dt_file->fileId()); - }, + [](const DMFilePtr & dt_file, FmtBuffer & fb) { fb.fmtAppend("{}", dt_file->fileId()); }, ","); return fmt_buf.toString(); }(), @@ -646,7 +671,8 @@ bool shouldCompactStableWithTooMuchDataOutOfSegmentRange(const DMContext & conte LOG_TRACE( log, "GC - Checking shouldCompactStableWithTooMuchDataOutOfSegmentRange, " - "check_result={} first_pack_inc={} last_pack_inc={} rows_at_least={} bytes_at_least={} file_rows={} file_bytes={} segment={} ", + "check_result={} first_pack_inc={} last_pack_inc={} rows_at_least={} bytes_at_least={} file_rows={} " + "file_bytes={} segment={} ", check_result, magic_enum::enum_name(at_least_result.first_pack_intersection), magic_enum::enum_name(at_least_result.last_pack_intersection), @@ -686,11 +712,7 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMerge(const DMContextPtr & dm_context, c return {}; } - LOG_INFO( - log, - "GC - Trigger Merge, segment={} table={}", - segment->simpleInfo(), - table_name); + LOG_INFO(log, "GC - Trigger Merge, segment={} table={}", segment->simpleInfo(), table_name); auto new_segment = segmentMerge(*dm_context, segments_to_merge, SegmentMergeReason::BackgroundGCThread); if (new_segment) { @@ -700,7 +722,12 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMerge(const DMContextPtr & dm_context, c return new_segment; } -SegmentPtr DeltaMergeStore::gcTrySegmentMergeDelta(const DMContextPtr & dm_context, const SegmentPtr & segment, const SegmentPtr & prev_segment, const SegmentPtr & next_segment, DB::Timestamp gc_safe_point) +SegmentPtr DeltaMergeStore::gcTrySegmentMergeDelta( + const DMContextPtr & dm_context, + const SegmentPtr & segment, + const SegmentPtr & prev_segment, + const SegmentPtr & next_segment, + DB::Timestamp gc_safe_point) { SegmentSnapshotPtr segment_snap; { @@ -709,11 +736,16 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMergeDelta(const DMContextPtr & dm_conte // The segment we just retrieved may be dropped from the map. Let's verify it again before creating a snapshot. if (!isSegmentValid(lock, segment)) { - LOG_TRACE(log, "GC - Skip checking MergeDelta because not valid, segment={} table={}", segment->simpleInfo(), table_name); + LOG_TRACE( + log, + "GC - Skip checking MergeDelta because not valid, segment={} table={}", + segment->simpleInfo(), + table_name); return {}; } - segment_snap = segment->createSnapshot(*dm_context, /* for_update */ true, CurrentMetrics::DT_SnapshotOfDeltaMerge); + segment_snap + = segment->createSnapshot(*dm_context, /* for_update */ true, CurrentMetrics::DT_SnapshotOfDeltaMerge); if (!segment_snap) { LOG_TRACE( @@ -789,11 +821,7 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMergeDelta(const DMContextPtr & dm_conte if (!should_compact) { - LOG_TRACE( - log, - "GC - MergeDelta skipped, segment={} table={}", - segment->simpleInfo(), - table_name); + LOG_TRACE(log, "GC - MergeDelta skipped, segment={} table={}", segment->simpleInfo(), table_name); return {}; } @@ -840,13 +868,14 @@ UInt64 DeltaMergeStore::onSyncGc(Int64 limit, const GCOptions & gc_options) } DB::Timestamp gc_safe_point = latest_gc_safe_point.load(std::memory_order_acquire); - LOG_TRACE(log, - "GC on table start, table={} check_key={} options={} gc_safe_point={} max_gc_limit={}", - table_name, - next_gc_check_key.toDebugString(), - gc_options.toString(), - gc_safe_point, - limit); + LOG_TRACE( + log, + "GC on table start, table={} check_key={} options={} gc_safe_point={} max_gc_limit={}", + table_name, + next_gc_check_key.toDebugString(), + gc_options.toString(), + gc_safe_point, + limit); UInt64 check_segments_num = 0; Int64 gc_segments_num = 0; @@ -901,11 +930,7 @@ UInt64 DeltaMergeStore::onSyncGc(Int64 limit, const GCOptions & gc_options) if (!new_seg) { - LOG_TRACE( - log, - "GC - Skipped segment, segment={} table={}", - segment->simpleInfo(), - table_name); + LOG_TRACE(log, "GC - Skipped segment, segment={} table={}", segment->simpleInfo(), table_name); continue; } diff --git a/dbms/src/Storages/DeltaMerge/RowKeyRange.h b/dbms/src/Storages/DeltaMerge/RowKeyRange.h index 3bb74bb3dca..5170725e118 100644 --- a/dbms/src/Storages/DeltaMerge/RowKeyRange.h +++ b/dbms/src/Storages/DeltaMerge/RowKeyRange.h @@ -38,8 +38,7 @@ inline int compare(const char * a, size_t a_size, const char * b, size_t b_size) int res = memcmp(a, b, std::min(a_size, b_size)); if (res != 0) return res; - return a_size == b_size ? 0 : a_size > b_size ? 1 - : -1; + return a_size == b_size ? 0 : a_size > b_size ? 1 : -1; } struct RowKeyValue; @@ -264,7 +263,8 @@ using RowKeyValues = std::vector; inline int compare(const RowKeyValueRef & a, const RowKeyValueRef & b) { if (unlikely(a.is_common_handle != b.is_common_handle)) - throw Exception("Should not reach here, common handle rowkey value compare with non common handle rowkey value"); + throw Exception( + "Should not reach here, common handle rowkey value compare with non common handle rowkey value"); if (a.is_common_handle) { return compare(a.data, a.size, b.data, b.size); @@ -274,7 +274,8 @@ inline int compare(const RowKeyValueRef & a, const RowKeyValueRef & b) /// in case of non common handle, we can compare the int value directly in most cases if (a.int_value != b.int_value) return a.int_value > b.int_value ? 1 : -1; - if (likely(a.int_value != RowKeyValue::INT_HANDLE_MAX_KEY.int_value || (a.data == nullptr && b.data == nullptr))) + if (likely( + a.int_value != RowKeyValue::INT_HANDLE_MAX_KEY.int_value || (a.data == nullptr && b.data == nullptr))) return 0; /// if a.int_value == b.int_value == Int64::max_value, we need to further check the data field because even if @@ -282,10 +283,18 @@ inline int compare(const RowKeyValueRef & a, const RowKeyValueRef & b) bool a_inf = false; bool b_inf = false; if (a.data != nullptr) - a_inf = compare(a.data, a.size, RowKeyValue::INT_HANDLE_MAX_KEY.value->data(), RowKeyValue::INT_HANDLE_MAX_KEY.value->size()) + a_inf = compare( + a.data, + a.size, + RowKeyValue::INT_HANDLE_MAX_KEY.value->data(), + RowKeyValue::INT_HANDLE_MAX_KEY.value->size()) == 0; if (b.data != nullptr) - b_inf = compare(b.data, b.size, RowKeyValue::INT_HANDLE_MAX_KEY.value->data(), RowKeyValue::INT_HANDLE_MAX_KEY.value->size()) + b_inf = compare( + b.data, + b.size, + RowKeyValue::INT_HANDLE_MAX_KEY.value->data(), + RowKeyValue::INT_HANDLE_MAX_KEY.value->size()) == 0; if (a_inf != b_inf) { @@ -413,7 +422,11 @@ struct RowKeyColumnContainer if (is_common_handle) { size_t prev_offset = index == 0 ? 0 : (*string_offsets)[index - 1]; - return RowKeyValueRef{is_common_handle, reinterpret_cast(&(*string_data)[prev_offset]), (*string_offsets)[index] - prev_offset - 1, 0}; + return RowKeyValueRef{ + is_common_handle, + reinterpret_cast(&(*string_data)[prev_offset]), + (*string_offsets)[index] - prev_offset - 1, + 0}; } else { @@ -489,7 +502,11 @@ struct RowKeyRange static std::shared_mutex table_mutex; static const TableRangeMinMax & getTableMinMaxData(KeyspaceID keyspace_id, TableID table_id, bool is_common_handle); - RowKeyRange(const RowKeyValue & start_, const RowKeyValue & end_, bool is_common_handle_, size_t rowkey_column_size_) + RowKeyRange( + const RowKeyValue & start_, + const RowKeyValue & end_, + bool is_common_handle_, + size_t rowkey_column_size_) : is_common_handle(is_common_handle_) , start(start_) , end(end_) @@ -515,11 +532,19 @@ struct RowKeyRange { if (is_common_handle) { - return RowKeyRange(start_value.toRowKeyValue(), RowKeyValue::COMMON_HANDLE_MAX_KEY, is_common_handle, rowkey_column_size); + return RowKeyRange( + start_value.toRowKeyValue(), + RowKeyValue::COMMON_HANDLE_MAX_KEY, + is_common_handle, + rowkey_column_size); } else { - return RowKeyRange(start_value.toRowKeyValue(), RowKeyValue::INT_HANDLE_MAX_KEY, is_common_handle, rowkey_column_size); + return RowKeyRange( + start_value.toRowKeyValue(), + RowKeyValue::INT_HANDLE_MAX_KEY, + is_common_handle, + rowkey_column_size); } } @@ -527,11 +552,19 @@ struct RowKeyRange { if (is_common_handle) { - return RowKeyRange(RowKeyValue::COMMON_HANDLE_MIN_KEY, end_value.toRowKeyValue(), is_common_handle, rowkey_column_size); + return RowKeyRange( + RowKeyValue::COMMON_HANDLE_MIN_KEY, + end_value.toRowKeyValue(), + is_common_handle, + rowkey_column_size); } else { - return RowKeyRange(RowKeyValue::INT_HANDLE_MIN_KEY, end_value.toRowKeyValue(), is_common_handle, rowkey_column_size); + return RowKeyRange( + RowKeyValue::INT_HANDLE_MIN_KEY, + end_value.toRowKeyValue(), + is_common_handle, + rowkey_column_size); } } @@ -548,7 +581,11 @@ struct RowKeyRange } else { - return RowKeyRange(RowKeyValue::INT_HANDLE_MIN_KEY, RowKeyValue::INT_HANDLE_MAX_KEY, is_common_handle, rowkey_column_size); + return RowKeyRange( + RowKeyValue::INT_HANDLE_MIN_KEY, + RowKeyValue::INT_HANDLE_MAX_KEY, + is_common_handle, + rowkey_column_size); } } @@ -565,7 +602,11 @@ struct RowKeyRange } else { - return RowKeyRange(RowKeyValue::INT_HANDLE_MAX_KEY, RowKeyValue::INT_HANDLE_MIN_KEY, is_common_handle, rowkey_column_size); + return RowKeyRange( + RowKeyValue::INT_HANDLE_MAX_KEY, + RowKeyValue::INT_HANDLE_MIN_KEY, + is_common_handle, + rowkey_column_size); } } @@ -654,9 +695,10 @@ struct RowKeyRange { if (is_common_handle) { - return (start.value->size() == 1 - && ((static_cast((*start.value)[0]) == TiDB::CodecFlagBytes) - || (static_cast((*start.value)[0]) == TiDB::CodecFlagNil))); + return ( + start.value->size() == 1 + && ((static_cast((*start.value)[0]) == TiDB::CodecFlagBytes) + || (static_cast((*start.value)[0]) == TiDB::CodecFlagNil))); } else { @@ -729,8 +771,9 @@ struct RowKeyRange std::pair getPosRange(const ColumnPtr & column, const size_t offset, const size_t limit) const { RowKeyColumnContainer rowkey_column(column, is_common_handle); - size_t start_index - = check(rowkey_column.getRowKeyValue(offset)) ? offset : lowerBound(rowkey_column, offset, offset + limit, getStart()); + size_t start_index = check(rowkey_column.getRowKeyValue(offset)) + ? offset + : lowerBound(rowkey_column, offset, offset + limit, getStart()); size_t end_index = check(rowkey_column.getRowKeyValue(offset + limit - 1)) ? offset + limit : lowerBound(rowkey_column, offset, offset + limit, getEnd()); @@ -743,7 +786,11 @@ struct RowKeyRange { if (handle_range.all()) { - return RowKeyRange(RowKeyValue::COMMON_HANDLE_MIN_KEY, RowKeyValue::COMMON_HANDLE_MAX_KEY, /*is_common_handle=*/is_common_handle, 1); + return RowKeyRange( + RowKeyValue::COMMON_HANDLE_MIN_KEY, + RowKeyValue::COMMON_HANDLE_MAX_KEY, + /*is_common_handle=*/is_common_handle, + 1); } WriteBufferFromOwnString ss; DB::EncodeUInt(static_cast(TiDB::CodecFlagInt), ss); @@ -755,17 +802,22 @@ struct RowKeyRange DB::EncodeInt64(handle_range.end, ss); String end = ss.releaseStr(); /// when handle_range.end == HandleRange::MAX, according to previous implementation, it should be +Inf - return RowKeyRange(RowKeyValue(is_common_handle, std::make_shared(start)), - handle_range.end == HandleRange::MAX ? RowKeyValue::COMMON_HANDLE_MAX_KEY - : RowKeyValue(is_common_handle, std::make_shared(end)), - /*is_common_handle=*/is_common_handle, - 1); + return RowKeyRange( + RowKeyValue(is_common_handle, std::make_shared(start)), + handle_range.end == HandleRange::MAX ? RowKeyValue::COMMON_HANDLE_MAX_KEY + : RowKeyValue(is_common_handle, std::make_shared(end)), + /*is_common_handle=*/is_common_handle, + 1); } else { if (handle_range.all()) { - return RowKeyRange(RowKeyValue::INT_HANDLE_MIN_KEY, RowKeyValue::INT_HANDLE_MAX_KEY, /*is_common_handle=*/is_common_handle, 1); + return RowKeyRange( + RowKeyValue::INT_HANDLE_MIN_KEY, + RowKeyValue::INT_HANDLE_MAX_KEY, + /*is_common_handle=*/is_common_handle, + 1); } WriteBufferFromOwnString ss; DB::EncodeInt64(handle_range.start, ss); @@ -775,26 +827,35 @@ struct RowKeyRange DB::EncodeInt64(handle_range.end, ss); String end = ss.releaseStr(); /// when handle_range.end == HandleRange::MAX, according to previous implementation, it should be +Inf - return RowKeyRange(RowKeyValue(is_common_handle, std::make_shared(start), handle_range.start), - handle_range.end == HandleRange::MAX ? RowKeyValue::INT_HANDLE_MAX_KEY - : RowKeyValue(is_common_handle, std::make_shared(end), handle_range.end), - /*is_common_handle=*/false, - 1); + return RowKeyRange( + RowKeyValue(is_common_handle, std::make_shared(start), handle_range.start), + handle_range.end == HandleRange::MAX + ? RowKeyValue::INT_HANDLE_MAX_KEY + : RowKeyValue(is_common_handle, std::make_shared(end), handle_range.end), + /*is_common_handle=*/false, + 1); } } - static RowKeyRange fromRegionRange(const std::shared_ptr & region_range, - const TableID table_id, - bool is_common_handle, - size_t rowkey_column_size) + static RowKeyRange fromRegionRange( + const std::shared_ptr & region_range, + const TableID table_id, + bool is_common_handle, + size_t rowkey_column_size) { - return fromRegionRange(region_range->rawKeys(), region_range->getMappedTableID(), table_id, is_common_handle, rowkey_column_size); + return fromRegionRange( + region_range->rawKeys(), + region_range->getMappedTableID(), + table_id, + is_common_handle, + rowkey_column_size); } - static RowKeyRange fromRegionRange(const std::pair & raw_keys, - const TableID table_id_in_raw_key, - const TableID table_id, - bool is_common_handle, - size_t rowkey_column_size) + static RowKeyRange fromRegionRange( + const std::pair & raw_keys, + const TableID table_id_in_raw_key, + const TableID table_id, + bool is_common_handle, + size_t rowkey_column_size) { if (likely(table_id_in_raw_key == table_id)) { @@ -812,8 +873,9 @@ struct RowKeyRange } else { - start_value = RowKeyValue(is_common_handle, - std::make_shared(RecordKVFormat::getRawTiDBPKView(start_key))); + start_value = RowKeyValue( + is_common_handle, + std::make_shared(RecordKVFormat::getRawTiDBPKView(start_key))); } if (end_key.compare(*table_range_min_max.max) >= 0) { @@ -823,8 +885,9 @@ struct RowKeyRange end_value = RowKeyValue::INT_HANDLE_MAX_KEY; } else - end_value = RowKeyValue(is_common_handle, - std::make_shared(RecordKVFormat::getRawTiDBPKView(end_key))); + end_value = RowKeyValue( + is_common_handle, + std::make_shared(RecordKVFormat::getRawTiDBPKView(end_key))); return RowKeyRange(start_value, end_value, is_common_handle, rowkey_column_size); } else diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp index 33ca164fd05..b13ef9a38a1 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp @@ -259,18 +259,19 @@ try { new_cols = DMTestEnv::getDefaultColumns(); ColumnDefine handle_column_define = (*new_cols)[0]; - new_store = std::make_shared(*db_context, - false, - "test", - "t_200", - NullspaceID, - 200, - true, - *new_cols, - handle_column_define, - false, - 1, - DeltaMergeStore::Settings()); + new_store = std::make_shared( + *db_context, + false, + "test", + "t_200", + NullspaceID, + 200, + true, + *new_cols, + handle_column_define, + false, + 1, + DeltaMergeStore::Settings()); auto block = DMTestEnv::prepareSimpleWriteBlock(0, 100, false); new_store->write(*db_context, db_context->getSettingsRef(), block); new_store->flushCache(*db_context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())); @@ -279,19 +280,20 @@ try sp_gc.next(); // continue the page storage gc th_gc.get(); - BlockInputStreamPtr in = new_store->read(*db_context, - db_context->getSettingsRef(), - *new_cols, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - "", - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = new_store->read( + *db_context, + db_context->getSettingsRef(), + *new_cols, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + "", + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_NROWS(in, 100); } CATCH @@ -341,22 +343,25 @@ try auto cols = DMTestEnv::getDefaultColumns(pk_type); store = reload(cols, (pk_type == DMTestEnv::PkType::CommonHandle), 1); - ASSERT_EQ(store->isCommonHandle(), pk_type == DMTestEnv::PkType::CommonHandle) << DMTestEnv::PkTypeToString(pk_type); - ASSERT_EQ(DeltaMergeStore::pkIsHandle(store->getHandle()), - (pk_type == DMTestEnv::PkType::PkIsHandleInt64 || pk_type == DMTestEnv::PkType::PkIsHandleInt32)) + ASSERT_EQ(store->isCommonHandle(), pk_type == DMTestEnv::PkType::CommonHandle) + << DMTestEnv::PkTypeToString(pk_type); + ASSERT_EQ( + DeltaMergeStore::pkIsHandle(store->getHandle()), + (pk_type == DMTestEnv::PkType::PkIsHandleInt64 || pk_type == DMTestEnv::PkType::PkIsHandleInt32)) << DMTestEnv::PkTypeToString(pk_type); const size_t nrows = 20; const auto & handle = store->getHandle(); - auto block1 = DMTestEnv::prepareSimpleWriteBlock(0, - nrows, - false, - /*tso*/ 2, - /*pk_name*/ handle.name, - handle.id, - handle.type, - store->isCommonHandle(), - store->getRowKeyColumnSize()); + auto block1 = DMTestEnv::prepareSimpleWriteBlock( + 0, + nrows, + false, + /*tso*/ 2, + /*pk_name*/ handle.name, + handle.id, + handle.type, + store->isCommonHandle(), + store->getRowKeyColumnSize()); block1 = DeltaMergeStore::addExtraColumnIfNeed(*db_context, store->getHandle(), std::move(block1)); ASSERT_EQ(block1.rows(), nrows); ASSERT_TRUE(block1.has(EXTRA_HANDLE_COLUMN_NAME)); @@ -364,15 +369,16 @@ try // Make a block that is overlapped with `block1` and it should be squashed by `PKSquashingBlockInputStream` size_t nrows_2 = 2; - auto block2 = DMTestEnv::prepareSimpleWriteBlock(nrows - 1, - nrows - 1 + nrows_2, - false, - /*tso*/ 4, - /*pk_name*/ handle.name, - handle.id, - handle.type, - store->isCommonHandle(), - store->getRowKeyColumnSize()); + auto block2 = DMTestEnv::prepareSimpleWriteBlock( + nrows - 1, + nrows - 1 + nrows_2, + false, + /*tso*/ 4, + /*pk_name*/ handle.name, + handle.id, + handle.type, + store->isCommonHandle(), + store->getRowKeyColumnSize()); block2 = DeltaMergeStore::addExtraColumnIfNeed(*db_context, store->getHandle(), std::move(block2)); ASSERT_EQ(block2.rows(), nrows_2); ASSERT_TRUE(block2.has(EXTRA_HANDLE_COLUMN_NAME)); @@ -380,7 +386,10 @@ try BlockInputStreamPtr stream = std::make_shared(BlocksList{block1, block2}); - stream = std::make_shared>(stream, EXTRA_HANDLE_COLUMN_ID, store->isCommonHandle()); + stream = std::make_shared>( + stream, + EXTRA_HANDLE_COLUMN_ID, + store->isCommonHandle()); ASSERT_INPUTSTREAM_NROWS(stream, nrows + nrows_2); LOG_INFO(log, "Test case for {} done.", DMTestEnv::PkTypeToString(pk_type)); @@ -458,19 +467,20 @@ try // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name, col_str_define.name, col_i8_define.name}), @@ -484,7 +494,8 @@ try { // test readRaw const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->readRaw(*db_context, db_context->getSettingsRef(), columns, 1, /* keep_order= */ false)[0]; + BlockInputStreamPtr in + = store->readRaw(*db_context, db_context->getSettingsRef(), columns, 1, /* keep_order= */ false)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name, col_str_define.name, col_i8_define.name}), @@ -544,22 +555,23 @@ try const auto & columns = store->getTableColumns(); auto scan_context = std::make_shared(); - auto in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024, - /* read_segments */ {}, - /* extra_table_id_index */ InvalidColumnID, - /* scan_context */ scan_context)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024, + /* read_segments */ {}, + /* extra_table_id_index */ InvalidColumnID, + /* scan_context */ scan_context)[0]; in->readPrefix(); while (in->read()) {}; in->readSuffix(); @@ -569,24 +581,28 @@ try ASSERT_EQ(scan_context->total_dmfile_skipped_packs, 0); ASSERT_EQ(scan_context->total_dmfile_skipped_rows, 0); - auto filter = createGreater(Attr{col_a_define.name, col_a_define.id, DataTypeFactory::instance().get("Int64")}, Field(static_cast(10000)), 0); + auto filter = createGreater( + Attr{col_a_define.name, col_a_define.id, DataTypeFactory::instance().get("Int64")}, + Field(static_cast(10000)), + 0); scan_context = std::make_shared(); - in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - std::make_shared(filter), - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024, - /* read_segments */ {}, - /* extra_table_id_index */ InvalidColumnID, - /* scan_context */ scan_context)[0]; + in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + std::make_shared(filter), + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024, + /* read_segments */ {}, + /* extra_table_id_index */ InvalidColumnID, + /* scan_context */ scan_context)[0]; in->readPrefix(); while (in->read()) {}; @@ -662,19 +678,20 @@ try { // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_NROWS(in, 0); } } @@ -729,7 +746,9 @@ try FailPointHelper::enableFailPoint(FailPoints::force_set_page_file_write_errno); SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::force_set_page_file_write_errno); }); store->write(*db_context, db_context->getSettingsRef(), block); // Will not write PS. - ASSERT_THROW(store->flushCache(*db_context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())), DB::Exception); + ASSERT_THROW( + store->flushCache(*db_context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())), + DB::Exception); try { store->flushCache(*db_context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())); @@ -744,19 +763,20 @@ try { // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name, col_str_define.name, col_i8_define.name}), @@ -796,19 +816,20 @@ try // Test Reading first { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name}), @@ -825,19 +846,20 @@ try // Read after deletion { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name}), @@ -905,19 +927,20 @@ try { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name}), @@ -934,7 +957,8 @@ try UInt64 tso2 = 100; Block block1 = DMTestEnv::prepareSimpleWriteBlock(0, 1 * num_write_rows, false, tso1); Block block2 = DMTestEnv::prepareSimpleWriteBlock(1 * num_write_rows, 2 * num_write_rows, false, tso1); - Block block3 = DMTestEnv::prepareSimpleWriteBlock(num_write_rows / 2, num_write_rows / 2 + num_write_rows, false, tso2); + Block block3 + = DMTestEnv::prepareSimpleWriteBlock(num_write_rows / 2, num_write_rows / 2 + num_write_rows, false, tso2); switch (mode) { @@ -981,19 +1005,20 @@ try // Read without version { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name}), @@ -1004,19 +1029,20 @@ try // Read with version { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ static_cast(1), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ static_cast(1), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name}), @@ -1051,19 +1077,20 @@ try { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - settings, - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + settings, + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name}), @@ -1083,19 +1110,20 @@ try // segment2: 4, 5, 6, 7, 8 { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - settings, - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + settings, + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name}), @@ -1113,19 +1141,20 @@ try auto settings = db_context->getSettings(); const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - settings, - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + settings, + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; auto b = in->read(); ASSERT_FALSE(static_cast(b)); } @@ -1153,19 +1182,20 @@ try { // read all data of max_version const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; ASSERT_INPUTSTREAM_NROWS(in, num_rows_tso1 + num_rows_tso2); @@ -1174,19 +1204,20 @@ try { // read all data <= tso2 const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ tso2, - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso2, + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; ASSERT_INPUTSTREAM_NROWS(in, num_rows_tso1 + num_rows_tso2); @@ -1195,19 +1226,20 @@ try { // read all data <= tso1 const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ tso1, - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso1, + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; ASSERT_INPUTSTREAM_NROWS(in, num_rows_tso1); @@ -1216,19 +1248,20 @@ try { // read all data < tso1 const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ tso1 - 1, - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso1 - 1, + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; ASSERT_INPUTSTREAM_NROWS(in, 0); @@ -1275,19 +1308,20 @@ try // Read all data <= tso1 // We can only get [0, 32) with tso1 const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ tso1, - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso1, + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( @@ -1303,19 +1337,20 @@ try { // Read all data between [tso, tso2) const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ tso2 - 1, - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso2 - 1, + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( @@ -1325,46 +1360,51 @@ try createColumn(createNumbers(0, 32)), createColumn(std::vector(32, tso1)), })) - << fmt::format("Data [32, 128) after ingest with tso less than: {} are erased, should only get [0, 32)", tso2); + << fmt::format( + "Data [32, 128) after ingest with tso less than: {} are erased, should only get [0, 32)", + tso2); } { // Read all data between [tso2, tso3) const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ tso3 - 1, - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso3 - 1, + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; - ASSERT_INPUTSTREAM_NROWS(in, 32 + 16) << fmt::format("The rows number after ingest with tso less than {} is not match", tso3); + ASSERT_INPUTSTREAM_NROWS(in, 32 + 16) + << fmt::format("The rows number after ingest with tso less than {} is not match", tso3); } { // Read all data between [tso2, tso3) const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; ASSERT_INPUTSTREAM_NROWS(in, 32 + (48 - 32) + (256 - 80)) << "The rows number after ingest is not match"; @@ -1375,19 +1415,20 @@ try auto range0 = RowKeyRange::fromHandleRange(HandleRange(32, 33)); auto range1 = RowKeyRange::fromHandleRange(HandleRange(40, 41)); const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {range0, range1}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {range0, range1}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; ASSERT_INPUTSTREAM_NROWS(in, 2) << "The rows number of two point get is not match"; @@ -1421,7 +1462,8 @@ try { // Prepare DTFiles for ingesting auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); - auto [ingest_range, file_ids] = genDMFile(*dm_context, DMTestEnv::prepareSimpleWriteBlock(32, 128, false, tso2)); + auto [ingest_range, file_ids] + = genDMFile(*dm_context, DMTestEnv::prepareSimpleWriteBlock(32, 128, false, tso2)); // Enable failpoint for testing FailPointHelper::enableFailPoint(FailPoints::force_set_segment_ingest_packs_fail); FailPointHelper::enableFailPoint(FailPoints::segment_merge_after_ingest_packs); @@ -1434,19 +1476,20 @@ try // Read all data <= tso1 // We can only get [0, 32) with tso1 const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ tso1, - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso1, + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( @@ -1462,19 +1505,20 @@ try { // Read all data between [tso, tso2) const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ tso2 - 1, - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso2 - 1, + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( @@ -1484,25 +1528,28 @@ try createColumn(createNumbers(0, 32)), createColumn(std::vector(32, tso1)), })) - << fmt::format("Data [32, 128) after ingest with tso less than: {} are erased, should only get [0, 32)", tso2); + << fmt::format( + "Data [32, 128) after ingest with tso less than: {} are erased, should only get [0, 32)", + tso2); } { // Read all data between [tso2, tso3) const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; ASSERT_INPUTSTREAM_NROWS(in, 32 + 128 - 32) << "The rows number after ingest is not match"; @@ -1578,24 +1625,28 @@ try // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - // settings, - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + // settings, + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; - LOG_TRACE(&Poco::Logger::get(GET_GTEST_FULL_NAME), "start to check data of [1,{}]", num_rows_write_in_total); + LOG_TRACE( + &Poco::Logger::get(GET_GTEST_FULL_NAME), + "start to check data of [1,{}]", + num_rows_write_in_total); ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name}), @@ -1644,10 +1695,8 @@ try { block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); // Add a column of i8:Int8 for test - block.insert(DB::tests::createColumn( - createSignedNumbers(0, num_rows_write), - col_name_ddl, - col_id_ddl)); + block.insert( + DB::tests::createColumn(createSignedNumbers(0, num_rows_write), col_name_ddl, col_id_ddl)); } store->write(*db_context, db_context->getSettingsRef(), block); } @@ -1665,19 +1714,20 @@ try { // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; { @@ -1750,19 +1800,20 @@ try { // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; { @@ -1803,10 +1854,7 @@ try { block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); // Add a column of i8:Int8 for test - block.insert(DB::tests::createColumn( - createSignedNumbers(0, num_rows_write), - col_name_c1, - col_id_c1)); + block.insert(DB::tests::createColumn(createSignedNumbers(0, num_rows_write), col_name_c1, col_id_c1)); } store->write(*db_context, db_context->getSettingsRef(), block); } @@ -1823,19 +1871,20 @@ try { // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; { @@ -1891,19 +1940,20 @@ try // try read { - auto in = store->read(*db_context, - db_context->getSettingsRef(), - store->getTableColumns(), - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + store->getTableColumns(), + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, @@ -1941,19 +1991,20 @@ try // try read { - auto in = store->read(*db_context, - db_context->getSettingsRef(), - store->getTableColumns(), - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + store->getTableColumns(), + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name, col_name_to_add}), @@ -1990,19 +2041,20 @@ try // try read { - auto in = store->read(*db_context, - db_context->getSettingsRef(), - store->getTableColumns(), - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + store->getTableColumns(), + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name, col_name_to_add}), @@ -2039,19 +2091,20 @@ try // try read { - auto in = store->read(*db_context, - db_context->getSettingsRef(), - store->getTableColumns(), - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + store->getTableColumns(), + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, @@ -2088,19 +2141,20 @@ try // try read { - auto in = store->read(*db_context, - db_context->getSettingsRef(), - store->getTableColumns(), - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + store->getTableColumns(), + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; std::vector datetime_data( num_rows_write, @@ -2143,19 +2197,20 @@ try // try read { - auto in = store->read(*db_context, - db_context->getSettingsRef(), - store->getTableColumns(), - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + store->getTableColumns(), + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name, col_name_to_add}), @@ -2199,10 +2254,8 @@ try { block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); // Add a column of i8:Int8 for test - block.insert(DB::tests::createColumn( - createSignedNumbers(0, num_rows_write), - col_name_before_ddl, - col_id_ddl)); + block.insert( + DB::tests::createColumn(createSignedNumbers(0, num_rows_write), col_name_before_ddl, col_id_ddl)); } store->write(*db_context, db_context->getSettingsRef(), block); } @@ -2220,19 +2273,20 @@ try { // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; { @@ -2327,19 +2381,20 @@ try { // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; { @@ -2377,19 +2432,20 @@ try { // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; { @@ -2449,19 +2505,20 @@ try // try read { - auto in = store->read(*db_context, - db_context->getSettingsRef(), - store->getTableColumns(), - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + store->getTableColumns(), + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, Strings({DMTestEnv::pk_name, col_name_to_add}), @@ -2495,19 +2552,20 @@ try // wait till delta-merge is done FAIL_POINT_PAUSE(FailPoints::pause_until_dt_background_delta_merge); { - auto in = store->read(*db_context, - db_context->getSettingsRef(), - store->getTableColumns(), - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + store->getTableColumns(), + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; // FIXME!!! ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( @@ -2577,15 +2635,16 @@ try // write to store Block block; { - block = DMTestEnv::prepareSimpleWriteBlock(0, - num_rows_write, - false, - 2, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_STRING_TYPE, - true, - rowkey_column_size); + block = DMTestEnv::prepareSimpleWriteBlock( + 0, + num_rows_write, + false, + 2, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_STRING_TYPE, + true, + rowkey_column_size); // Add a column of col2:String for test block.insert(DB::tests::createColumn( createNumberStrings(0, num_rows_write), @@ -2608,25 +2667,28 @@ try // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { auto tmp = createNumbers(0, num_rows_write); Strings res; - std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { return genMockCommonHandle(v, rowkey_column_size); }); + std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { + return genMockCommonHandle(v, rowkey_column_size); + }); return res; }(); ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( @@ -2642,12 +2704,15 @@ try { // test readRaw const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->readRaw(*db_context, db_context->getSettingsRef(), columns, 1, /* keep_order= */ false)[0]; + BlockInputStreamPtr in + = store->readRaw(*db_context, db_context->getSettingsRef(), columns, 1, /* keep_order= */ false)[0]; // mock common handle auto common_handle_coldata = []() { auto tmp = createNumbers(0, num_rows_write); Strings res; - std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { return genMockCommonHandle(v, rowkey_column_size); }); + std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { + return genMockCommonHandle(v, rowkey_column_size); + }); return res; }(); ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( @@ -2676,33 +2741,36 @@ try // Test write multi blocks without overlap { - Block block1 = DMTestEnv::prepareSimpleWriteBlock(0, - 1 * num_write_rows, - false, - 2, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_STRING_TYPE, - true, - rowkey_column_size); - Block block2 = DMTestEnv::prepareSimpleWriteBlock(1 * num_write_rows, - 2 * num_write_rows, - false, - 2, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_STRING_TYPE, - true, - rowkey_column_size); - Block block3 = DMTestEnv::prepareSimpleWriteBlock(2 * num_write_rows, - 3 * num_write_rows, - false, - 2, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_STRING_TYPE, - true, - rowkey_column_size); + Block block1 = DMTestEnv::prepareSimpleWriteBlock( + 0, + 1 * num_write_rows, + false, + 2, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_STRING_TYPE, + true, + rowkey_column_size); + Block block2 = DMTestEnv::prepareSimpleWriteBlock( + 1 * num_write_rows, + 2 * num_write_rows, + false, + 2, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_STRING_TYPE, + true, + rowkey_column_size); + Block block3 = DMTestEnv::prepareSimpleWriteBlock( + 2 * num_write_rows, + 3 * num_write_rows, + false, + 2, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_STRING_TYPE, + true, + rowkey_column_size); store->write(*db_context, db_context->getSettingsRef(), block1); store->write(*db_context, db_context->getSettingsRef(), block2); store->write(*db_context, db_context->getSettingsRef(), block3); @@ -2712,24 +2780,27 @@ try { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { auto tmp = createNumbers(0, 3 * num_write_rows); Strings res; - std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { return genMockCommonHandle(v, rowkey_column_size); }); + std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { + return genMockCommonHandle(v, rowkey_column_size); + }); return res; }(); ASSERT_EQ(common_handle_coldata.size(), 3 * num_write_rows); @@ -2747,33 +2818,36 @@ try { UInt64 tso1 = 1; UInt64 tso2 = 100; - Block block1 = DMTestEnv::prepareSimpleWriteBlock(0, - 1 * num_write_rows, - false, - tso1, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_STRING_TYPE, - true, - rowkey_column_size); - Block block2 = DMTestEnv::prepareSimpleWriteBlock(1 * num_write_rows, - 2 * num_write_rows, - false, - tso1, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_STRING_TYPE, - true, - rowkey_column_size); - Block block3 = DMTestEnv::prepareSimpleWriteBlock(num_write_rows / 2, - num_write_rows / 2 + num_write_rows, - false, - tso2, - EXTRA_HANDLE_COLUMN_NAME, - EXTRA_HANDLE_COLUMN_ID, - EXTRA_HANDLE_COLUMN_STRING_TYPE, - true, - rowkey_column_size); + Block block1 = DMTestEnv::prepareSimpleWriteBlock( + 0, + 1 * num_write_rows, + false, + tso1, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_STRING_TYPE, + true, + rowkey_column_size); + Block block2 = DMTestEnv::prepareSimpleWriteBlock( + 1 * num_write_rows, + 2 * num_write_rows, + false, + tso1, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_STRING_TYPE, + true, + rowkey_column_size); + Block block3 = DMTestEnv::prepareSimpleWriteBlock( + num_write_rows / 2, + num_write_rows / 2 + num_write_rows, + false, + tso2, + EXTRA_HANDLE_COLUMN_NAME, + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_STRING_TYPE, + true, + rowkey_column_size); store->write(*db_context, db_context->getSettingsRef(), block1); store->write(*db_context, db_context->getSettingsRef(), block2); store->write(*db_context, db_context->getSettingsRef(), block3); @@ -2786,24 +2860,27 @@ try // Read without version { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { auto tmp = createNumbers(0, 3 * num_write_rows); Strings res; - std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { return genMockCommonHandle(v, rowkey_column_size); }); + std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { + return genMockCommonHandle(v, rowkey_column_size); + }); return res; }(); ASSERT_EQ(common_handle_coldata.size(), 3 * num_write_rows); @@ -2817,24 +2894,27 @@ try // Read with version { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ static_cast(1), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ static_cast(1), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { auto tmp = createNumbers(0, 2 * num_write_rows); Strings res; - std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { return genMockCommonHandle(v, rowkey_column_size); }); + std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { + return genMockCommonHandle(v, rowkey_column_size); + }); return res; }(); ASSERT_EQ(common_handle_coldata.size(), 2 * num_write_rows); @@ -2875,24 +2955,27 @@ try // Test Reading first { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { auto tmp = createNumbers(0, num_rows_write); Strings res; - std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { return genMockCommonHandle(v, rowkey_column_size); }); + std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { + return genMockCommonHandle(v, rowkey_column_size); + }); return res; }(); ASSERT_EQ(common_handle_coldata.size(), num_rows_write); @@ -2914,24 +2997,27 @@ try // Read after deletion { const auto & columns = store->getTableColumns(); - BlockInputStreamPtr in = store->read(*db_context, - db_context->getSettingsRef(), - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024)[0]; + BlockInputStreamPtr in = store->read( + *db_context, + db_context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024)[0]; // mock common handle, data range after deletion is [64, 128) auto common_handle_coldata = []() { auto tmp = createNumbers(num_deleted_rows, num_rows_write); Strings res; - std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { return genMockCommonHandle(v, rowkey_column_size); }); + std::transform(tmp.begin(), tmp.end(), std::back_inserter(res), [](Int64 v) { + return genMockCommonHandle(v, rowkey_column_size); + }); return res; }(); ASSERT_EQ(common_handle_coldata.size(), num_rows_write - num_deleted_rows); @@ -2981,24 +3067,28 @@ try // read all columns from store const auto & columns = store->getTableColumns(); - BlockInputStreams ins = store->read(*db_context, - db_context->getSettingsRef(), - // settings, - columns, - {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* max_version= */ std::numeric_limits::max(), - EMPTY_FILTER, - std::vector{}, - 0, - TRACING_NAME, - /* keep_order= */ false, - /* is_fast_scan= */ false, - /* expected_block_size= */ 1024); + BlockInputStreams ins = store->read( + *db_context, + db_context->getSettingsRef(), + // settings, + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + std::vector{}, + 0, + TRACING_NAME, + /* keep_order= */ false, + /* is_fast_scan= */ false, + /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; - LOG_TRACE(&Poco::Logger::get(GET_GTEST_FULL_NAME), "start to check data of [1,{}]", num_rows_write_in_total); + LOG_TRACE( + &Poco::Logger::get(GET_GTEST_FULL_NAME), + "start to check data of [1,{}]", + num_rows_write_in_total); ASSERT_UNORDERED_INPUTSTREAM_COLS_UR( in, @@ -3019,7 +3109,14 @@ CATCH INSTANTIATE_TEST_CASE_P( TestMode, DeltaMergeStoreRWTest, - testing::Values(TestMode::V1_BlockOnly, TestMode::V2_BlockOnly, TestMode::V2_FileOnly, TestMode::V2_Mix, TestMode::V3_BlockOnly, TestMode::V3_FileOnly, TestMode::V3_Mix), + testing::Values( + TestMode::V1_BlockOnly, + TestMode::V2_BlockOnly, + TestMode::V2_FileOnly, + TestMode::V2_Mix, + TestMode::V3_BlockOnly, + TestMode::V3_FileOnly, + TestMode::V3_Mix), testModeToString); @@ -3028,10 +3125,7 @@ class DeltaMergeStoreMergeDeltaBySegmentTest , public testing::WithParamInterface> { public: - DeltaMergeStoreMergeDeltaBySegmentTest() - { - std::tie(ps_ver, pk_type) = GetParam(); - } + DeltaMergeStoreMergeDeltaBySegmentTest() { std::tie(ps_ver, pk_type) = GetParam(); } void SetUp() override { @@ -3052,19 +3146,23 @@ class DeltaMergeStoreMergeDeltaBySegmentTest void setupDMStore() { auto cols = DMTestEnv::getDefaultColumns(pk_type); - store = std::make_shared(*db_context, - false, - "test", - DB::base::TiFlashStorageTestBasic::getCurrentFullTestName(), - NullspaceID, - 101, - true, - *cols, - (*cols)[0], - pk_type == DMTestEnv::PkType::CommonHandle, - 1, - DeltaMergeStore::Settings()); - dm_context = store->newDMContext(*db_context, db_context->getSettingsRef(), DB::base::TiFlashStorageTestBasic::getCurrentFullTestName()); + store = std::make_shared( + *db_context, + false, + "test", + DB::base::TiFlashStorageTestBasic::getCurrentFullTestName(), + NullspaceID, + 101, + true, + *cols, + (*cols)[0], + pk_type == DMTestEnv::PkType::CommonHandle, + 1, + DeltaMergeStore::Settings()); + dm_context = store->newDMContext( + *db_context, + db_context->getSettingsRef(), + DB::base::TiFlashStorageTestBasic::getCurrentFullTestName()); } protected: @@ -3081,7 +3179,10 @@ INSTANTIATE_TEST_CASE_P( DeltaMergeStoreMergeDeltaBySegmentTest, ::testing::Combine( ::testing::Values(2, 3), - ::testing::Values(DMTestEnv::PkType::HiddenTiDBRowID, DMTestEnv::PkType::CommonHandle, DMTestEnv::PkType::PkIsHandleInt64)), + ::testing::Values( + DMTestEnv::PkType::HiddenTiDBRowID, + DMTestEnv::PkType::CommonHandle, + DMTestEnv::PkType::PkIsHandleInt64)), [](const testing::TestParamInfo> & info) { const auto [ps_ver, pk_type] = info.param; return fmt::format("PsV{}_{}", ps_ver, DMTestEnv::PkTypeToString(pk_type)); @@ -3094,7 +3195,8 @@ try { { // Write data to first 3 segments. - auto newly_written_rows = helper->rows_by_segments[0] + helper->rows_by_segments[1] + helper->rows_by_segments[2]; + auto newly_written_rows + = helper->rows_by_segments[0] + helper->rows_by_segments[1] + helper->rows_by_segments[2]; Block block = DMTestEnv::prepareSimpleWriteBlock(0, newly_written_rows, false, pk_type, 5 /* new tso */); store->write(*db_context, db_context->getSettingsRef(), block); store->flushCache(dm_context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())); @@ -3204,7 +3306,8 @@ try { { // Write data to first 3 segments. - auto newly_written_rows = helper->rows_by_segments[0] + helper->rows_by_segments[1] + helper->rows_by_segments[2]; + auto newly_written_rows + = helper->rows_by_segments[0] + helper->rows_by_segments[1] + helper->rows_by_segments[2]; Block block = DMTestEnv::prepareSimpleWriteBlock(0, newly_written_rows, false, pk_type, 5 /* new tso */); store->write(*db_context, db_context->getSettingsRef(), block); store->flushCache(dm_context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())); @@ -3236,7 +3339,8 @@ try { { // Write data to first 3 segments and flush. - auto newly_written_rows = helper->rows_by_segments[0] + helper->rows_by_segments[1] + helper->rows_by_segments[2]; + auto newly_written_rows + = helper->rows_by_segments[0] + helper->rows_by_segments[1] + helper->rows_by_segments[2]; Block block = DMTestEnv::prepareSimpleWriteBlock(0, newly_written_rows, false, pk_type, 5 /* new tso */); store->write(*db_context, db_context->getSettingsRef(), block); store->flushCache(dm_context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())); @@ -3252,7 +3356,12 @@ try { // Write new data to segment[1] without flush. auto newly_written_rows = helper->rows_by_segments[1]; - Block block = DMTestEnv::prepareSimpleWriteBlock(helper->rows_by_segments[0], helper->rows_by_segments[0] + newly_written_rows, false, pk_type, 10 /* new tso */); + Block block = DMTestEnv::prepareSimpleWriteBlock( + helper->rows_by_segments[0], + helper->rows_by_segments[0] + newly_written_rows, + false, + pk_type, + 10 /* new tso */); store->write(*db_context, db_context->getSettingsRef(), block); helper->expected_delta_rows[1] += helper->rows_by_segments[1]; @@ -3286,14 +3395,20 @@ try { // Write new data to segment[1] without flush. auto newly_written_rows = helper->rows_by_segments[1]; - Block block = DMTestEnv::prepareSimpleWriteBlock(helper->rows_by_segments[0], helper->rows_by_segments[0] + newly_written_rows, false, pk_type, 10 /* new tso */); + Block block = DMTestEnv::prepareSimpleWriteBlock( + helper->rows_by_segments[0], + helper->rows_by_segments[0] + newly_written_rows, + false, + pk_type, + 10 /* new tso */); store->write(*db_context, db_context->getSettingsRef(), block); helper->expected_delta_rows[1] += helper->rows_by_segments[1]; helper->verifyExpectedRowsForAllSegments(); } auto sp_flush_commit = SyncPointCtl::enableInScope("before_ColumnFileFlushTask::commit"); - auto sp_merge_delta_retry = SyncPointCtl::enableInScope("before_DeltaMergeStore::mergeDeltaBySegment|retry_segment"); + auto sp_merge_delta_retry + = SyncPointCtl::enableInScope("before_DeltaMergeStore::mergeDeltaBySegment|retry_segment"); // Start a flush and suspend it before flushCommit. auto th_flush = std::async([&]() { @@ -3336,7 +3451,8 @@ TEST_P(DeltaMergeStoreMergeDeltaBySegmentTest, RetryBySplit) try { auto sp_split_prepare = SyncPointCtl::enableInScope("before_Segment::prepareSplit"); - auto sp_merge_delta_retry = SyncPointCtl::enableInScope("before_DeltaMergeStore::mergeDeltaBySegment|retry_segment"); + auto sp_merge_delta_retry + = SyncPointCtl::enableInScope("before_DeltaMergeStore::mergeDeltaBySegment|retry_segment"); // Start a split and suspend it during prepareSplit to simulate a long-running split. auto th_split = std::async([&] { @@ -3382,7 +3498,12 @@ try { // Write to the new segment1 + segment2 after split. auto newly_written_rows = helper->rows_by_segments[1] + helper->rows_by_segments[2]; - Block block = DMTestEnv::prepareSimpleWriteBlock(helper->rows_by_segments[0], helper->rows_by_segments[0] + newly_written_rows, false, pk_type, 10 /* new tso */); + Block block = DMTestEnv::prepareSimpleWriteBlock( + helper->rows_by_segments[0], + helper->rows_by_segments[0] + newly_written_rows, + false, + pk_type, + 10 /* new tso */); store->write(*db_context, db_context->getSettingsRef(), block); helper->expected_delta_rows[1] += helper->rows_by_segments[1]; helper->expected_delta_rows[2] += helper->rows_by_segments[2]; diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index b52fae1bdfc..7b10fb69947 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -120,7 +120,12 @@ void StorageDeltaMerge::updateTableColumnInfo() { const ColumnsDescription & columns = getColumns(); - LOG_INFO(log, "updateTableColumnInfo, table_name={} ordinary=\"{}\" materialized=\"{}\"", table_column_info->table_name, columns.ordinary.toString(), columns.materialized.toString()); + LOG_INFO( + log, + "updateTableColumnInfo, table_name={} ordinary=\"{}\" materialized=\"{}\"", + table_column_info->table_name, + columns.ordinary.toString(), + columns.materialized.toString()); auto & pk_expr_ast = table_column_info->pk_expr_ast; auto & handle_column_define = table_column_info->handle_column_define; @@ -176,7 +181,9 @@ void StorageDeltaMerge::updateTableColumnInfo() /// If TableInfo from TiDB is not empty, we get column id and default value from TiDB auto & columns = tidb_table_info.columns; col_def.id = tidb_table_info.getColumnID(col_def.name); - auto itr = std::find_if(columns.begin(), columns.end(), [&](const ColumnInfo & v) { return v.id == col_def.id; }); + auto itr = std::find_if(columns.begin(), columns.end(), [&](const ColumnInfo & v) { + return v.id == col_def.id; + }); if (itr != columns.end()) { col_def.default_value = itr->defaultValueToField(); @@ -205,7 +212,10 @@ void StorageDeltaMerge::updateTableColumnInfo() pks_combined_bytes += col.type->getSizeOfValueInMemory(); if (pks_combined_bytes > sizeof(Handle)) { - throw Exception(fmt::format("pk columns bytes exceeds size limit, {} > {}", pks_combined_bytes, sizeof(Handle))); + throw Exception(fmt::format( + "pk columns bytes exceeds size limit, {} > {}", + pks_combined_bytes, + sizeof(Handle))); } } if (pks.size() == 1) @@ -277,14 +287,12 @@ void StorageDeltaMerge::updateTableColumnInfo() fmt_buf.joinStr( all_columns.begin(), all_columns.end(), - [](const auto & col, FmtBuffer & fb) { - fb.append(col.name); - }, + [](const auto & col, FmtBuffer & fb) { fb.append(col.name); }, ","); - throw Exception( - fmt::format("Can not create table without primary key. Primary keys should be: {}, but only these columns are found:{}", - fmt::join(pks, ","), - fmt_buf.toString())); + throw Exception(fmt::format( + "Can not create table without primary key. Primary keys should be: {}, but only these columns are found:{}", + fmt::join(pks, ","), + fmt_buf.toString())); } assert(!table_column_defines.empty()); @@ -351,7 +359,12 @@ Block StorageDeltaMerge::buildInsertBlock(bool is_import, bool is_delete, const appendIntoHandleColumn(handle_data, pk_column_types[c], pk_columns[c]); } - addColumnToBlock(to_write, EXTRA_HANDLE_COLUMN_ID, EXTRA_HANDLE_COLUMN_NAME, is_common_handle ? EXTRA_HANDLE_COLUMN_STRING_TYPE : EXTRA_HANDLE_COLUMN_INT_TYPE, std::move(handle_column)); + addColumnToBlock( + to_write, + EXTRA_HANDLE_COLUMN_ID, + EXTRA_HANDLE_COLUMN_NAME, + is_common_handle ? EXTRA_HANDLE_COLUMN_STRING_TYPE : EXTRA_HANDLE_COLUMN_INT_TYPE, + std::move(handle_column)); } auto block = DeltaMergeStore::addExtraColumnIfNeed(global_context, store->getHandle(), std::move(to_write)); @@ -454,7 +467,8 @@ class DMBlockOutputStream : public IBlockOutputStream } catch (DB::Exception & e) { - e.addMessage(fmt::format("(while writing to table `{}`.`{}`)", store->getDatabaseName(), store->getTableName())); + e.addMessage( + fmt::format("(while writing to table `{}`.`{}`)", store->getDatabaseName(), store->getTableName())); throw; } @@ -529,7 +543,8 @@ WriteResult StorageDeltaMerge::write(Block & block, const Settings & settings) } if (!ok) { - throw Exception(fmt::format("The column-id in written block is not properly set [name={}] [id={}]", name, cid)); + throw Exception( + fmt::format("The column-id in written block is not properly set [name={}] [id={}]", name, cid)); } } #endif @@ -592,10 +607,15 @@ std::unordered_set parseSegmentSet(const ASTPtr & ast) return ids; } - throw Exception(fmt::format("Unable to parse segment IDs in literal form: `{}`", partition_ast.fields_str.toString())); + throw Exception( + fmt::format("Unable to parse segment IDs in literal form: `{}`", partition_ast.fields_str.toString())); } -void setColumnsToRead(const DeltaMergeStorePtr & store, ColumnDefines & columns_to_read, size_t & extra_table_id_index, const Names & column_names) +void setColumnsToRead( + const DeltaMergeStorePtr & store, + ColumnDefines & columns_to_read, + size_t & extra_table_id_index, + const Names & column_names) { auto header = store->getHeader(); for (size_t i = 0; i < column_names.size(); i++) @@ -702,9 +722,7 @@ DM::RowKeyRanges StorageDeltaMerge::parseMvccQueryInfo( fmt_buf.joinStr( ranges.begin(), ranges.end(), - [](const auto & range, FmtBuffer & fb) { - fb.append(range.toDebugString()); - }, + [](const auto & range, FmtBuffer & fb) { fb.append(range.toDebugString()); }, ","); LOG_TRACE(tracing_logger, "reading ranges: {}", fmt_buf.toString()); } @@ -712,10 +730,11 @@ DM::RowKeyRanges StorageDeltaMerge::parseMvccQueryInfo( return ranges; } -DM::RSOperatorPtr StorageDeltaMerge::buildRSOperator(const std::unique_ptr & dag_query, - const ColumnDefines & columns_to_read, - const Context & context, - const LoggerPtr & tracing_logger) +DM::RSOperatorPtr StorageDeltaMerge::buildRSOperator( + const std::unique_ptr & dag_query, + const ColumnDefines & columns_to_read, + const Context & context, + const LoggerPtr & tracing_logger) { RUNTIME_CHECK(dag_query != nullptr); // build rough set operator @@ -726,16 +745,16 @@ DM::RSOperatorPtr StorageDeltaMerge::buildRSOperator(const std::unique_ptr Attr { const ColumnDefines & defines = this->getAndMaybeInitStore()->getTableColumns(); - auto iter = std::find_if( - defines.begin(), - defines.end(), - [column_id](const ColumnDefine & d) -> bool { return d.id == column_id; }); + auto iter = std::find_if(defines.begin(), defines.end(), [column_id](const ColumnDefine & d) -> bool { + return d.id == column_id; + }); if (iter != defines.end()) return Attr{.col_name = iter->name, .col_id = iter->id, .type = iter->type}; // Maybe throw an exception? Or check if `type` is nullptr before creating filter? return Attr{.col_name = "", .col_id = column_id, .type = DataTypePtr{}}; }; - rs_operator = FilterParser::parseDAGQuery(*dag_query, columns_to_read, std::move(create_attr_by_column_id), log); + rs_operator + = FilterParser::parseDAGQuery(*dag_query, columns_to_read, std::move(create_attr_by_column_id), log); if (likely(rs_operator != DM::EMPTY_RS_OPERATOR)) LOG_DEBUG(tracing_logger, "Rough set filter: {}", rs_operator->toDebugString()); } @@ -745,12 +764,13 @@ DM::RSOperatorPtr StorageDeltaMerge::buildRSOperator(const std::unique_ptr & pushed_down_filters, - const ColumnDefines & columns_to_read, - const Context & context, - const LoggerPtr & tracing_logger) +DM::PushDownFilterPtr StorageDeltaMerge::buildPushDownFilter( + const RSOperatorPtr & rs_operator, + const ColumnInfos & table_scan_column_info, + const google::protobuf::RepeatedPtrField & pushed_down_filters, + const ColumnDefines & columns_to_read, + const Context & context, + const LoggerPtr & tracing_logger) { if (pushed_down_filters.empty()) { @@ -771,7 +791,10 @@ DM::PushDownFilterPtr StorageDeltaMerge::buildPushDownFilter(const RSOperatorPtr filter_columns->reserve(filter_col_id_set.size()); for (const auto & cid : filter_col_id_set) { - RUNTIME_CHECK_MSG(columns_to_read_map.contains(cid), "Filter ColumnID({}) not found in columns_to_read_map", cid); + RUNTIME_CHECK_MSG( + columns_to_read_map.contains(cid), + "Filter ColumnID({}) not found in columns_to_read_map", + cid); filter_columns->emplace_back(columns_to_read_map.at(cid)); } @@ -798,7 +821,8 @@ DM::PushDownFilterPtr StorageDeltaMerge::buildPushDownFilter(const RSOperatorPtr RUNTIME_CHECK_MSG(columns_to_read_map.contains(cid), "ColumnID({}) not found in columns_to_read_map", cid); source_columns_of_analyzer.emplace_back(columns_to_read_map.at(cid).name, columns_to_read_map.at(cid).type); } - std::unique_ptr analyzer = std::make_unique(source_columns_of_analyzer, context); + std::unique_ptr analyzer + = std::make_unique(source_columns_of_analyzer, context); // Build the extra cast ExpressionActionsPtr extra_cast = nullptr; @@ -806,11 +830,14 @@ DM::PushDownFilterPtr StorageDeltaMerge::buildPushDownFilter(const RSOperatorPtr std::vector may_need_add_cast_column; may_need_add_cast_column.reserve(table_scan_column_info.size()); for (const auto & col : table_scan_column_info) - may_need_add_cast_column.push_back(!col.hasGeneratedColumnFlag() && filter_col_id_set.contains(col.id) && col.id != -1); + may_need_add_cast_column.push_back( + !col.hasGeneratedColumnFlag() && filter_col_id_set.contains(col.id) && col.id != -1); ExpressionActionsChain chain; auto & step = analyzer->initAndGetLastStep(chain); auto & actions = step.actions; - if (auto [has_cast, casted_columns] = analyzer->buildExtraCastsAfterTS(actions, may_need_add_cast_column, table_scan_column_info); has_cast) + if (auto [has_cast, casted_columns] + = analyzer->buildExtraCastsAfterTS(actions, may_need_add_cast_column, table_scan_column_info); + has_cast) { NamesWithAliases project_cols; for (size_t i = 0; i < columns_to_read.size(); ++i) @@ -830,7 +857,8 @@ DM::PushDownFilterPtr StorageDeltaMerge::buildPushDownFilter(const RSOperatorPtr } // build filter expression actions - auto [before_where, filter_column_name, project_after_where] = ::DB::buildPushDownFilter(pushed_down_filters, *analyzer); + auto [before_where, filter_column_name, project_after_where] + = ::DB::buildPushDownFilter(pushed_down_filters, *analyzer); LOG_DEBUG(tracing_logger, "Push down filter: {}", before_where->dumpActions()); // record current column defines @@ -841,22 +869,35 @@ DM::PushDownFilterPtr StorageDeltaMerge::buildPushDownFilter(const RSOperatorPtr const auto & current_names_and_types = analyzer->getCurrentInputColumns(); for (size_t i = 0; i < table_scan_column_info.size(); ++i) { - if (table_scan_column_info[i].hasGeneratedColumnFlag() || table_scan_column_info[i].id == EXTRA_TABLE_ID_COLUMN_ID) + if (table_scan_column_info[i].hasGeneratedColumnFlag() + || table_scan_column_info[i].id == EXTRA_TABLE_ID_COLUMN_ID) continue; auto col = columns_to_read_map.at(table_scan_column_info[i].id); - RUNTIME_CHECK_MSG(col.name == current_names_and_types[i].name, "Column name mismatch, expect: {}, actual: {}", col.name, current_names_and_types[i].name); + RUNTIME_CHECK_MSG( + col.name == current_names_and_types[i].name, + "Column name mismatch, expect: {}, actual: {}", + col.name, + current_names_and_types[i].name); columns_after_cast->push_back(col); columns_after_cast->back().type = current_names_and_types[i].type; } } - return std::make_shared(rs_operator, before_where, project_after_where, filter_columns, filter_column_name, extra_cast, columns_after_cast); + return std::make_shared( + rs_operator, + before_where, + project_after_where, + filter_columns, + filter_column_name, + extra_cast, + columns_after_cast); } -DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter(const SelectQueryInfo & query_info, - const ColumnDefines & columns_to_read, - const Context & context, - const LoggerPtr & tracing_logger) +DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter( + const SelectQueryInfo & query_info, + const ColumnDefines & columns_to_read, + const Context & context, + const LoggerPtr & tracing_logger) { const auto & dag_query = query_info.dag_query; if (unlikely(dag_query == nullptr)) @@ -869,11 +910,25 @@ DM::PushDownFilterPtr StorageDeltaMerge::parsePushDownFilter(const SelectQueryIn const auto & pushed_down_filters = dag_query->pushed_down_filters; if (unlikely(context.getSettingsRef().force_push_down_all_filters_to_scan) && !dag_query->filters.empty()) { - google::protobuf::RepeatedPtrField merged_filters{pushed_down_filters.begin(), pushed_down_filters.end()}; + google::protobuf::RepeatedPtrField merged_filters{ + pushed_down_filters.begin(), + pushed_down_filters.end()}; merged_filters.MergeFrom(dag_query->filters); - return buildPushDownFilter(rs_operator, columns_to_read_info, merged_filters, columns_to_read, context, tracing_logger); + return buildPushDownFilter( + rs_operator, + columns_to_read_info, + merged_filters, + columns_to_read, + context, + tracing_logger); } - return buildPushDownFilter(rs_operator, columns_to_read_info, pushed_down_filters, columns_to_read, context, tracing_logger); + return buildPushDownFilter( + rs_operator, + columns_to_read_info, + pushed_down_filters, + columns_to_read, + context, + tracing_logger); } BlockInputStreams StorageDeltaMerge::read( @@ -947,7 +1002,9 @@ BlockInputStreams StorageDeltaMerge::read( return streams; } -RuntimeFilteList StorageDeltaMerge::parseRuntimeFilterList(const SelectQueryInfo & query_info, const Context & db_context) const +RuntimeFilteList StorageDeltaMerge::parseRuntimeFilterList( + const SelectQueryInfo & query_info, + const Context & db_context) const { if (db_context.getDAGContext() == nullptr || query_info.dag_query == nullptr) { @@ -1034,8 +1091,7 @@ void StorageDeltaMerge::read( LOG_TRACE(tracing_logger, "[ranges: {}] [concurrency: {}]", ranges.size(), group_builder.concurrency()); } -DM::Remote::DisaggPhysicalTableReadSnapshotPtr -StorageDeltaMerge::writeNodeBuildRemoteReadSnapshot( +DM::Remote::DisaggPhysicalTableReadSnapshotPtr StorageDeltaMerge::writeNodeBuildRemoteReadSnapshot( const Names & column_names, const SelectQueryInfo & query_info, const Context & context, @@ -1081,7 +1137,10 @@ void StorageDeltaMerge::flushCache(const Context & context) flushCache(context, DM::RowKeyRange::newAll(is_common_handle, rowkey_column_size), /* try_until_succeed */ true); } -bool StorageDeltaMerge::flushCache(const Context & context, const DM::RowKeyRange & range_to_flush, bool try_until_succeed) +bool StorageDeltaMerge::flushCache( + const Context & context, + const DM::RowKeyRange & range_to_flush, + bool try_until_succeed) { return getAndMaybeInitStore()->flushCache(context, range_to_flush, try_until_succeed); } @@ -1091,7 +1150,9 @@ void StorageDeltaMerge::mergeDelta(const Context & context) getAndMaybeInitStore()->mergeDeltaAll(context); } -std::optional StorageDeltaMerge::mergeDeltaBySegment(const Context & context, const DM::RowKeyValue & start_key) +std::optional StorageDeltaMerge::mergeDeltaBySegment( + const Context & context, + const DM::RowKeyValue & start_key) { return getAndMaybeInitStore()->mergeDeltaBySegment(context, start_key); } @@ -1106,10 +1167,7 @@ void StorageDeltaMerge::cleanPreIngestFiles( const std::vector & external_files, const Settings & settings) { - getAndMaybeInitStore()->cleanPreIngestFiles( - global_context, - settings, - external_files); + getAndMaybeInitStore()->cleanPreIngestFiles(global_context, settings, external_files); } UInt64 StorageDeltaMerge::ingestFiles( @@ -1119,12 +1177,7 @@ UInt64 StorageDeltaMerge::ingestFiles( const Settings & settings) { GET_METRIC(tiflash_storage_command_count, type_ingest).Increment(); - return getAndMaybeInitStore()->ingestFiles( - global_context, - settings, - range, - external_files, - clear_data_in_range); + return getAndMaybeInitStore()->ingestFiles(global_context, settings, range, external_files, clear_data_in_range); } void StorageDeltaMerge::ingestSegmentsFromCheckpointInfo( @@ -1133,11 +1186,7 @@ void StorageDeltaMerge::ingestSegmentsFromCheckpointInfo( const Settings & settings) { GET_METRIC(tiflash_storage_command_count, type_ingest_checkpoint).Increment(); - return getAndMaybeInitStore()->ingestSegmentsFromCheckpointInfo( - global_context, - settings, - range, - checkpoint_info); + return getAndMaybeInitStore()->ingestSegmentsFromCheckpointInfo(global_context, settings, range, checkpoint_info); } UInt64 StorageDeltaMerge::onSyncGc(Int64 limit, const GCOptions & gc_options) @@ -1230,7 +1279,11 @@ void StorageDeltaMerge::deleteRows(const Context & context, size_t delete_rows) size_t after_delete_rows = getRows(store, context, DM::RowKeyRange::newAll(is_common_handle, rowkey_column_size)); if (after_delete_rows != total_rows - delete_rows) - LOG_ERROR(log, "Rows after delete range not match, expected: {}, got: {}", (total_rows - delete_rows), after_delete_rows); + LOG_ERROR( + log, + "Rows after delete range not match, expected: {}, got: {}", + (total_rows - delete_rows), + after_delete_rows); } DM::DeltaMergeStorePtr StorageDeltaMerge::getStoreIfInited() const @@ -1242,14 +1295,20 @@ DM::DeltaMergeStorePtr StorageDeltaMerge::getStoreIfInited() const return nullptr; } -std::pair StorageDeltaMerge::getSchemaSnapshotAndBlockForDecoding(const TableStructureLockHolder & table_structure_lock, bool need_block) +std::pair StorageDeltaMerge::getSchemaSnapshotAndBlockForDecoding( + const TableStructureLockHolder & table_structure_lock, + bool need_block) { (void)table_structure_lock; std::lock_guard lock{decode_schema_mutex}; if (!decoding_schema_snapshot || decoding_schema_changed) { auto & store = getAndMaybeInitStore(); - decoding_schema_snapshot = std::make_shared(store->getStoreColumns(), tidb_table_info, store->getHandle(), decoding_schema_epoch++); + decoding_schema_snapshot = std::make_shared( + store->getStoreColumns(), + tidb_table_info, + store->getHandle(), + decoding_schema_epoch++); cache_blocks.clear(); decoding_schema_changed = false; } @@ -1317,11 +1376,7 @@ void StorageDeltaMerge::updateTombstone( const SchemaNameMapper & name_mapper, const Context & context) { - alterImpl( - commands, - database_name, - name_mapper.mapTableName(table_info), - context); + alterImpl(commands, database_name, name_mapper.mapTableName(table_info), context); } void StorageDeltaMerge::alter( @@ -1331,11 +1386,7 @@ void StorageDeltaMerge::alter( const String & table_name_, const Context & context) { - alterImpl( - commands, - database_name, - table_name_, - context); + alterImpl(commands, database_name, table_name_, context); } /// If any ddl statement change StorageDeltaMerge's schema, @@ -1509,7 +1560,8 @@ ColumnDefines StorageDeltaMerge::getStoreColumnDefines() const cols.emplace_back(getTagColumnDefine()); for (const auto & col : table_column_info->table_column_defines) { - if (col.id != table_column_info->handle_column_define.id && col.id != VERSION_COLUMN_ID && col.id != TAG_COLUMN_ID) + if (col.id != table_column_info->handle_column_define.id && col.id != VERSION_COLUMN_ID + && col.id != TAG_COLUMN_ID) { cols.emplace_back(col); } @@ -1533,11 +1585,12 @@ void StorageDeltaMerge::rename( // For DatabaseTiFlash, simply update store's database is OK. // `store->getTableName() == new_table_name` only keep for mock test. bool clean_rename = !data_path_contains_database_name && getTableName() == new_table_name; - RUNTIME_ASSERT(clean_rename, - log, - "should never rename the directories when renaming table, new_database_name={}, new_table_name={}", - new_database_name, - new_table_name); + RUNTIME_ASSERT( + clean_rename, + log, + "should never rename the directories when renaming table, new_database_name={}, new_table_name={}", + new_database_name, + new_table_name); } if (storeInited()) { @@ -1623,11 +1676,15 @@ void updateDeltaMergeTableCreateStatement( } else { - throw Exception(fmt::format("Try to update table({}.{}) statement with no primary key. ", database_name, table_name)); + throw Exception(fmt::format( + "Try to update table({}.{}) statement with no primary key. ", + database_name, + table_name)); } } - std::shared_ptr tableinfo_literal = std::make_shared(Field(table_info->get().serialize())); + std::shared_ptr tableinfo_literal + = std::make_shared(Field(table_info->get().serialize())); auto tombstone_ast = std::make_shared(Field(tombstone)); auto & storage_ast = typeid_cast(ast); @@ -1684,7 +1741,10 @@ void StorageDeltaMerge::modifyASTStorage(ASTStorage * storage_ast, const TiDB::T args->children.at(1) = literal; else throw Exception( - fmt::format("Wrong arguments num: {} in table: {} in modifyASTStorage", args->children.size(), this->getTableName()), + fmt::format( + "Wrong arguments num: {} in table: {} in modifyASTStorage", + args->children.size(), + this->getTableName()), ErrorCodes::BAD_ARGUMENTS); } diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 91f89c4a12a..73615f68d21 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -81,8 +81,7 @@ class StorageDeltaMerge size_t max_block_size, unsigned num_streams) override; - DM::Remote::DisaggPhysicalTableReadSnapshotPtr - writeNodeBuildRemoteReadSnapshot( + DM::Remote::DisaggPhysicalTableReadSnapshotPtr writeNodeBuildRemoteReadSnapshot( const Names & column_names, const SelectQueryInfo & query_info, const Context & context, @@ -113,9 +112,7 @@ class StorageDeltaMerge // If the "ingest" has been aborted, we use this method to // clean the generated external files on the fly. - void cleanPreIngestFiles( - const std::vector & external_files, - const Settings & settings); + void cleanPreIngestFiles(const std::vector & external_files, const Settings & settings); /// Return the 'ingtested bytes'. UInt64 ingestFiles( @@ -182,10 +179,7 @@ class StorageDeltaMerge void checkStatus(const Context & context) override; void deleteRows(const Context &, size_t rows) override; - const DM::DeltaMergeStorePtr & getStore() - { - return getAndMaybeInitStore(); - } + const DM::DeltaMergeStorePtr & getStore() { return getAndMaybeInitStore(); } DM::DeltaMergeStorePtr getStoreIfInited() const; @@ -193,23 +187,23 @@ class StorageDeltaMerge size_t getRowKeyColumnSize() const override { return rowkey_column_size; } - std::pair getSchemaSnapshotAndBlockForDecoding(const TableStructureLockHolder & table_structure_lock, bool /* need_block */) override; + std::pair getSchemaSnapshotAndBlockForDecoding( + const TableStructureLockHolder & table_structure_lock, + bool /* need_block */) override; void releaseDecodingBlock(Int64 block_decoding_schema_epoch, BlockUPtr block) override; bool initStoreIfDataDirExist(ThreadPool * thread_pool) override; - DM::DMConfigurationOpt createChecksumConfig() const - { - return DM::DMChecksumConfig::fromDBContext(global_context); - } + DM::DMConfigurationOpt createChecksumConfig() const { return DM::DMChecksumConfig::fromDBContext(global_context); } - static DM::PushDownFilterPtr buildPushDownFilter(const DM::RSOperatorPtr & rs_operator, - const ColumnInfos & table_scan_column_info, - const google::protobuf::RepeatedPtrField & pushed_down_filters, - const DM::ColumnDefines & columns_to_read, - const Context & context, - const LoggerPtr & tracing_logger); + static DM::PushDownFilterPtr buildPushDownFilter( + const DM::RSOperatorPtr & rs_operator, + const ColumnInfos & table_scan_column_info, + const google::protobuf::RepeatedPtrField & pushed_down_filters, + const DM::ColumnDefines & columns_to_read, + const Context & context, + const LoggerPtr & tracing_logger); #ifndef DBMS_PUBLIC_GTEST protected: @@ -240,31 +234,31 @@ class StorageDeltaMerge DataTypePtr getPKTypeImpl() const override; DM::DeltaMergeStorePtr & getAndMaybeInitStore(ThreadPool * thread_pool = nullptr); - bool storeInited() const - { - return store_inited.load(std::memory_order_acquire); - } + bool storeInited() const { return store_inited.load(std::memory_order_acquire); } void updateTableColumnInfo(); ColumnsDescription getNewColumnsDescription(const TiDB::TableInfo & table_info); DM::ColumnDefines getStoreColumnDefines() const override; bool dataDirExist(); void shutdownImpl(); - DM::RSOperatorPtr buildRSOperator(const std::unique_ptr & dag_query, - const DM::ColumnDefines & columns_to_read, - const Context & context, - const LoggerPtr & tracing_logger); + DM::RSOperatorPtr buildRSOperator( + const std::unique_ptr & dag_query, + const DM::ColumnDefines & columns_to_read, + const Context & context, + const LoggerPtr & tracing_logger); /// Get filters from query to construct rough set operation and push down filters. - DM::PushDownFilterPtr parsePushDownFilter(const SelectQueryInfo & query_info, - const DM::ColumnDefines & columns_to_read, - const Context & context, - const LoggerPtr & tracing_logger); - - DM::RowKeyRanges parseMvccQueryInfo(const DB::MvccQueryInfo & mvcc_query_info, - unsigned num_streams, - const Context & context, - const String & req_id, - const LoggerPtr & tracing_logger); + DM::PushDownFilterPtr parsePushDownFilter( + const SelectQueryInfo & query_info, + const DM::ColumnDefines & columns_to_read, + const Context & context, + const LoggerPtr & tracing_logger); + + DM::RowKeyRanges parseMvccQueryInfo( + const DB::MvccQueryInfo & mvcc_query_info, + unsigned num_streams, + const Context & context, + const String & req_id, + const LoggerPtr & tracing_logger); RuntimeFilteList parseRuntimeFilterList(const SelectQueryInfo & query_info, const Context & db_context) const; diff --git a/dbms/src/Storages/Transaction/ApplySnapshot.cpp b/dbms/src/Storages/Transaction/ApplySnapshot.cpp index 21f5a608eef..d1726e78c98 100644 --- a/dbms/src/Storages/Transaction/ApplySnapshot.cpp +++ b/dbms/src/Storages/Transaction/ApplySnapshot.cpp @@ -74,9 +74,11 @@ void KVStore::checkAndApplyPreHandledSnapshot(const RegionPtrWrap & new_region, } else if (old_applied_index == new_index) { - LOG_WARNING(log, - "{} already has same applied index, just ignore next process. Please check log whether server crashed after successfully applied snapshot.", - old_region->getDebugString()); + LOG_WARNING( + log, + "{} already has same applied index, just ignore next process. Please check log whether server crashed " + "after successfully applied snapshot.", + old_region->getDebugString()); return; } @@ -103,15 +105,20 @@ void KVStore::checkAndApplyPreHandledSnapshot(const RegionPtrWrap & new_region, auto state = getProxyHelper()->getRegionLocalState(overlapped_region.first); if (state.state() != raft_serverpb::PeerState::Tombstone) { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "range of region_id={} is overlapped with region_id={}, state: {}", - region_id, - overlapped_region.first, - state.ShortDebugString()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "range of region_id={} is overlapped with region_id={}, state: {}", + region_id, + overlapped_region.first, + state.ShortDebugString()); } else { - LOG_INFO(log, "range of region_id={} is overlapped with `Tombstone` region_id={}", region_id, overlapped_region.first); + LOG_INFO( + log, + "range of region_id={} is overlapped with `Tombstone` region_id={}", + region_id, + overlapped_region.first); handleDestroy(overlapped_region.first, tmt, task_lock); } } @@ -149,14 +156,19 @@ std::pair getTiFlashReplicaSyncInfo(StorageDeltaMergePtr & dm_stor } template -void KVStore::onSnapshot(const RegionPtrWrap & new_region_wrap, RegionPtr old_region, UInt64 old_region_index, TMTContext & tmt) +void KVStore::onSnapshot( + const RegionPtrWrap & new_region_wrap, + RegionPtr old_region, + UInt64 old_region_index, + TMTContext & tmt) { RegionID region_id = new_region_wrap->id(); auto keyspace_id = new_region_wrap->getKeyspaceID(); { auto table_id = new_region_wrap->getMappedTableID(); - if (auto storage = tmt.getStorages().get(keyspace_id, table_id); storage && storage->engineType() == TiDB::StorageEngine::DT) + if (auto storage = tmt.getStorages().get(keyspace_id, table_id); + storage && storage->engineType() == TiDB::StorageEngine::DT) { try { @@ -178,7 +190,12 @@ void KVStore::onSnapshot(const RegionPtrWrap & new_region_wrap, RegionPtr old_re storage->getRowKeyColumnSize()); if (old_key_range != new_key_range) { - LOG_INFO(log, "clear old range before apply snapshot, region_id={} old_range={} new_range={}", region_id, old_key_range.toDebugString(), new_key_range.toDebugString()); + LOG_INFO( + log, + "clear old range before apply snapshot, region_id={} old_range={} new_range={}", + region_id, + old_key_range.toDebugString(), + new_key_range.toDebugString()); dm_storage->deleteRange(old_key_range, context.getSettingsRef()); // We must flush the deletion to the disk here, because we only flush new range when persisting this region later. dm_storage->flushCache(context, old_key_range, /*try_until_succeed*/ true); @@ -187,17 +204,26 @@ void KVStore::onSnapshot(const RegionPtrWrap & new_region_wrap, RegionPtr old_re if constexpr (std::is_same_v) { // Call `ingestFiles` to delete data for range and ingest external DTFiles. - auto ingested_bytes = dm_storage->ingestFiles(new_key_range, new_region_wrap.external_files, /*clear_data_in_range=*/true, context.getSettingsRef()); + auto ingested_bytes = dm_storage->ingestFiles( + new_key_range, + new_region_wrap.external_files, + /*clear_data_in_range=*/true, + context.getSettingsRef()); if (auto [count, is_syncing] = getTiFlashReplicaSyncInfo(dm_storage); is_syncing) { // For write, 1 RU per KB. Reference: https://docs.pingcap.com/tidb/v7.0/tidb-resource-control // Only calculate RU of one replica. So each replica reports 1/count consumptions. - TiFlashMetrics::instance().addReplicaSyncRU(keyspace_id, std::ceil(static_cast(ingested_bytes) / 1024.0 / count)); + TiFlashMetrics::instance().addReplicaSyncRU( + keyspace_id, + std::ceil(static_cast(ingested_bytes) / 1024.0 / count)); } } else if constexpr (std::is_same_v) { - dm_storage->ingestSegmentsFromCheckpointInfo(new_key_range, new_region_wrap.checkpoint_info, context.getSettingsRef()); + dm_storage->ingestSegmentsFromCheckpointInfo( + new_key_range, + new_region_wrap.checkpoint_info, + context.getSettingsRef()); } else { @@ -299,11 +325,13 @@ std::vector KVStore::preHandleSnapshotToFiles( try { SCOPE_EXIT({ new_region->afterPrehandleSnapshot(); }); - external_files = preHandleSSTsToDTFiles(new_region, snaps, index, term, DM::FileConvertJobType::ApplySnapshot, tmt); + external_files + = preHandleSSTsToDTFiles(new_region, snaps, index, term, DM::FileConvertJobType::ApplySnapshot, tmt); } catch (DB::Exception & e) { - e.addMessage(fmt::format("(while preHandleSnapshot region_id={}, index={}, term={})", new_region->id(), index, term)); + e.addMessage( + fmt::format("(while preHandleSnapshot region_id={}, index={}, term={})", new_region->id(), index, term)); e.rethrow(); } return external_files; @@ -333,7 +361,10 @@ std::vector KVStore::preHandleSSTsToDTFiles( fiu_do_on(FailPoints::force_set_sst_to_dtfile_block_size, { expected_block_size = 3; }); Stopwatch watch; - SCOPE_EXIT({ GET_METRIC(tiflash_raft_command_duration_seconds, type_apply_snapshot_predecode).Observe(watch.elapsedSeconds()); }); + SCOPE_EXIT({ + GET_METRIC(tiflash_raft_command_duration_seconds, type_apply_snapshot_predecode) + .Observe(watch.elapsedSeconds()); + }); std::vector generated_ingest_ids; TableID physical_table_id = InvalidTableID; @@ -360,10 +391,11 @@ std::vector KVStore::preHandleSSTsToDTFiles( Timestamp gc_safepoint = 0; if (auto pd_client = tmt.getPDClient(); !pd_client->isMock()) { - gc_safepoint = PDClientHelper::getGCSafePointWithRetry(pd_client, - keyspace_id, - /* ignore_cache= */ false, - context.getSettingsRef().safe_point_update_interval_seconds); + gc_safepoint = PDClientHelper::getGCSafePointWithRetry( + pd_client, + keyspace_id, + /* ignore_cache= */ false, + context.getSettingsRef().safe_point_update_interval_seconds); } physical_table_id = storage->getTableInfo().id; auto log_prefix = fmt::format("table_id={}", physical_table_id); @@ -381,7 +413,10 @@ std::vector KVStore::preHandleSSTsToDTFiles( force_decode, tmt, expected_block_size); - auto bounded_stream = std::make_shared(sst_stream, ::DB::TiDBPkColumnID, schema_snap); + auto bounded_stream = std::make_shared( + sst_stream, + ::DB::TiDBPkColumnID, + schema_snap); stream = std::make_shared>( log_prefix, bounded_stream, @@ -399,7 +434,12 @@ std::vector KVStore::preHandleSSTsToDTFiles( stream->writeSuffix(); if (stream->isAbort()) { - LOG_INFO(log, "Apply snapshot is aborted, cancelling. region_id={} term={} index={}", region_id, term, index); + LOG_INFO( + log, + "Apply snapshot is aborted, cancelling. region_id={} term={} index={}", + region_id, + term, + index); stream->cancel(); } generated_ingest_ids = stream->outputFiles(); @@ -427,7 +467,11 @@ std::vector KVStore::preHandleSSTsToDTFiles( } // Update schema and try to decode again - LOG_INFO(log, "Decoding Region snapshot data meet error, sync schema and try to decode again {} [error={}]", new_region->toString(true), e.displayText()); + LOG_INFO( + log, + "Decoding Region snapshot data meet error, sync schema and try to decode again {} [error={}]", + new_region->toString(true), + e.displayText()); GET_METRIC(tiflash_schema_trigger_count, type_raft_decode).Increment(); tmt.getSchemaSyncerManager()->syncTableSchema(context, keyspace_id, physical_table_id); // Next time should force_decode @@ -438,7 +482,10 @@ std::vector KVStore::preHandleSSTsToDTFiles( else if (e.code() == ErrorCodes::TABLE_IS_DROPPED) { // We can ignore if storage is dropped. - LOG_INFO(log, "Pre-handle snapshot to DTFiles is ignored because the table is dropped {}", new_region->toString(true)); + LOG_INFO( + log, + "Pre-handle snapshot to DTFiles is ignored because the table is dropped {}", + new_region->toString(true)); break; } else @@ -459,7 +506,9 @@ void KVStore::applyPreHandledSnapshot(const RegionPtrWrap & new_region, TMTConte LOG_INFO(log, "Begin apply snapshot, new_region={}", new_region->toString(true)); Stopwatch watch; - SCOPE_EXIT({ GET_METRIC(tiflash_raft_command_duration_seconds, type_apply_snapshot_flush).Observe(watch.elapsedSeconds()); }); + SCOPE_EXIT({ + GET_METRIC(tiflash_raft_command_duration_seconds, type_apply_snapshot_flush).Observe(watch.elapsedSeconds()); + }); checkAndApplyPreHandledSnapshot(new_region, tmt); @@ -468,15 +517,25 @@ void KVStore::applyPreHandledSnapshot(const RegionPtrWrap & new_region, TMTConte LOG_INFO(log, "Finish apply snapshot, new_region={}", new_region->toString(true)); } -template void KVStore::applyPreHandledSnapshot(const RegionPtrWithSnapshotFiles &, TMTContext &); +template void KVStore::applyPreHandledSnapshot( + const RegionPtrWithSnapshotFiles &, + TMTContext &); template void KVStore::checkAndApplyPreHandledSnapshot(const RegionPtrWithBlock &, TMTContext &); -template void KVStore::checkAndApplyPreHandledSnapshot(const RegionPtrWithSnapshotFiles &, TMTContext &); +template void KVStore::checkAndApplyPreHandledSnapshot( + const RegionPtrWithSnapshotFiles &, + TMTContext &); template void KVStore::onSnapshot(const RegionPtrWithBlock &, RegionPtr, UInt64, TMTContext &); -template void KVStore::onSnapshot(const RegionPtrWithSnapshotFiles &, RegionPtr, UInt64, TMTContext &); +template void KVStore::onSnapshot( + const RegionPtrWithSnapshotFiles &, + RegionPtr, + UInt64, + TMTContext &); template <> -void KVStore::releasePreHandledSnapshot(const RegionPtrWithSnapshotFiles & s, TMTContext & tmt) +void KVStore::releasePreHandledSnapshot( + const RegionPtrWithSnapshotFiles & s, + TMTContext & tmt) { auto & storages = tmt.getStorages(); auto keyspace_id = s.base->getKeyspaceID(); @@ -487,7 +546,13 @@ void KVStore::releasePreHandledSnapshot(const Region return; } auto dm_storage = std::dynamic_pointer_cast(storage); - LOG_INFO(log, "Release prehandled snapshot, clean {} dmfiles, region_id={} keyspace={} table_id={}", s.external_files.size(), s.base->id(), keyspace_id, table_id); + LOG_INFO( + log, + "Release prehandled snapshot, clean {} dmfiles, region_id={} keyspace={} table_id={}", + s.external_files.size(), + s.base->id(), + keyspace_id, + table_id); auto & context = tmt.getContext(); dm_storage->cleanPreIngestFiles(s.external_files, context.getSettingsRef()); } @@ -560,7 +625,12 @@ void KVStore::handleIngestCheckpoint(RegionPtr region, CheckpointInfoPtr checkpo applyPreHandledSnapshot(RegionPtrWithCheckpointInfo{region, checkpoint_info}, tmt); } -EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) +EngineStoreApplyRes KVStore::handleIngestSST( + UInt64 region_id, + const SSTViewVec snaps, + UInt64 index, + UInt64 term, + TMTContext & tmt) { auto region_task_lock = region_manager.genRegionTaskLock(region_id); @@ -570,7 +640,12 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec const RegionPtr region = getRegion(region_id); if (region == nullptr) { - LOG_WARNING(log, "region not found, might be removed already, region_id={} term={} index={}", region_id, term, index); + LOG_WARNING( + log, + "region not found, might be removed already, region_id={} term={} index={}", + region_id, + term, + index); return EngineStoreApplyRes::NotFound; } @@ -614,7 +689,12 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec } } -RegionPtr KVStore::handleIngestSSTByDTFile(const RegionPtr & region, const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) +RegionPtr KVStore::handleIngestSSTByDTFile( + const RegionPtr & region, + const SSTViewVec snaps, + UInt64 index, + UInt64 term, + TMTContext & tmt) { if (index <= region->appliedIndex()) return nullptr; @@ -636,7 +716,8 @@ RegionPtr KVStore::handleIngestSSTByDTFile(const RegionPtr & region, const SSTVi } catch (DB::Exception & e) { - e.addMessage(fmt::format("(while handleIngestSST region_id={} index={} term={})", tmp_region->id(), index, term)); + e.addMessage( + fmt::format("(while handleIngestSST region_id={} index={} term={})", tmp_region->id(), index, term)); e.rethrow(); } @@ -662,7 +743,8 @@ RegionPtr KVStore::handleIngestSSTByDTFile(const RegionPtr & region, const SSTVi // Call `ingestFiles` to ingest external DTFiles. // Note that ingest sst won't remove the data in the key range auto dm_storage = std::dynamic_pointer_cast(storage); - dm_storage->ingestFiles(key_range, external_files, /*clear_data_in_range=*/false, context.getSettingsRef()); + dm_storage + ->ingestFiles(key_range, external_files, /*clear_data_in_range=*/false, context.getSettingsRef()); } catch (DB::Exception & e) { diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index cb13ca0379b..2eab827db05 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -50,9 +50,10 @@ extern const char pause_passive_flush_before_persist_region[]; } // namespace FailPoints KVStore::KVStore(Context & context) - : region_persister(context.getSharedContextDisagg()->isDisaggregatedComputeMode() - ? nullptr - : std::make_unique(context, region_manager)) + : region_persister( + context.getSharedContextDisagg()->isDisaggregatedComputeMode() + ? nullptr + : std::make_unique(context, region_manager)) , raft_cmd_res(std::make_unique()) , log(Logger::get()) , region_compact_log_period(120) @@ -147,7 +148,11 @@ void KVStore::traverseRegions(std::function & callback(region.first, region.second); } -bool KVStore::tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & region, const LoggerPtr & log, bool try_until_succeed) +bool KVStore::tryFlushRegionCacheInStorage( + TMTContext & tmt, + const Region & region, + const LoggerPtr & log, + bool try_until_succeed) { fiu_do_on(FailPoints::force_fail_in_flush_region_data, { return false; }); auto keyspace_id = region.getKeyspaceID(); @@ -155,10 +160,11 @@ bool KVStore::tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & regi auto storage = tmt.getStorages().get(keyspace_id, table_id); if (unlikely(storage == nullptr)) { - LOG_WARNING(log, - "tryFlushRegionCacheInStorage can not get table, region {} table_id={}, ignored", - region.toString(), - table_id); + LOG_WARNING( + log, + "tryFlushRegionCacheInStorage can not get table, region {} table_id={}, ignored", + region.toString(), + table_id); return true; } @@ -194,18 +200,28 @@ void KVStore::gcPersistedRegion(Seconds gc_persist_period) if (now < (last_gc_time.load() + gc_persist_period)) return; last_gc_time = now; - RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); + RUNTIME_CHECK_MSG( + region_persister, + "try access to region_persister without initialization, stack={}", + StackTrace().toString()); region_persister->gc(); } -void KVStore::removeRegion(RegionID region_id, bool remove_data, RegionTable & region_table, const KVStoreTaskLock & task_lock, const RegionTaskLock & region_lock) +void KVStore::removeRegion( + RegionID region_id, + bool remove_data, + RegionTable & region_table, + const KVStoreTaskLock & task_lock, + const RegionTaskLock & region_lock) { LOG_INFO(log, "Start to remove region_id={}", region_id); { auto manage_lock = genRegionMgrWriteLock(task_lock); auto it = manage_lock.regions.find(region_id); - manage_lock.index.remove(it->second->makeRaftCommandDelegate(task_lock).getRange().comparableKeys(), region_id); // remove index + manage_lock.index.remove( + it->second->makeRaftCommandDelegate(task_lock).getRange().comparableKeys(), + region_id); // remove index manage_lock.regions.erase(it); } { @@ -216,7 +232,10 @@ void KVStore::removeRegion(RegionID region_id, bool remove_data, RegionTable & r } } - RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); + RUNTIME_CHECK_MSG( + region_persister, + "try access to region_persister without initialization, stack={}", + StackTrace().toString()); region_persister->drop(region_id, region_lock); LOG_INFO(log, "Persisted region_id={} deleted", region_id); @@ -275,7 +294,12 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmdInner( return res; } -EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt) +EngineStoreApplyRes KVStore::handleWriteRaftCmd( + const WriteCmdsView & cmds, + UInt64 region_id, + UInt64 index, + UInt64 term, + TMTContext & tmt) { DM::WriteResult write_result; return handleWriteRaftCmdInner(cmds, region_id, index, term, tmt, write_result); @@ -296,7 +320,12 @@ void KVStore::handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTas } LOG_INFO(log, "Handle destroy {}", region->toString()); region->setPendingRemove(); - removeRegion(region_id, /* remove_data */ true, tmt.getRegionTable(), task_lock, region_manager.genRegionTaskLock(region_id)); + removeRegion( + region_id, + /* remove_data */ true, + tmt.getRegionTable(), + task_lock, + region_manager.genRegionTaskLock(region_id)); } void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes, UInt64 gap) @@ -306,23 +335,29 @@ void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes, U region_compact_log_min_bytes = bytes; region_compact_log_gap = gap; - LOG_INFO( - log, - "threshold config: period {}, rows {}, bytes {}, gap {}", - sec, - rows, - bytes, - gap); + LOG_INFO(log, "threshold config: period {}, rows {}, bytes {}, gap {}", sec, rows, bytes, gap); } -void KVStore::persistRegion(const Region & region, std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg) const +void KVStore::persistRegion( + const Region & region, + std::optional region_task_lock, + PersistRegionReason reason, + const char * extra_msg) const { - RUNTIME_CHECK_MSG(region_persister, "try access to region_persister without initialization, stack={}", StackTrace().toString()); + RUNTIME_CHECK_MSG( + region_persister, + "try access to region_persister without initialization, stack={}", + StackTrace().toString()); if (region_task_lock.has_value()) { auto reason_id = magic_enum::enum_underlying(reason); std::string caller = fmt::format("{} {}", PersistRegionReasonMap[reason_id], extra_msg); - LOG_INFO(log, "Start to persist {}, cache size: {} bytes for `{}`", region.toString(true), region.dataSize(), caller); + LOG_INFO( + log, + "Start to persist {}, cache size: {} bytes for `{}`", + region.toString(true), + region.dataSize(), + caller); region_persister->persist(region, *region_task_lock.value()); LOG_DEBUG(log, "Persist {} done", region.toString(false)); } @@ -367,7 +402,15 @@ bool KVStore::needFlushRegionData(UInt64 region_id, TMTContext & tmt) return canFlushRegionDataImpl(curr_region_ptr, false, false, tmt, region_task_lock, 0, 0, 0, 0); } -bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term, uint64_t truncated_index, uint64_t truncated_term) +bool KVStore::tryFlushRegionData( + UInt64 region_id, + bool force_persist, + bool try_until_succeed, + TMTContext & tmt, + UInt64 index, + UInt64 term, + uint64_t truncated_index, + uint64_t truncated_term) { auto region_task_lock = region_manager.genRegionTaskLock(region_id); const RegionPtr curr_region_ptr = getRegion(region_id); @@ -378,19 +421,38 @@ bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_ /// The triggered CompactLog will be handled by `handleUselessAdminRaftCmd`, /// and result in a `EngineStoreApplyRes::NotFound`. /// Proxy will print this message and continue: `region not found in engine-store, maybe have exec `RemoveNode` first`. - LOG_WARNING(log, "[region_id={} term={} index={}] not exist when flushing, maybe have exec `RemoveNode` first", region_id, term, index); + LOG_WARNING( + log, + "[region_id={} term={} index={}] not exist when flushing, maybe have exec `RemoveNode` first", + region_id, + term, + index); return true; } if (!force_persist) { // try to flush RegionData according to the mem cache rows/bytes/interval - return canFlushRegionDataImpl(curr_region_ptr, true, try_until_succeed, tmt, region_task_lock, index, term, truncated_index, truncated_term); + return canFlushRegionDataImpl( + curr_region_ptr, + true, + try_until_succeed, + tmt, + region_task_lock, + index, + term, + truncated_index, + truncated_term); } // force persist auto & curr_region = *curr_region_ptr; - LOG_DEBUG(log, "flush region due to tryFlushRegionData by force, region_id={} term={} index={}", curr_region.id(), term, index); + LOG_DEBUG( + log, + "flush region due to tryFlushRegionData by force, region_id={} term={} index={}", + curr_region.id(), + term, + index); if (!forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Force flush region failed, region_id={}", region_id); @@ -398,7 +460,16 @@ bool KVStore::tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_ return true; } -bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term, UInt64 truncated_index, UInt64 truncated_term) +bool KVStore::canFlushRegionDataImpl( + const RegionPtr & curr_region_ptr, + UInt8 flush_if_possible, + bool try_until_succeed, + TMTContext & tmt, + const RegionTaskLock & region_task_lock, + UInt64 index, + UInt64 term, + UInt64 truncated_index, + UInt64 truncated_term) { if (curr_region_ptr == nullptr) { @@ -439,13 +510,30 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl GET_METRIC(tiflash_raft_raft_events_count, type_pre_exec_compact).Increment(1); // GET_METRIC(tiflash_raft_raft_log_lag_count, type_compact_index).Observe(current_gap); - LOG_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}, gap {}/{}", curr_region.toString(false), rows, size_bytes, current_applied_gap, gap_threshold); + LOG_DEBUG( + log, + "{} approx mem cache info: rows {}, bytes {}, gap {}/{}", + curr_region.toString(false), + rows, + size_bytes, + current_applied_gap, + gap_threshold); if (can_flush && flush_if_possible) { GET_METRIC(tiflash_raft_raft_events_count, type_exec_compact).Increment(1); // This rarely happens when there are too may raft logs, which don't trigger a proactive flush. - LOG_INFO(log, "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap {}/{}", curr_region.toString(false), index, term, truncated_index, truncated_term, current_applied_gap, gap_threshold); + LOG_INFO( + log, + "{} flush region due to tryFlushRegionData, index {} term {} truncated_index {} truncated_term {} gap " + "{}/{}", + curr_region.toString(false), + index, + term, + truncated_index, + truncated_term, + current_applied_gap, + gap_threshold); GET_METRIC(tiflash_raft_region_flush_size, type_flushed).Observe(size_bytes); return forceFlushRegionDataImpl(curr_region, try_until_succeed, tmt, region_task_lock, index, term); } @@ -457,7 +545,13 @@ bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 fl return can_flush; } -bool KVStore::forceFlushRegionDataImpl(Region & curr_region, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term) const +bool KVStore::forceFlushRegionDataImpl( + Region & curr_region, + bool try_until_succeed, + TMTContext & tmt, + const RegionTaskLock & region_task_lock, + UInt64 index, + UInt64 term) const { Stopwatch watch; if (index) @@ -495,12 +589,13 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( auto & curr_region = *curr_region_ptr; - LOG_DEBUG(log, - "{} handle ignorable admin command {} at [term: {}, index: {}]", - curr_region.toString(false), - raft_cmdpb::AdminCmdType_Name(cmd_type), - term, - index); + LOG_DEBUG( + log, + "{} handle ignorable admin command {} at [term: {}, index: {}]", + curr_region.toString(false), + raft_cmdpb::AdminCmdType_Name(cmd_type), + term, + index); if (curr_region.getClusterRaftstoreVer() == RaftstoreVer::V2) { @@ -518,12 +613,15 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( } curr_region.handleWriteRaftCmd({}, index, term, tmt); - if (cmd_type == raft_cmdpb::AdminCmdType::PrepareFlashback - || cmd_type == raft_cmdpb::AdminCmdType::FinishFlashback + if (cmd_type == raft_cmdpb::AdminCmdType::PrepareFlashback || cmd_type == raft_cmdpb::AdminCmdType::FinishFlashback || cmd_type == raft_cmdpb::AdminCmdType::BatchSwitchWitness) { tryFlushRegionCacheInStorage(tmt, curr_region, log); - persistRegion(curr_region, ®ion_task_lock, PersistRegionReason::UselessAdminCommand, fmt::format("{}", cmd_type).c_str()); + persistRegion( + curr_region, + ®ion_task_lock, + PersistRegionReason::UselessAdminCommand, + fmt::format("{}", cmd_type).c_str()); return EngineStoreApplyRes::Persist; } return EngineStoreApplyRes::None; @@ -538,9 +636,8 @@ EngineStoreApplyRes KVStore::handleAdminRaftCmd( TMTContext & tmt) { Stopwatch watch; - SCOPE_EXIT({ - GET_METRIC(tiflash_raft_apply_write_command_duration_seconds, type_admin).Observe(watch.elapsedSeconds()); - }); + SCOPE_EXIT( + { GET_METRIC(tiflash_raft_apply_write_command_duration_seconds, type_admin).Observe(watch.elapsedSeconds()); }); auto type = request.cmd_type(); switch (request.cmd_type()) { @@ -566,12 +663,13 @@ EngineStoreApplyRes KVStore::handleAdminRaftCmd( const RegionPtr curr_region_ptr = getRegion(curr_region_id); if (curr_region_ptr == nullptr) { - LOG_WARNING(log, - "region not found, might be removed already, region_id={} term={} index={} cmd={}", - curr_region_id, - term, - index, - raft_cmdpb::AdminCmdType_Name(type)); + LOG_WARNING( + log, + "region not found, might be removed already, region_id={} term={} index={} cmd={}", + curr_region_id, + term, + index, + raft_cmdpb::AdminCmdType_Name(type)); return EngineStoreApplyRes::NotFound; } @@ -583,14 +681,8 @@ EngineStoreApplyRes KVStore::handleAdminRaftCmd( curr_region.orphanKeysInfo().advanceAppliedIndex(index); } - curr_region.makeRaftCommandDelegate(task_lock).handleAdminRaftCmd( - request, - response, - index, - term, - *this, - region_table, - *raft_cmd_res); + curr_region.makeRaftCommandDelegate(task_lock) + .handleAdminRaftCmd(request, response, index, term, *this, region_table, *raft_cmd_res); RaftCommandResult & result = *raft_cmd_res; // After region split / merge, try to flush it @@ -699,13 +791,19 @@ EngineStoreApplyRes KVStore::handleAdminRaftCmd( { if (auto source_region = getRegion(request.commit_merge().source().id()); source_region) { - LOG_WARNING(log, - "Admin cmd {} has been applied, try to remove source {}", - raft_cmdpb::AdminCmdType_Name(type), - source_region->toString(false)); + LOG_WARNING( + log, + "Admin cmd {} has been applied, try to remove source {}", + raft_cmdpb::AdminCmdType_Name(type), + source_region->toString(false)); source_region->setPendingRemove(); // `source_region` is merged, don't remove its data in storage. - removeRegion(source_region->id(), /* remove_data */ false, region_table, task_lock, region_manager.genRegionTaskLock(source_region->id())); + removeRegion( + source_region->id(), + /* remove_data */ false, + region_table, + task_lock, + region_manager.genRegionTaskLock(source_region->id())); } } break; @@ -739,18 +837,20 @@ void WaitCheckRegionReady( constexpr double batch_read_index_time_rate = 0.2; // part of time for waiting shall be assigned to batch-read-index auto log = Logger::get(__FUNCTION__); - LOG_INFO(log, - "start to check regions ready, min-wait-tick {}s, max-wait-tick {}s, wait-region-ready-timeout {:.3f}s", - wait_tick_time, - max_wait_tick_time, - get_wait_region_ready_timeout_sec); + LOG_INFO( + log, + "start to check regions ready, min-wait-tick {}s, max-wait-tick {}s, wait-region-ready-timeout {:.3f}s", + wait_tick_time, + max_wait_tick_time, + get_wait_region_ready_timeout_sec); std::unordered_set remain_regions; std::unordered_map regions_to_check; Stopwatch region_check_watch; size_t total_regions_cnt = 0; { - kvstore.traverseRegions([&remain_regions](RegionID region_id, const RegionPtr &) { remain_regions.emplace(region_id); }); + kvstore.traverseRegions( + [&remain_regions](RegionID region_id, const RegionPtr &) { remain_regions.emplace(region_id); }); total_regions_cnt = remain_regions.size(); } while (region_check_watch.elapsedSeconds() < get_wait_region_ready_timeout_sec * batch_read_index_time_rate @@ -779,11 +879,12 @@ void WaitCheckRegionReady( const auto & region_error = resp.region_error(); if (region_error.has_region_not_found() || region_error.has_epoch_not_match()) need_retry = false; - LOG_DEBUG(log, - "neglect error region_id={} not found {} epoch not match {}", - region_id, - region_error.has_region_not_found(), - region_error.has_epoch_not_match()); + LOG_DEBUG( + log, + "neglect error region_id={} not found {} epoch not match {}", + region_id, + region_error.has_region_not_found(), + region_error.has_epoch_not_match()); } if (!need_retry) { @@ -801,10 +902,11 @@ void WaitCheckRegionReady( if (remain_regions.empty()) break; - LOG_INFO(log, - "{} regions need to fetch latest commit-index in next round, sleep for {:.3f}s", - remain_regions.size(), - wait_tick_time); + LOG_INFO( + log, + "{} regions need to fetch latest commit-index in next round, sleep for {:.3f}s", + remain_regions.size(), + wait_tick_time); std::this_thread::sleep_for(std::chrono::milliseconds(static_cast(wait_tick_time * 1000))); wait_tick_time = std::min(max_wait_tick_time, wait_tick_time * 2); } @@ -848,10 +950,11 @@ void WaitCheckRegionReady( if (regions_to_check.empty()) break; - LOG_INFO(log, - "{} regions need to apply to latest index, sleep for {:.3f}s", - regions_to_check.size(), - wait_tick_time); + LOG_INFO( + log, + "{} regions need to apply to latest index, sleep for {:.3f}s", + regions_to_check.size(), + wait_tick_time); std::this_thread::sleep_for(std::chrono::milliseconds(static_cast(wait_tick_time * 1000))); wait_tick_time = std::min(max_wait_tick_time, wait_tick_time * 2); } while (region_check_watch.elapsedSeconds() < get_wait_region_ready_timeout_sec @@ -874,23 +977,39 @@ void WaitCheckRegionReady( } }, " "); - LOG_WARNING(log, "{} regions CANNOT catch up with latest index, (region-id,latest-index,apply-index): {}", regions_to_check.size(), buffer.toString()); + LOG_WARNING( + log, + "{} regions CANNOT catch up with latest index, (region-id,latest-index,apply-index): {}", + regions_to_check.size(), + buffer.toString()); } - LOG_INFO(log, - "finish to check {} regions, time cost {:.3f}s", - total_regions_cnt, - region_check_watch.elapsedSeconds()); + LOG_INFO( + log, + "finish to check {} regions, time cost {:.3f}s", + total_regions_cnt, + region_check_watch.elapsedSeconds()); } -void WaitCheckRegionReady(const TMTContext & tmt, KVStore & kvstore, const std::atomic_size_t & terminate_signals_counter) +void WaitCheckRegionReady( + const TMTContext & tmt, + KVStore & kvstore, + const std::atomic_size_t & terminate_signals_counter) { // wait interval to check region ready, not recommended to modify only if for tesing auto wait_region_ready_tick = tmt.getContext().getConfigRef().getUInt64("flash.wait_region_ready_tick", 0); auto wait_region_ready_timeout_sec = static_cast(tmt.waitRegionReadyTimeout()); const double max_wait_tick_time = 0 == wait_region_ready_tick ? 20.0 : wait_region_ready_timeout_sec; - double min_wait_tick_time = 0 == wait_region_ready_tick ? 2.5 : static_cast(wait_region_ready_tick); // default tick in TiKV is about 2s (without hibernate-region) - return WaitCheckRegionReady(tmt, kvstore, terminate_signals_counter, min_wait_tick_time, max_wait_tick_time, wait_region_ready_timeout_sec); + double min_wait_tick_time = 0 == wait_region_ready_tick + ? 2.5 + : static_cast(wait_region_ready_tick); // default tick in TiKV is about 2s (without hibernate-region) + return WaitCheckRegionReady( + tmt, + kvstore, + terminate_signals_counter, + min_wait_tick_time, + max_wait_tick_time, + wait_region_ready_timeout_sec); } void KVStore::setStore(metapb::Store store_) diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 4912bd65da5..473594b86f6 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -93,15 +93,15 @@ enum class PersistRegionReason IngestSst }; -constexpr const char * PersistRegionReasonMap[magic_enum::enum_count()] = { - "debug", - "admin cmd useless", - "admin raft cmd", - "tryFlushRegionData", - "ProactiveFlush", - "save previous region before apply", - "save current region after apply", - "ingestsst"}; +constexpr const char * PersistRegionReasonMap[magic_enum::enum_count()] + = {"debug", + "admin cmd useless", + "admin raft cmd", + "tryFlushRegionData", + "ProactiveFlush", + "save previous region before apply", + "save current region after apply", + "ingestsst"}; static_assert(magic_enum::enum_count() == sizeof(PersistRegionReasonMap) / sizeof(const char *)); @@ -122,7 +122,11 @@ class KVStore final : private boost::noncopyable void gcPersistedRegion(Seconds gc_persist_period = Seconds(60 * 5)); - static bool tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & region, const LoggerPtr & log, bool try_until_succeed = true); + static bool tryFlushRegionCacheInStorage( + TMTContext & tmt, + const Region & region, + const LoggerPtr & log, + bool try_until_succeed = true); size_t regionSize() const; EngineStoreApplyRes handleAdminRaftCmd( @@ -147,12 +151,27 @@ class KVStore final : private boost::noncopyable DM::WriteResult & write_result); bool needFlushRegionData(UInt64 region_id, TMTContext & tmt); - bool tryFlushRegionData(UInt64 region_id, bool force_persist, bool try_until_succeed, TMTContext & tmt, UInt64 index, UInt64 term, uint64_t truncated_index, uint64_t truncated_term); + bool tryFlushRegionData( + UInt64 region_id, + bool force_persist, + bool try_until_succeed, + TMTContext & tmt, + UInt64 index, + UInt64 term, + uint64_t truncated_index, + uint64_t truncated_term); /** * Only used in tests. In production we will call preHandleSnapshotToFiles + applyPreHandledSnapshot. */ - void handleApplySnapshot(metapb::Region && region, uint64_t peer_id, SSTViewVec, uint64_t index, uint64_t term, std::optional, TMTContext & tmt); + void handleApplySnapshot( + metapb::Region && region, + uint64_t peer_id, + SSTViewVec, + uint64_t index, + uint64_t term, + std::optional, + TMTContext & tmt); void handleIngestCheckpoint(RegionPtr region, CheckpointInfoPtr checkpoint_info, TMTContext & tmt); @@ -215,7 +234,12 @@ class KVStore final : private boost::noncopyable // TODO(proactive flush) // void proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRange & rowkey_range, KeyspaceID keyspace_id, TableID table_id, bool is_background); - void notifyCompactLog(RegionID region_id, UInt64 compact_index, UInt64 compact_term, bool is_background, bool lock_held = true); + void notifyCompactLog( + RegionID region_id, + UInt64 compact_index, + UInt64 compact_term, + bool is_background, + bool lock_held = true); #ifndef DBMS_PUBLIC_GTEST private: #endif @@ -225,7 +249,14 @@ class KVStore final : private boost::noncopyable friend struct MockRaftCommand; friend class RegionMockTest; friend class NaturalDag; - friend void RegionBench::concurrentBatchInsert(const TiDB::TableInfo &, Int64, Int64, Int64, UInt64, UInt64, Context &); + friend void RegionBench::concurrentBatchInsert( + const TiDB::TableInfo &, + Int64, + Int64, + Int64, + UInt64, + UInt64, + Context &); using DBGInvokerPrinter = std::function; friend void dbgFuncRemoveRegion(Context &, const ASTs &, DBGInvokerPrinter); friend void dbgFuncPutRegion(Context &, const ASTs &, DBGInvokerPrinter); @@ -258,16 +289,22 @@ class KVStore final : private boost::noncopyable template void onSnapshot(const RegionPtrWrap &, RegionPtr old_region, UInt64 old_region_index, TMTContext & tmt); - RegionPtr handleIngestSSTByDTFile(const RegionPtr & region, const SSTViewVec, UInt64 index, UInt64 term, TMTContext & tmt); + RegionPtr handleIngestSSTByDTFile( + const RegionPtr & region, + const SSTViewVec, + UInt64 index, + UInt64 term, + TMTContext & tmt); // Remove region from this TiFlash node. // If region is destroy or moved to another node(change peer), // set `remove_data` true to remove obsolete data from storage. - void removeRegion(RegionID region_id, - bool remove_data, - RegionTable & region_table, - const KVStoreTaskLock & task_lock, - const RegionTaskLock & region_lock); + void removeRegion( + RegionID region_id, + bool remove_data, + RegionTable & region_table, + const KVStoreTaskLock & task_lock, + const RegionTaskLock & region_lock); void mockRemoveRegion(RegionID region_id, RegionTable & region_table); KVStoreTaskLock genTaskLock() const; @@ -285,10 +322,29 @@ class KVStore final : private boost::noncopyable /// Notice that if flush_if_possible is set to false, we only check if a flush is allowed by rowsize/size/interval. /// It will not check if a flush will eventually succeed. /// In other words, `canFlushRegionDataImpl(flush_if_possible=true)` can return false. - bool canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term, UInt64 truncated_index, UInt64 truncated_term); - bool forceFlushRegionDataImpl(Region & curr_region, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock, UInt64 index, UInt64 term) const; + bool canFlushRegionDataImpl( + const RegionPtr & curr_region_ptr, + UInt8 flush_if_possible, + bool try_until_succeed, + TMTContext & tmt, + const RegionTaskLock & region_task_lock, + UInt64 index, + UInt64 term, + UInt64 truncated_index, + UInt64 truncated_term); + bool forceFlushRegionDataImpl( + Region & curr_region, + bool try_until_succeed, + TMTContext & tmt, + const RegionTaskLock & region_task_lock, + UInt64 index, + UInt64 term) const; - void persistRegion(const Region & region, std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg) const; + void persistRegion( + const Region & region, + std::optional region_task_lock, + PersistRegionReason reason, + const char * extra_msg) const; void releaseReadIndexWorkers(); void handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTaskLock &); diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index f8f740d10a9..c4f37bcac54 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -98,7 +98,12 @@ static DM::WriteResult writeRegionDataToStorage( auto schema_version = storage->getTableInfo().schema_version; std::stringstream ss; region.pre_decode_cache->toString(ss); - LOG_DEBUG(log, "{} got pre-decode cache {}, storage schema version: {}", region->toString(), ss.str(), schema_version); + LOG_DEBUG( + log, + "{} got pre-decode cache {}, storage schema version: {}", + region->toString(), + ss.str(), + schema_version); if (region.pre_decode_cache->schema_version == schema_version) { @@ -128,7 +133,8 @@ static DM::WriteResult writeRegionDataToStorage( if (!reader.read(*block_ptr, data_list_read, force_decode)) return false; region_decode_cost = watch.elapsedMilliseconds(); - GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_decode).Observe(region_decode_cost / 1000.0); + GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_decode) + .Observe(region_decode_cost / 1000.0); } /// Write block into storage. @@ -153,7 +159,10 @@ static DM::WriteResult writeRegionDataToStorage( break; } default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown StorageEngine: {}", static_cast(storage->engineType())); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unknown StorageEngine: {}", + static_cast(storage->engineType())); } write_part_cost = watch.elapsedMilliseconds(); @@ -161,7 +170,14 @@ static DM::WriteResult writeRegionDataToStorage( if (need_decode) storage->releaseDecodingBlock(block_decoding_schema_epoch, std::move(block_ptr)); - LOG_TRACE(log, "keyspace={} table_id={} region_id={} cost [region decode {}, write part {}] ms", keyspace_id, table_id, region->id(), region_decode_cost, write_part_cost); + LOG_TRACE( + log, + "keyspace={} table_id={} region_id={} cost [region decode {}, write part {}] ms", + keyspace_id, + table_id, + region->id(), + region_decode_cost, + write_part_cost); return true; }; @@ -197,7 +213,12 @@ static DM::WriteResult writeRegionDataToStorage( { // Failure won't be tolerated this time. // TODO: Enrich exception message. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Write region failed! region_id={} keyspace={} table_id={}", region->id(), keyspace_id, table_id); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Write region failed! region_id={} keyspace={} table_id={}", + region->id(), + keyspace_id, + table_id); } return write_result; } @@ -236,18 +257,21 @@ std::variantrawKeys().first, mapped_table_id) || mapped_table_id != table_id) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Should not happen, region not belong to table, table_id={} expect_table_id={}", - mapped_table_id, - table_id); + if (!computeMappedTableID(*meta_snap.range->rawKeys().first, mapped_table_id) + || mapped_table_id != table_id) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Should not happen, region not belong to table, table_id={} expect_table_id={}", + mapped_table_id, + table_id); } /// Deal with locks. if (resolve_locks) { /// Check if there are any lock should be resolved, if so, throw LockException. - lock_value = scanner.getLockInfo(RegionLockReadQuery{.read_tso = start_ts, .bypass_lock_ts = bypass_lock_ts}); + lock_value + = scanner.getLockInfo(RegionLockReadQuery{.read_tso = start_ts, .bypass_lock_ts = bypass_lock_ts}); } /// If there is no lock, leave scope of region scanner and raise LockException. @@ -331,7 +355,9 @@ static inline void reportUpstreamLatency(const RegionDataReadInfoList & data_lis auto ts = std::get<2>(data_list_read.front()); auto [physical_ms, logical] = parseTS(ts); std::ignore = logical; - UInt64 curr_ms = std::chrono::time_point_cast(std::chrono::system_clock::now()).time_since_epoch().count(); + UInt64 curr_ms = std::chrono::time_point_cast(std::chrono::system_clock::now()) + .time_since_epoch() + .count(); if (likely(curr_ms > physical_ms)) { auto latency_ms = curr_ms - physical_ms; @@ -370,37 +396,40 @@ DM::WriteResult RegionTable::writeBlockByRegion( return write_result; } -RegionTable::ResolveLocksAndWriteRegionRes RegionTable::resolveLocksAndWriteRegion(TMTContext & tmt, - const TiDB::TableID table_id, - const RegionPtr & region, - const Timestamp start_ts, - const std::unordered_set * bypass_lock_ts, - RegionVersion region_version, - RegionVersion conf_version, - const LoggerPtr & log) +RegionTable::ResolveLocksAndWriteRegionRes RegionTable::resolveLocksAndWriteRegion( + TMTContext & tmt, + const TiDB::TableID table_id, + const RegionPtr & region, + const Timestamp start_ts, + const std::unordered_set * bypass_lock_ts, + RegionVersion region_version, + RegionVersion conf_version, + const LoggerPtr & log) { - auto region_data_lock = resolveLocksAndReadRegionData(table_id, - region, - start_ts, - bypass_lock_ts, - region_version, - conf_version, - /* resolve_locks */ true, - /* need_data_value */ true); - - return std::visit(variant_op::overloaded{ - [&](RegionDataReadInfoList & data_list_read) -> ResolveLocksAndWriteRegionRes { - if (data_list_read.empty()) - return RegionException::RegionReadStatus::OK; - auto & context = tmt.getContext(); - // There is no raft input here, so we can just ignore the fg flush request. - writeRegionDataToStorage(context, region, data_list_read, log); - RemoveRegionCommitCache(region, data_list_read); - return RegionException::RegionReadStatus::OK; - }, - [](auto & r) -> ResolveLocksAndWriteRegionRes { return std::move(r); }, - }, - region_data_lock); + auto region_data_lock = resolveLocksAndReadRegionData( + table_id, + region, + start_ts, + bypass_lock_ts, + region_version, + conf_version, + /* resolve_locks */ true, + /* need_data_value */ true); + + return std::visit( + variant_op::overloaded{ + [&](RegionDataReadInfoList & data_list_read) -> ResolveLocksAndWriteRegionRes { + if (data_list_read.empty()) + return RegionException::RegionReadStatus::OK; + auto & context = tmt.getContext(); + // There is no raft input here, so we can just ignore the fg flush request. + writeRegionDataToStorage(context, region, data_list_read, log); + RemoveRegionCommitCache(region, data_list_read); + return RegionException::RegionReadStatus::OK; + }, + [](auto & r) -> ResolveLocksAndWriteRegionRes { return std::move(r); }, + }, + region_data_lock); } std::tuple, DecodingStorageSchemaSnapshotConstPtr> // @@ -439,10 +468,20 @@ AtomicGetStorageSchema(const RegionPtr & region, TMTContext & tmt) Stopwatch watch; tmt.getSchemaSyncerManager()->syncTableSchema(context, keyspace_id, table_id); auto schema_sync_cost = watch.elapsedMilliseconds(); - LOG_INFO(Logger::get("AtomicGetStorageSchema"), "sync schema cost {} ms, keyspace={} table_id={}", schema_sync_cost, keyspace_id, table_id); + LOG_INFO( + Logger::get("AtomicGetStorageSchema"), + "sync schema cost {} ms, keyspace={} table_id={}", + schema_sync_cost, + keyspace_id, + table_id); if (!atomic_get(true)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "AtomicGetStorageSchema failed, region={} keyspace={} table_id={}", region->toString(), keyspace_id, table_id); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "AtomicGetStorageSchema failed, region={} keyspace={} table_id={}", + region->toString(), + keyspace_id, + table_id); } return {std::move(drop_lock), std::move(dm_storage), std::move(schema_snapshot)}; @@ -454,8 +493,9 @@ static Block sortColumnsBySchemaSnap(Block && ori, const DM::ColumnDefines & sch // Some trival check to ensure the input is legal if (ori.columns() != schema.size()) { - throw Exception("Try to sortColumnsBySchemaSnap with different column size [block_columns=" + DB::toString(ori.columns()) - + "] [schema_columns=" + DB::toString(schema.size()) + "]"); + throw Exception( + "Try to sortColumnsBySchemaSnap with different column size [block_columns=" + DB::toString(ori.columns()) + + "] [schema_columns=" + DB::toString(schema.size()) + "]"); } #endif @@ -481,16 +521,18 @@ static Block sortColumnsBySchemaSnap(Block && ori, const DM::ColumnDefines & sch /// Decode region data into block and belonging schema snapshot, remove committed data from `region` /// The return value is a block that store the committed data scanned and removed from `region`. /// The columns of returned block is sorted by `schema_snap`. -Block GenRegionBlockDataWithSchema(const RegionPtr & region, // - const DecodingStorageSchemaSnapshotConstPtr & schema_snap, - Timestamp gc_safepoint, - bool force_decode, - TMTContext & /* */) +Block GenRegionBlockDataWithSchema( + const RegionPtr & region, // + const DecodingStorageSchemaSnapshotConstPtr & schema_snap, + Timestamp gc_safepoint, + bool force_decode, + TMTContext & /* */) { // In 5.0.1, feature `compaction filter` is enabled by default. Under such feature tikv will do gc in write & default cf individually. // If some rows were updated and add tiflash replica, tiflash store may receive region snapshot with unmatched data in write & default cf sst files. - fiu_do_on(FailPoints::force_set_safepoint_when_decode_block, - { gc_safepoint = 10000000; }); // Mock a GC safepoint for testing compaction filter + fiu_do_on(FailPoints::force_set_safepoint_when_decode_block, { + gc_safepoint = 10000000; + }); // Mock a GC safepoint for testing compaction filter region->tryCompactionFilter(gc_safepoint); std::optional data_list_read = ReadRegionCommitCache(region, true); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index c34a5a8be84..3152eeca122 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -73,7 +73,9 @@ const std::string & CFToName(const ColumnFamilyType type) case ColumnFamilyType::Lock: return ColumnFamilyName::Lock; default: - throw Exception("Can not tell cf type " + std::to_string(static_cast(type)), ErrorCodes::LOGICAL_ERROR); + throw Exception( + "Can not tell cf type " + std::to_string(static_cast(type)), + ErrorCodes::LOGICAL_ERROR); } } @@ -87,14 +89,12 @@ static_assert(alignof(EngineStoreServerHelper) == alignof(RawVoidPtr)); static_assert(sizeof(RaftStoreProxyPtr) == sizeof(ConstRawVoidPtr)); static_assert(alignof(RaftStoreProxyPtr) == alignof(ConstRawVoidPtr)); -EngineStoreApplyRes HandleWriteRaftCmd( - const EngineStoreServerWrap * server, - WriteCmdsView cmds, - RaftCmdHeader header) +EngineStoreApplyRes HandleWriteRaftCmd(const EngineStoreServerWrap * server, WriteCmdsView cmds, RaftCmdHeader header) { try { - return server->tmt->getKVStore()->handleWriteRaftCmd(cmds, header.region_id, header.index, header.term, *server->tmt); + return server->tmt->getKVStore() + ->handleWriteRaftCmd(cmds, header.region_id, header.index, header.term, *server->tmt); } catch (...) { @@ -145,12 +145,27 @@ uint8_t NeedFlushData(EngineStoreServerWrap * server, uint64_t region_id) } } -uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t flush_pattern, uint64_t index, uint64_t term, uint64_t truncated_index, uint64_t truncated_term) +uint8_t TryFlushData( + EngineStoreServerWrap * server, + uint64_t region_id, + uint8_t flush_pattern, + uint64_t index, + uint64_t term, + uint64_t truncated_index, + uint64_t truncated_term) { try { auto & kvstore = server->tmt->getKVStore(); - return kvstore->tryFlushRegionData(region_id, false, flush_pattern, *server->tmt, index, term, truncated_index, truncated_term); + return kvstore->tryFlushRegionData( + region_id, + false, + flush_pattern, + *server->tmt, + index, + term, + truncated_index, + truncated_term); } catch (...) { @@ -179,7 +194,9 @@ void WriteBatchPutPage(RawVoidPtr ptr, BaseBuffView page_id, BaseBuffView value) { try { - LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI write page {}", UniversalPageId(page_id.data, page_id.len))); + LOG_TRACE( + &Poco::Logger::get("ProxyFFI"), + fmt::format("FFI write page {}", UniversalPageId(page_id.data, page_id.len))); auto * wb = reinterpret_cast(ptr); MemoryWriteBuffer buf(0, value.len); buf.write(value.data, value.len); @@ -198,7 +215,9 @@ void WriteBatchDelPage(RawVoidPtr ptr, BaseBuffView page_id) { try { - LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI delete page {}", UniversalPageId(page_id.data, page_id.len))); + LOG_TRACE( + &Poco::Logger::get("ProxyFFI"), + fmt::format("FFI delete page {}", UniversalPageId(page_id.data, page_id.len))); auto * wb = reinterpret_cast(ptr); wb->delPage(UniversalPageId(page_id.data, page_id.len)); } @@ -292,12 +311,18 @@ CppStrWithView HandleReadPage(const EngineStoreServerWrap * server, BaseBuffView auto * page = new Page(reader.read(UniversalPageId(page_id.data, page_id.len))); if (page->isValid()) { - LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI read page {} success", UniversalPageId(page_id.data, page_id.len))); - return CppStrWithView{.inner = GenRawCppPtr(page, RawCppPtrTypeImpl::UniversalPage), .view = BaseBuffView{page->data.begin(), page->data.size()}}; + LOG_TRACE( + &Poco::Logger::get("ProxyFFI"), + fmt::format("FFI read page {} success", UniversalPageId(page_id.data, page_id.len))); + return CppStrWithView{ + .inner = GenRawCppPtr(page, RawCppPtrTypeImpl::UniversalPage), + .view = BaseBuffView{page->data.begin(), page->data.size()}}; } else { - LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI read page {} fail", UniversalPageId(page_id.data, page_id.len))); + LOG_TRACE( + &Poco::Logger::get("ProxyFFI"), + fmt::format("FFI read page {} fail", UniversalPageId(page_id.data, page_id.len))); return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}; } } @@ -312,7 +337,12 @@ RawCppPtrCarr HandleScanPage(const EngineStoreServerWrap * server, BaseBuffView { try { - LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI scan page from {} to {}", UniversalPageId(start_page_id.data, start_page_id.len), UniversalPageId(end_page_id.data, end_page_id.len))); + LOG_TRACE( + &Poco::Logger::get("ProxyFFI"), + fmt::format( + "FFI scan page from {} to {}", + UniversalPageId(start_page_id.data, start_page_id.len), + UniversalPageId(end_page_id.data, end_page_id.len))); auto uni_ps = server->tmt->getContext().getWriteNodePageStorage(); RaftDataReader reader(*uni_ps); std::vector page_ids; @@ -325,7 +355,8 @@ RawCppPtrCarr HandleScanPage(const EngineStoreServerWrap * server, BaseBuffView UniversalPageId(start_page_id.data, start_page_id.len), UniversalPageId(end_page_id.data, end_page_id.len), checker); - auto * data = static_cast(malloc(pages.size() * sizeof(PageAndCppStrWithView))); // NOLINT(cppcoreguidelines-no-malloc) + auto * data = static_cast( + malloc(pages.size() * sizeof(PageAndCppStrWithView))); // NOLINT(cppcoreguidelines-no-malloc) for (size_t i = 0; i < pages.size(); i++) { auto * target = data + i; @@ -336,7 +367,10 @@ RawCppPtrCarr HandleScanPage(const EngineStoreServerWrap * server, BaseBuffView .page_view = BaseBuffView{.data = pages[i]->data.begin(), .len = pages[i]->data.size()}, .key_view = BaseBuffView{.data = key_str->data(), .len = key_str->size()}}; } - return RawCppPtrCarr{.inner = data, .len = pages.size(), .type = static_cast(RawCppPtrTypeImpl::PageAndCppStr)}; + return RawCppPtrCarr{ + .inner = data, + .len = pages.size(), + .type = static_cast(RawCppPtrTypeImpl::PageAndCppStr)}; } catch (...) { @@ -354,13 +388,23 @@ CppStrWithView HandleGetLowerBound(const EngineStoreServerWrap * server, BaseBuf auto page_id_opt = reader.getLowerBound(UniversalPageId(raw_page_id.data, raw_page_id.len)); if (page_id_opt.has_value()) { - LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI get lower bound for page {} success", UniversalPageId(raw_page_id.data, raw_page_id.len))); + LOG_TRACE( + &Poco::Logger::get("ProxyFFI"), + fmt::format( + "FFI get lower bound for page {} success", + UniversalPageId(raw_page_id.data, raw_page_id.len))); auto * s = RawCppString::New(page_id_opt->asStr()); - return CppStrWithView{.inner = GenRawCppPtr(s, RawCppPtrTypeImpl::String), .view = BaseBuffView{s->data(), s->size()}}; + return CppStrWithView{ + .inner = GenRawCppPtr(s, RawCppPtrTypeImpl::String), + .view = BaseBuffView{s->data(), s->size()}}; } else { - LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI get lower bound for page {} fail", UniversalPageId(raw_page_id.data, raw_page_id.len))); + LOG_TRACE( + &Poco::Logger::get("ProxyFFI"), + fmt::format( + "FFI get lower bound for page {} fail", + UniversalPageId(raw_page_id.data, raw_page_id.len))); return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}; } } @@ -404,17 +448,11 @@ static_assert(alignof(RaftStoreProxyFFIHelper) == alignof(TiFlashRaftProxyHelper struct RustGcHelper : public ext::Singleton { - void gcRustPtr(RawVoidPtr ptr, RawRustPtrType type) const - { - fn_gc_rust_ptr(ptr, type); - } + void gcRustPtr(RawVoidPtr ptr, RawRustPtrType type) const { fn_gc_rust_ptr(ptr, type); } RustGcHelper() = default; - void setRustPtrGcFn(void (*fn_gc_rust_ptr)(RawVoidPtr, RawRustPtrType)) - { - this->fn_gc_rust_ptr = fn_gc_rust_ptr; - } + void setRustPtrGcFn(void (*fn_gc_rust_ptr)(RawVoidPtr, RawRustPtrType)) { this->fn_gc_rust_ptr = fn_gc_rust_ptr; } private: void (*fn_gc_rust_ptr)(RawVoidPtr, RawRustPtrType); @@ -549,7 +587,9 @@ void InsertBatchReadIndexResp(RawVoidPtr resp, BaseBuffView view, uint64_t regio reinterpret_cast(resp)->emplace_back(std::move(res), region_id); } -BatchReadIndexRes TiFlashRaftProxyHelper::batchReadIndex_v1(const std::vector & req, uint64_t timeout_ms) const +BatchReadIndexRes TiFlashRaftProxyHelper::batchReadIndex_v1( + const std::vector & req, + uint64_t timeout_ms) const { std::vector req_strs; req_strs.reserve(req.size()); @@ -567,8 +607,7 @@ BatchReadIndexRes TiFlashRaftProxyHelper::batchReadIndex_v1(const std::vectortmt->getKVStore(); if constexpr (std::is_same_v) { - kvstore->applyPreHandledSnapshot(RegionPtrWithSnapshotFiles{snap->region, std::move(snap->external_files)}, *server->tmt); + kvstore->applyPreHandledSnapshot( + RegionPtrWithSnapshotFiles{snap->region, std::move(snap->external_files)}, + *server->tmt); } } catch (...) @@ -776,7 +817,10 @@ void GcSpecialRawCppPtr(void * ptr, uint64_t hint_size, SpecialCppPtrType type) break; } default: - LOG_ERROR(&Poco::Logger::get(__FUNCTION__), "unknown type {}", static_cast>(type)); + LOG_ERROR( + &Poco::Logger::get(__FUNCTION__), + "unknown type {}", + static_cast>(type)); exit(-1); } } @@ -809,8 +853,7 @@ CppStrWithView GetConfig(EngineStoreServerWrap * server, [[maybe_unused]] uint8_ std::ifstream stream(config_file_path); if (!stream) return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}; - auto * s = RawCppString::New((std::istreambuf_iterator(stream)), - std::istreambuf_iterator()); + auto * s = RawCppString::New((std::istreambuf_iterator(stream)), std::istreambuf_iterator()); stream.close(); /** the returned str must be formated as TOML, proxy will parse and show in form of JASON. * curl `http://{status-addr}/config`, got: @@ -818,7 +861,9 @@ CppStrWithView GetConfig(EngineStoreServerWrap * server, [[maybe_unused]] uint8_ * * if proxy can NOT parse it, return 500 Internal Server Error. * */ - return CppStrWithView{.inner = GenRawCppPtr(s, RawCppPtrTypeImpl::String), .view = BaseBuffView{s->data(), s->size()}}; + return CppStrWithView{ + .inner = GenRawCppPtr(s, RawCppPtrTypeImpl::String), + .view = BaseBuffView{s->data(), s->size()}}; } catch (...) { @@ -864,9 +909,7 @@ raft_serverpb::RegionLocalState TiFlashRaftProxyHelper::getRegionLocalState(uint raft_serverpb::RegionLocalState state; RawCppStringPtr error_msg_ptr{}; - SCOPE_EXIT({ - delete error_msg_ptr; - }); + SCOPE_EXIT({ delete error_msg_ptr; }); auto res = this->fn_get_region_local_state(this->proxy_ptr, region_id, &state, &error_msg_ptr); switch (res) { @@ -874,7 +917,9 @@ raft_serverpb::RegionLocalState TiFlashRaftProxyHelper::getRegionLocalState(uint break; case KVGetStatus::Error: { - throw Exception(fmt::format("{} meet internal error: {}", __FUNCTION__, *error_msg_ptr), ErrorCodes::LOGICAL_ERROR); + throw Exception( + fmt::format("{} meet internal error: {}", __FUNCTION__, *error_msg_ptr), + ErrorCodes::LOGICAL_ERROR); } case KVGetStatus::NotFound: // make not found as `Tombstone` @@ -884,12 +929,20 @@ raft_serverpb::RegionLocalState TiFlashRaftProxyHelper::getRegionLocalState(uint return state; } -void TiFlashRaftProxyHelper::notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term, uint64_t applied_index) const +void TiFlashRaftProxyHelper::notifyCompactLog( + uint64_t region_id, + uint64_t compact_index, + uint64_t compact_term, + uint64_t applied_index) const { this->fn_notify_compact_log(this->proxy_ptr, region_id, compact_index, compact_term, applied_index); } -void HandleSafeTSUpdate(EngineStoreServerWrap * server, uint64_t region_id, uint64_t self_safe_ts, uint64_t leader_safe_ts) +void HandleSafeTSUpdate( + EngineStoreServerWrap * server, + uint64_t region_id, + uint64_t self_safe_ts, + uint64_t leader_safe_ts) { try { diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index 57ed97c8848..dd8a19b2e7d 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -116,7 +116,8 @@ struct TiFlashRaftProxyHelper : RaftStoreProxyFFIHelper TimerTask makeTimerTask(uint64_t time_ms) const; bool pollTimerTask(TimerTask & task, RawVoidPtr waker = nullptr) const; raft_serverpb::RegionLocalState getRegionLocalState(uint64_t region_id) const; - void notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term, uint64_t applied_index) const; + void notifyCompactLog(uint64_t region_id, uint64_t compact_index, uint64_t compact_term, uint64_t applied_index) + const; }; extern "C" { @@ -126,14 +127,19 @@ EngineStoreApplyRes HandleAdminRaftCmd( BaseBuffView req_buff, BaseBuffView resp_buff, RaftCmdHeader header); -EngineStoreApplyRes HandleWriteRaftCmd(const EngineStoreServerWrap * server, - WriteCmdsView cmds, - RaftCmdHeader header); +EngineStoreApplyRes HandleWriteRaftCmd(const EngineStoreServerWrap * server, WriteCmdsView cmds, RaftCmdHeader header); uint8_t NeedFlushData(EngineStoreServerWrap * server, uint64_t region_id); // `flush_pattern` values: // 0: try, but can fail. // 1: try until succeed. -uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t flush_pattern, uint64_t index, uint64_t term, uint64_t truncated_index, uint64_t truncated_term); +uint8_t TryFlushData( + EngineStoreServerWrap * server, + uint64_t region_id, + uint8_t flush_pattern, + uint64_t index, + uint64_t term, + uint64_t truncated_index, + uint64_t truncated_term); RawCppPtr CreateWriteBatch(const EngineStoreServerWrap * dummy); void WriteBatchPutPage(RawVoidPtr ptr, BaseBuffView page_id, BaseBuffView value); void WriteBatchDelPage(RawVoidPtr ptr, BaseBuffView page_id); @@ -143,7 +149,10 @@ void HandleMergeWriteBatch(RawVoidPtr lhs, RawVoidPtr rhs); void HandleClearWriteBatch(RawVoidPtr ptr); void HandleConsumeWriteBatch(const EngineStoreServerWrap * server, RawVoidPtr ptr); CppStrWithView HandleReadPage(const EngineStoreServerWrap * server, BaseBuffView page_id); -RawCppPtrCarr HandleScanPage(const EngineStoreServerWrap * server, BaseBuffView start_page_id, BaseBuffView end_page_id); +RawCppPtrCarr HandleScanPage( + const EngineStoreServerWrap * server, + BaseBuffView start_page_id, + BaseBuffView end_page_id); CppStrWithView HandleGetLowerBound(const EngineStoreServerWrap * server, BaseBuffView raw_page_id); uint8_t IsPSEmpty(const EngineStoreServerWrap * server); void HandlePurgePageStorage(const EngineStoreServerWrap * server); @@ -171,12 +180,15 @@ BaseBuffView strIntoView(const std::string * str_ptr); CppStrWithView GetConfig(EngineStoreServerWrap *, uint8_t full); void SetStore(EngineStoreServerWrap *, BaseBuffView); void SetPBMsByBytes(MsgPBType type, RawVoidPtr ptr, BaseBuffView view); -void HandleSafeTSUpdate(EngineStoreServerWrap * server, uint64_t region_id, uint64_t self_safe_ts, uint64_t leader_safe_ts); +void HandleSafeTSUpdate( + EngineStoreServerWrap * server, + uint64_t region_id, + uint64_t self_safe_ts, + uint64_t leader_safe_ts); FastAddPeerRes FastAddPeer(EngineStoreServerWrap * server, uint64_t region_id, uint64_t new_peer_id); } -inline EngineStoreServerHelper GetEngineStoreServerHelper( - EngineStoreServerWrap * tiflash_instance_wrap) +inline EngineStoreServerHelper GetEngineStoreServerHelper(EngineStoreServerWrap * tiflash_instance_wrap) { return EngineStoreServerHelper{ // a special number, also defined in proxy diff --git a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp index 70da200f4c1..224769e013c 100644 --- a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp +++ b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp @@ -39,8 +39,7 @@ void F_TEST_LOG_FMT(const std::string & s) } #else #define TEST_LOG_FMT(...) -void F_TEST_LOG_FMT(const std::string &) -{} +void F_TEST_LOG_FMT(const std::string &) {} #endif } // namespace DB @@ -88,14 +87,12 @@ void AsyncWaker::wake(RawVoidPtr notifier_) AsyncWaker::AsyncWaker(const TiFlashRaftProxyHelper & helper_) : AsyncWaker(helper_, new AsyncWaker::Notifier{}) -{ -} +{} AsyncWaker::AsyncWaker(const TiFlashRaftProxyHelper & helper_, AsyncNotifier * notifier_) : inner(helper_.makeAsyncWaker(AsyncWaker::wake, GenRawCppPtr(notifier_, RawCppPtrTypeImpl::WakerNotifier))) , notifier(*notifier_) -{ -} +{} AsyncNotifier::Status AsyncWaker::waitUtil(SteadyClock::time_point time_point) { @@ -132,13 +129,9 @@ struct BlockedReadIndexHelper final : BlockedReadIndexHelperTrait BlockedReadIndexHelper(uint64_t timeout_ms_, AsyncWaker & waker_) : BlockedReadIndexHelperTrait(timeout_ms_) , waker(waker_) - { - } + {} - const AsyncWaker & getWaker() const - { - return waker; - } + const AsyncWaker & getWaker() const { return waker; } AsyncNotifier::Status blockedWaitUtil(SteadyClock::time_point time_point) override { @@ -156,8 +149,7 @@ struct BlockedReadIndexHelperV3 final : BlockedReadIndexHelperTrait BlockedReadIndexHelperV3(uint64_t timeout_ms_, AsyncWaker::Notifier & notifier_) : BlockedReadIndexHelperTrait(timeout_ms_) , notifier(notifier_) - { - } + {} AsyncNotifier::Status blockedWaitUtil(SteadyClock::time_point time_point) override { @@ -170,12 +162,16 @@ struct BlockedReadIndexHelperV3 final : BlockedReadIndexHelperTrait AsyncWaker::Notifier & notifier; }; -BatchReadIndexRes TiFlashRaftProxyHelper::batchReadIndex(const std::vector & req, uint64_t timeout_ms) const +BatchReadIndexRes TiFlashRaftProxyHelper::batchReadIndex( + const std::vector & req, + uint64_t timeout_ms) const { return batchReadIndex_v2(req, timeout_ms); } -BatchReadIndexRes TiFlashRaftProxyHelper::batchReadIndex_v2(const std::vector & req, uint64_t timeout_ms) const +BatchReadIndexRes TiFlashRaftProxyHelper::batchReadIndex_v2( + const std::vector & req, + uint64_t timeout_ms) const { AsyncWaker waker(*this); BlockedReadIndexHelper helper{timeout_ms, waker}; @@ -257,7 +253,10 @@ std::optional TiFlashRaftProxyHelper::makeReadIndexTask(const kvr } } -bool TiFlashRaftProxyHelper::pollReadIndexTask(ReadIndexTask & task, kvrpcpb::ReadIndexResponse & resp, RawVoidPtr waker) const +bool TiFlashRaftProxyHelper::pollReadIndexTask( + ReadIndexTask & task, + kvrpcpb::ReadIndexResponse & resp, + RawVoidPtr waker) const { return fn_poll_read_index_task(proxy_ptr, task.ptr, &resp, waker); } @@ -292,10 +291,7 @@ struct ReadIndexNotifyCtrl : MutexLockWrap return std::move(data); } - void wake() const - { - notifier->wake(); - } + void wake() const { notifier->wake(); } explicit ReadIndexNotifyCtrl(AsyncWaker::NotifierPtr notifier_) : notifier(notifier_) @@ -312,17 +308,11 @@ struct RegionReadIndexNotifier final : AsyncNotifier notify->add(region_id, ts); notify->wake(); } - Status blockedWaitUtil(const SteadyClock::time_point &) override - { - return Status::Timeout; - } + Status blockedWaitUtil(const SteadyClock::time_point &) override { return Status::Timeout; } ~RegionReadIndexNotifier() override = default; - RegionReadIndexNotifier( - RegionID region_id_, - Timestamp ts_, - const ReadIndexNotifyCtrlPtr & notify_) + RegionReadIndexNotifier(RegionID region_id_, Timestamp ts_, const ReadIndexNotifyCtrlPtr & notify_) : region_id(region_id_) , ts(ts_) , notify(notify_) @@ -333,7 +323,8 @@ struct RegionReadIndexNotifier final : AsyncNotifier ReadIndexNotifyCtrlPtr notify; }; -std::atomic ReadIndexWorker::max_read_index_task_timeout = std::chrono::milliseconds{8 * 1000}; +std::atomic ReadIndexWorker::max_read_index_task_timeout + = std::chrono::milliseconds{8 * 1000}; //std::atomic ReadIndexWorker::max_read_index_history{8}; void ReadIndexFuture::update(kvrpcpb::ReadIndexResponse resp) @@ -404,7 +395,9 @@ void ReadIndexDataNode::ReadIndexElement::doForceSet(const kvrpcpb::ReadIndexRes doTriggerCallbacks(); } -void ReadIndexDataNode::ReadIndexElement::doPoll(const TiFlashRaftProxyHelper & helper, std::chrono::milliseconds timeout) +void ReadIndexDataNode::ReadIndexElement::doPoll( + const TiFlashRaftProxyHelper & helper, + std::chrono::milliseconds timeout) { bool can_trigger = false; { @@ -416,7 +409,10 @@ void ReadIndexDataNode::ReadIndexElement::doPoll(const TiFlashRaftProxyHelper & bool clean_task = false; if (helper.pollReadIndexTask(task, resp, raw_waker)) { - TEST_LOG_FMT("poll ReadIndexElement success for region_id={}, resp {}", region_id, resp.ShortDebugString()); + TEST_LOG_FMT( + "poll ReadIndexElement success for region_id={}, resp {}", + region_id, + resp.ShortDebugString()); clean_task = true; } @@ -425,7 +421,8 @@ void ReadIndexDataNode::ReadIndexElement::doPoll(const TiFlashRaftProxyHelper & TEST_LOG_FMT("poll ReadIndexElement timeout for region_id={}", region_id); clean_task = true; - resp.mutable_region_error()->mutable_server_is_busy(); // set region_error `server_is_busy` for task timeout + resp.mutable_region_error() + ->mutable_server_is_busy(); // set region_error `server_is_busy` for task timeout } else { @@ -508,7 +505,8 @@ std::optional makeReadIndexTask(const TiFlashRaftProxyHelper & he else { auto ori_id = req.context().region_id(); - req.mutable_context()->set_region_id(ori_id % MockStressTestCfg::RegionIdPrefix); // set region id to original one. + req.mutable_context()->set_region_id( + ori_id % MockStressTestCfg::RegionIdPrefix); // set region id to original one. TEST_LOG_FMT("hacked ReadIndexTask to req {}", req.ShortDebugString()); auto res = helper.makeReadIndexTask(req); req.mutable_context()->set_region_id(ori_id); @@ -602,7 +600,11 @@ ReadIndexDataNode::~ReadIndexDataNode() kvrpcpb::ReadIndexResponse resp; resp.mutable_region_error()->mutable_region_not_found(); - TEST_LOG_FMT("~ReadIndexDataNode region_id={}: waiting_tasks {}, running_tasks {} ", region_id, waiting_tasks.size(), running_tasks.size()); + TEST_LOG_FMT( + "~ReadIndexDataNode region_id={}: waiting_tasks {}, running_tasks {} ", + region_id, + waiting_tasks.size(), + running_tasks.size()); if (auto waiting_tasks = this->waiting_tasks.popAll(); waiting_tasks) { @@ -730,12 +732,15 @@ ReadIndexWorker::ReadIndexWorker( : proxy_helper(proxy_helper_) , id(id_) , read_index_notify_ctrl(std::make_shared(notifier_)) -{ -} +{} bool ReadIndexWorker::lastRunTimeout(SteadyClock::duration timeout) const { - TEST_LOG_FMT("worker {}, last run time {}, timeout {}", getID(), last_run_time.load(std::memory_order_relaxed).time_since_epoch().count(), std::chrono::duration_cast(timeout).count()); + TEST_LOG_FMT( + "worker {}, last run time {}, timeout {}", + getID(), + last_run_time.load(std::memory_order_relaxed).time_since_epoch().count(), + std::chrono::duration_cast(timeout).count()); return last_run_time.load(std::memory_order_relaxed) + timeout < SteadyClock::now(); } @@ -770,10 +775,7 @@ ReadIndexWorkerManager::ReadIndexWorkerManager( { for (size_t wid = rid; wid < workers_cnt; wid += runner_cnt) { - workers[wid] = std::make_unique( - proxy_helper, - wid, - runners[rid]->global_notifier); + workers[wid] = std::make_unique(proxy_helper, wid, runners[rid]->global_notifier); } } } @@ -881,7 +883,8 @@ void ReadIndexWorker::removeRegion(uint64_t region_id) data_map.removeRegion(region_id); } -BatchReadIndexRes KVStore::batchReadIndex(const std::vector & reqs, uint64_t timeout_ms) const +BatchReadIndexRes KVStore::batchReadIndex(const std::vector & reqs, uint64_t timeout_ms) + const { assert(this->proxy_helper); if (read_index_worker_manager) @@ -906,7 +909,10 @@ std::unique_ptr ReadIndexWorkerManager::newReadIndexWork return std::make_unique(proxy_helper, cap, std::move(fn_min_dur_handle_region), runner_cnt); } -void KVStore::initReadIndexWorkers(ReadIndexWorkerManager::FnGetTickTime && fn_min_dur_handle_region, size_t runner_cnt, size_t worker_coefficient) +void KVStore::initReadIndexWorkers( + ReadIndexWorkerManager::FnGetTickTime && fn_min_dur_handle_region, + size_t runner_cnt, + size_t worker_coefficient) { if (!runner_cnt) { diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 37195b4626f..525cd9efe57 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -49,7 +49,10 @@ RegionData::WriteCFIter Region::removeDataByWriteIt(const RegionData::WriteCFIte return data.removeDataByWriteIt(write_it); } -std::optional Region::readDataByWriteIt(const RegionData::ConstWriteCFIter & write_it, bool need_value, bool hard_error) +std::optional Region::readDataByWriteIt( + const RegionData::ConstWriteCFIter & write_it, + bool need_value, + bool hard_error) { try { @@ -144,11 +147,12 @@ void RegionRaftCommandDelegate::execChangePeer( const UInt64 index, const UInt64 term) { - LOG_INFO(log, - "{} execute change peer cmd: {}", - toString(false), - (request.has_change_peer_v2() ? request.change_peer_v2().ShortDebugString() - : request.change_peer().ShortDebugString())); + LOG_INFO( + log, + "{} execute change peer cmd: {}", + toString(false), + (request.has_change_peer_v2() ? request.change_peer_v2().ShortDebugString() + : request.change_peer().ShortDebugString())); meta.makeRaftCommandDelegate().execChangePeer(request, response, index, term); LOG_INFO(log, "After execute change peer cmd, current region info: {}", getDebugString()); } @@ -161,10 +165,7 @@ static const metapb::Peer & findPeerByStore(const metapb::Region & region, UInt6 return peer; } - throw Exception(ErrorCodes::LOGICAL_ERROR, - "{}: peer not found, store_id={}", - __PRETTY_FUNCTION__, - store_id); + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: peer not found, store_id={}", __PRETTY_FUNCTION__, store_id); } Regions RegionRaftCommandDelegate::execBatchSplit( @@ -199,7 +200,9 @@ Regions RegionRaftCommandDelegate::execBatchSplit( if (new_region_index == -1) new_region_index = i; else - throw Exception(std::string(__PRETTY_FUNCTION__) + ": duplicate region index", ErrorCodes::LOGICAL_ERROR); + throw Exception( + std::string(__PRETTY_FUNCTION__) + ": duplicate region index", + ErrorCodes::LOGICAL_ERROR); } } @@ -235,11 +238,12 @@ void RegionRaftCommandDelegate::execPrepareMerge( const auto & target = prepare_merge_request.target(); - LOG_INFO(log, - "{} execute prepare merge, min_index {}, target region_id={}", - toString(false), - prepare_merge_request.min_index(), - target.id()); + LOG_INFO( + log, + "{} execute prepare merge, min_index {}, target region_id={}", + toString(false), + prepare_merge_request.min_index(), + target.id()); meta.makeRaftCommandDelegate().execPrepareMerge(request, response, index, term); } @@ -252,30 +256,28 @@ void RegionRaftCommandDelegate::execRollbackMerge( { const auto & rollback_request = request.rollback_merge(); - LOG_INFO( - log, - "{} execute rollback merge, commit index {}", - toString(false), - rollback_request.commit()); + LOG_INFO(log, "{} execute rollback merge, commit index {}", toString(false), rollback_request.commit()); meta.makeRaftCommandDelegate().execRollbackMerge(request, response, index, term); } -RegionID RegionRaftCommandDelegate::execCommitMerge(const raft_cmdpb::AdminRequest & request, - const raft_cmdpb::AdminResponse & response, - const UInt64 index, - const UInt64 term, - const KVStore & kvstore, - RegionTable & region_table) +RegionID RegionRaftCommandDelegate::execCommitMerge( + const raft_cmdpb::AdminRequest & request, + const raft_cmdpb::AdminResponse & response, + const UInt64 index, + const UInt64 term, + const KVStore & kvstore, + RegionTable & region_table) { const auto & commit_merge_request = request.commit_merge(); auto & meta_delegate = meta.makeRaftCommandDelegate(); const auto & source_meta = commit_merge_request.source(); auto source_region = kvstore.getRegion(source_meta.id()); - LOG_INFO(log, - "{} execute commit merge, source region_id={}, commit index={}", - toString(false), - source_meta.id(), - commit_merge_request.commit()); + LOG_INFO( + log, + "{} execute commit merge, source region_id={}, commit index={}", + toString(false), + source_meta.id(), + commit_merge_request.commit()); const auto & source_region_meta_delegate = source_region->meta.makeRaftCommandDelegate(); const auto res = meta_delegate.checkBeforeCommitMerge(request, source_region_meta_delegate); @@ -283,12 +285,16 @@ RegionID RegionRaftCommandDelegate::execCommitMerge(const raft_cmdpb::AdminReque source_region->setPendingRemove(); { - const std::string & new_start_key = res.source_at_left ? source_region_meta_delegate.regionState().getRegion().start_key() - : meta_delegate.regionState().getRegion().start_key(); - const std::string & new_end_key = res.source_at_left ? meta_delegate.regionState().getRegion().end_key() - : source_region_meta_delegate.regionState().getRegion().end_key(); - - region_table.extendRegionRange(id(), RegionRangeKeys(TiKVKey::copyFrom(new_start_key), TiKVKey::copyFrom(new_end_key))); + const std::string & new_start_key = res.source_at_left + ? source_region_meta_delegate.regionState().getRegion().start_key() + : meta_delegate.regionState().getRegion().start_key(); + const std::string & new_end_key = res.source_at_left + ? meta_delegate.regionState().getRegion().end_key() + : source_region_meta_delegate.regionState().getRegion().end_key(); + + region_table.extendRegionRange( + id(), + RegionRangeKeys(TiKVKey::copyFrom(new_start_key), TiKVKey::copyFrom(new_end_key))); } { @@ -305,13 +311,14 @@ RegionID RegionRaftCommandDelegate::execCommitMerge(const raft_cmdpb::AdminReque return source_meta.id(); } -void RegionRaftCommandDelegate::handleAdminRaftCmd(const raft_cmdpb::AdminRequest & request, - const raft_cmdpb::AdminResponse & response, - UInt64 index, - UInt64 term, - const KVStore & kvstore, - RegionTable & region_table, - RaftCommandResult & result) +void RegionRaftCommandDelegate::handleAdminRaftCmd( + const raft_cmdpb::AdminRequest & request, + const raft_cmdpb::AdminResponse & response, + UInt64 index, + UInt64 term, + const KVStore & kvstore, + RegionTable & region_table, + RaftCommandResult & result) { result.type = RaftCommandResult::Type::Default; if (index <= appliedIndex()) @@ -322,12 +329,13 @@ void RegionRaftCommandDelegate::handleAdminRaftCmd(const raft_cmdpb::AdminReques auto type = request.cmd_type(); - LOG_INFO(log, - "{} execute admin command {} at [term: {}, index: {}]", - toString(), - raft_cmdpb::AdminCmdType_Name(type), - term, - index); + LOG_INFO( + log, + "{} execute admin command {} at [term: {}, index: {}]", + toString(), + raft_cmdpb::AdminCmdType_Name(type), + term, + index); switch (type) { @@ -363,7 +371,9 @@ void RegionRaftCommandDelegate::handleAdminRaftCmd(const raft_cmdpb::AdminReques execRollbackMerge(request, response, index, term); break; default: - throw Exception(fmt::format("unsupported admin command type {}", raft_cmdpb::AdminCmdType_Name(type)), ErrorCodes::LOGICAL_ERROR); + throw Exception( + fmt::format("unsupported admin command type {}", raft_cmdpb::AdminCmdType_Name(type)), + ErrorCodes::LOGICAL_ERROR); } switch (type) @@ -406,9 +416,10 @@ RegionPtr Region::deserialize(ReadBuffer & buf, const TiFlashRaftProxyHelper * p { auto version = readBinary2(buf); if (version != Region::CURRENT_VERSION) - throw Exception(std::string(__PRETTY_FUNCTION__) + ": unexpected version: " + DB::toString(version) - + ", expected: " + DB::toString(CURRENT_VERSION), - ErrorCodes::UNKNOWN_FORMAT_VERSION); + throw Exception( + std::string(__PRETTY_FUNCTION__) + ": unexpected version: " + DB::toString(version) + + ", expected: " + DB::toString(CURRENT_VERSION), + ErrorCodes::UNKNOWN_FORMAT_VERSION); auto meta = RegionMeta::deserialize(buf); auto region = std::make_shared(std::move(meta), proxy_helper); @@ -523,7 +534,8 @@ void Region::updateLastCompactLogApplied() const uint64_t current_applied_index = appliedIndex(); if (last_compact_log_applied != 0) { - uint64_t gap = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; + uint64_t gap + = current_applied_index > last_compact_log_applied ? current_applied_index - last_compact_log_applied : 0; GET_METRIC(tiflash_raft_raft_log_lag_count, type_applied_index).Observe(gap); } last_compact_log_applied = current_applied_index; @@ -574,7 +586,12 @@ void Region::afterPrehandleSnapshot() if (getClusterRaftstoreVer() == RaftstoreVer::V2) { data.orphan_keys_info.pre_handling = false; - LOG_INFO(log, "After prehandle, remains orphan keys {} removed orphan keys {} [region_id={}]", data.orphan_keys_info.remainedKeyCount(), data.orphan_keys_info.removed_remained_keys.size(), id()); + LOG_INFO( + log, + "After prehandle, remains orphan keys {} removed orphan keys {} [region_id={}]", + data.orphan_keys_info.remainedKeyCount(), + data.orphan_keys_info.removed_remained_keys.size(), + id()); } } @@ -606,28 +623,24 @@ bool Region::checkIndex(UInt64 index) const return meta.checkIndex(index); } -std::tuple Region::waitIndex(UInt64 index, const UInt64 timeout_ms, std::function && check_running) +std::tuple Region::waitIndex( + UInt64 index, + const UInt64 timeout_ms, + std::function && check_running) { if (proxy_helper != nullptr) { if (!meta.checkIndex(index)) { Stopwatch wait_index_watch; - LOG_DEBUG(log, - "{} need to wait learner index {} timeout {}", - toString(), - index, - timeout_ms); + LOG_DEBUG(log, "{} need to wait learner index {} timeout {}", toString(), index, timeout_ms); auto wait_idx_res = meta.waitIndex(index, timeout_ms, std::move(check_running)); auto elapsed_secs = wait_index_watch.elapsedSeconds(); switch (wait_idx_res) { case WaitIndexResult::Finished: { - LOG_DEBUG(log, - "{} wait learner index {} done", - toString(false), - index); + LOG_DEBUG(log, "{} wait learner index {} done", toString(false), index); return {wait_idx_res, elapsed_secs}; } case WaitIndexResult::Terminated: @@ -698,14 +711,15 @@ void Region::tryCompactionFilter(const Timestamp safe_point) // No need to check default cf. Because tikv will gc default cf before write cf. if (del_write) { - LOG_INFO(log, - "delete {} records in write cf for region_id={}", - del_write, - meta.regionId()); + LOG_INFO(log, "delete {} records in write cf for region_id={}", del_write, meta.regionId()); } } -std::pair Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt) +std::pair Region::handleWriteRaftCmd( + const WriteCmdsView & cmds, + UInt64 index, + UInt64 term, + TMTContext & tmt) { if (index <= appliedIndex()) { @@ -713,7 +727,8 @@ std::pair Region::handleWriteRaftCmd(const } auto & context = tmt.getContext(); Stopwatch watch; - SCOPE_EXIT({ GET_METRIC(tiflash_raft_apply_write_command_duration_seconds, type_write).Observe(watch.elapsedSeconds()); }); + SCOPE_EXIT( + { GET_METRIC(tiflash_raft_apply_write_command_duration_seconds, type_write).Observe(watch.elapsedSeconds()); }); auto is_v2 = this->getClusterRaftstoreVer() == RaftstoreVer::V2; @@ -740,13 +755,14 @@ std::pair Region::handleWriteRaftCmd(const } catch (Exception & e) { - LOG_ERROR(log, - "{} catch exception: {}, while applying `CmdType::Put` on [term {}, index {}], CF {}", - toString(), - e.message(), - term, - index, - CFToName(cf)); + LOG_ERROR( + log, + "{} catch exception: {}, while applying `CmdType::Put` on [term {}, index {}], CF {}", + toString(), + e.message(), + term, + index, + CFToName(cf)); e.rethrow(); } break; @@ -760,14 +776,16 @@ std::pair Region::handleWriteRaftCmd(const } catch (Exception & e) { - LOG_ERROR(log, - "{} catch exception: {}, while applying `CmdType::Delete` on [term {}, index {}], key in hex: {}, CF {}", - toString(), - e.message(), - term, - index, - tikv_key.toDebugString(), - CFToName(cf)); + LOG_ERROR( + log, + "{} catch exception: {}, while applying `CmdType::Delete` on [term {}, index {}], key in hex: {}, " + "CF {}", + toString(), + e.message(), + term, + index, + tikv_key.toDebugString(), + CFToName(cf)); e.rethrow(); } break; @@ -819,7 +837,8 @@ std::pair Region::handleWriteRaftCmd(const RegionDataReadInfoList data_list_to_remove; try { - write_result = RegionTable::writeBlockByRegion(context, shared_from_this(), data_list_to_remove, log, true); + write_result + = RegionTable::writeBlockByRegion(context, shared_from_this(), data_list_to_remove, log, true); } catch (DB::Exception & e) { @@ -829,15 +848,21 @@ std::pair Region::handleWriteRaftCmd(const auto cf = cmds.cmd_cf[i]; auto type = cmds.cmd_types[i]; auto tikv_key = TiKVKey(cmds.keys[i].data, cmds.keys[i].len); - entry_infos.emplace_back(fmt::format("{}|{}|{}", type == DB::WriteCmdType::Put ? "PUT" : "DEL", CFToName(cf), tikv_key.toDebugString())); + entry_infos.emplace_back(fmt::format( + "{}|{}|{}", + type == DB::WriteCmdType::Put ? "PUT" : "DEL", + CFToName(cf), + tikv_key.toDebugString())); } - LOG_ERROR(log, - "{} catch exception: {}, while applying `RegionTable::writeBlockByRegion` on [term {}, index {}], entries {}", - toString(), - e.message(), - term, - index, - fmt::join(entry_infos.begin(), entry_infos.end(), ":")); + LOG_ERROR( + log, + "{} catch exception: {}, while applying `RegionTable::writeBlockByRegion` on [term {}, index {}], " + "entries {}", + toString(), + e.message(), + term, + index, + fmt::join(entry_infos.begin(), entry_infos.end(), ":")); e.rethrow(); } } @@ -867,12 +892,13 @@ void Region::finishIngestSSTByDTFile(RegionPtr && temp_region, UInt64 index, UIn meta.setApplied(index, term); } - LOG_INFO(log, - "{} finish ingest sst by DTFile, write_cf_keys={} default_cf_keys={} lock_cf_keys={}", - this->toString(false), - data.write_cf.getSize(), - data.default_cf.getSize(), - data.lock_cf.getSize()); + LOG_INFO( + log, + "{} finish ingest sst by DTFile, write_cf_keys={} default_cf_keys={} lock_cf_keys={}", + this->toString(false), + data.write_cf.getSize(), + data.default_cf.getSize(), + data.lock_cf.getSize()); meta.notifyAll(); } @@ -944,7 +970,9 @@ const raft_serverpb::MergeState & Region::getMergeState() const std::pair Region::getApproxMemCacheInfo() const { - return {approx_mem_cache_rows.load(std::memory_order_relaxed), approx_mem_cache_bytes.load(std::memory_order_relaxed)}; + return { + approx_mem_cache_rows.load(std::memory_order_relaxed), + approx_mem_cache_bytes.load(std::memory_order_relaxed)}; } void Region::cleanApproxMemCacheInfo() const diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index ccdae5fdc89..9ca465ef47c 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -168,7 +168,10 @@ class Region : public std::enable_shared_from_this bool checkIndex(UInt64 index) const; // Return for wait-index. - std::tuple waitIndex(UInt64 index, UInt64 timeout_ms, std::function && check_running); + std::tuple waitIndex( + UInt64 index, + UInt64 timeout_ms, + std::function && check_running); // Requires RegionMeta's lock UInt64 appliedIndex() const; @@ -197,7 +200,11 @@ class Region : public std::enable_shared_from_this TableID getMappedTableID() const; KeyspaceID getKeyspaceID() const; - std::pair handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt); + std::pair handleWriteRaftCmd( + const WriteCmdsView & cmds, + UInt64 index, + UInt64 term, + TMTContext & tmt); /// get approx rows, bytes info about mem cache. std::pair getApproxMemCacheInfo() const; @@ -231,7 +238,10 @@ class Region : public std::enable_shared_from_this void doCheckTable(const DecodedTiKVKey & key) const; void doRemove(ColumnFamilyType type, const TiKVKey & key); - std::optional readDataByWriteIt(const RegionData::ConstWriteCFIter & write_it, bool need_value, bool hard_error); + std::optional readDataByWriteIt( + const RegionData::ConstWriteCFIter & write_it, + bool need_value, + bool hard_error); RegionData::WriteCFIter removeDataByWriteIt(const RegionData::WriteCFIter & write_it); DecodedLockCFValuePtr getLockInfo(const RegionLockReadQuery & query) const; @@ -261,12 +271,20 @@ class Region : public std::enable_shared_from_this mutable std::atomic approx_mem_cache_bytes{0}; }; -class RegionRaftCommandDelegate : public Region +class RegionRaftCommandDelegate + : public Region , private boost::noncopyable { public: /// Only after the task mutex of KVStore is locked, region can apply raft command. - void handleAdminRaftCmd(const raft_cmdpb::AdminRequest &, const raft_cmdpb::AdminResponse &, UInt64, UInt64, const KVStore &, RegionTable &, RaftCommandResult &); + void handleAdminRaftCmd( + const raft_cmdpb::AdminRequest &, + const raft_cmdpb::AdminResponse &, + UInt64, + UInt64, + const KVStore &, + RegionTable &, + RaftCommandResult &); const RegionRangeKeys & getRange(); UInt64 appliedIndex(); diff --git a/dbms/src/Storages/Transaction/RegionCFDataBase.cpp b/dbms/src/Storages/Transaction/RegionCFDataBase.cpp index 15643a65294..2c24e7eab0f 100644 --- a/dbms/src/Storages/Transaction/RegionCFDataBase.cpp +++ b/dbms/src/Storages/Transaction/RegionCFDataBase.cpp @@ -79,22 +79,26 @@ RegionDataRes RegionCFDataBase::insert(std::pair && kv_pair, { if (mode == DupCheck::Deny) { - throw Exception("Found existing key in hex: " + getTiKVKey(it->second).toDebugString(), ErrorCodes::LOGICAL_ERROR); + throw Exception( + "Found existing key in hex: " + getTiKVKey(it->second).toDebugString(), + ErrorCodes::LOGICAL_ERROR); } else if (mode == DupCheck::AllowSame) { if (prev_value != getTiKVValue(it->second)) { - throw Exception("Found existing key in hex and val differs: " - + getTiKVKey(it->second).toDebugString() - + " prev_val: " + getTiKVValue(it->second).toDebugString() - + " new_val: " + prev_value.toDebugString(), - ErrorCodes::LOGICAL_ERROR); + throw Exception( + "Found existing key in hex and val differs: " + getTiKVKey(it->second).toDebugString() + + " prev_val: " + getTiKVValue(it->second).toDebugString() + + " new_val: " + prev_value.toDebugString(), + ErrorCodes::LOGICAL_ERROR); } } else { - throw Exception("Found existing key in hex: " + getTiKVKey(it->second).toDebugString(), ErrorCodes::LOGICAL_ERROR); + throw Exception( + "Found existing key in hex: " + getTiKVKey(it->second).toDebugString(), + ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/Storages/Transaction/RegionData.cpp b/dbms/src/Storages/Transaction/RegionData.cpp index f3d2078cb43..429f9c14c31 100644 --- a/dbms/src/Storages/Transaction/RegionData.cpp +++ b/dbms/src/Storages/Transaction/RegionData.cpp @@ -110,7 +110,12 @@ RegionData::WriteCFIter RegionData::removeDataByWriteIt(const WriteCFIter & writ } /// This function is called by `ReadRegionCommitCache`. -std::optional RegionData::readDataByWriteIt(const ConstWriteCFIter & write_it, bool need_value, RegionID region_id, UInt64 applied, bool hard_error) +std::optional RegionData::readDataByWriteIt( + const ConstWriteCFIter & write_it, + bool need_value, + RegionID region_id, + UInt64 applied, + bool hard_error) { const auto & [key, value, decoded_val] = write_it->second; const auto & [pk, ts] = write_it->first; @@ -139,8 +144,9 @@ std::optional RegionData::readDataByWriteIt(const ConstWrite { if (orphan_keys_info.pre_handling) { - RUNTIME_CHECK_MSG(orphan_keys_info.snapshot_index.has_value(), - "Snapshot index shall be set when Applying snapshot"); + RUNTIME_CHECK_MSG( + orphan_keys_info.snapshot_index.has_value(), + "Snapshot index shall be set when Applying snapshot"); // While pre-handling snapshot from raftstore v2, we accept and store the orphan keys in memory // These keys should be resolved in later raft logs orphan_keys_info.observeExtraKey(TiKVKey::copyFrom(*key)); @@ -177,20 +183,26 @@ std::optional RegionData::readDataByWriteIt(const ConstWrite } if (!hard_error) { - orphan_key_debug_msg = fmt::format("orphan_info: ({}, snapshot_index: {}, {}, orphan key size {})", - hard_error ? "" : ", not orphan key", - orphan_keys_info.snapshot_index.has_value() ? std::to_string(orphan_keys_info.snapshot_index.value()) : "", - orphan_keys_info.removed_remained_keys.contains(*key) ? "duplicated write" : "missing default", - orphan_keys_info.remainedKeyCount()); + orphan_key_debug_msg = fmt::format( + "orphan_info: ({}, snapshot_index: {}, {}, orphan key size {})", + hard_error ? "" : ", not orphan key", + orphan_keys_info.snapshot_index.has_value() + ? std::to_string(orphan_keys_info.snapshot_index.value()) + : "", + orphan_keys_info.removed_remained_keys.contains(*key) ? "duplicated write" : "missing default", + orphan_keys_info.remainedKeyCount()); } - throw Exception(fmt::format("Raw TiDB PK: {}, Prewrite ts: {} can not found in default cf for key: {}, region_id: {}, applied_index: {}{}", - pk.toDebugString(), - decoded_val.prewrite_ts, - key->toDebugString(), - region_id, - applied, - orphan_key_debug_msg), - ErrorCodes::ILLFORMAT_RAFT_ROW); + throw Exception( + fmt::format( + "Raw TiDB PK: {}, Prewrite ts: {} can not found in default cf for key: {}, region_id: {}, " + "applied_index: {}{}", + pk.toDebugString(), + decoded_val.prewrite_ts, + key->toDebugString(), + region_id, + applied, + orphan_key_debug_msg), + ErrorCodes::ILLFORMAT_RAFT_ROW); } } @@ -300,7 +312,8 @@ const RegionLockCFData & RegionData::lockCF() const bool RegionData::isEqual(const RegionData & r2) const { - return default_cf == r2.default_cf && write_cf == r2.write_cf && lock_cf == r2.lock_cf && cf_data_size == r2.cf_data_size; + return default_cf == r2.default_cf && write_cf == r2.write_cf && lock_cf == r2.lock_cf + && cf_data_size == r2.cf_data_size; } RegionData::RegionData(RegionData && data) @@ -365,7 +378,15 @@ void RegionData::OrphanKeysInfo::advanceAppliedIndex(uint64_t applied_index) if (applied_index >= deadline_index.value() && count > 0) { auto one = remained_keys.begin()->toDebugString(); - throw Exception(fmt::format("Orphan keys from snapshot still exists. One of total {} is {}. region_id={} snapshot_index={} deadline_index={} applied_index={}", count, one, region_id, snapshot_index.value(), deadline_index.value(), applied_index)); + throw Exception(fmt::format( + "Orphan keys from snapshot still exists. One of total {} is {}. region_id={} snapshot_index={} " + "deadline_index={} applied_index={}", + count, + one, + region_id, + snapshot_index.value(), + deadline_index.value(), + applied_index)); } } } diff --git a/dbms/src/Storages/Transaction/RegionManager.h b/dbms/src/Storages/Transaction/RegionManager.h index c375488a79b..9b23b20511a 100644 --- a/dbms/src/Storages/Transaction/RegionManager.h +++ b/dbms/src/Storages/Transaction/RegionManager.h @@ -56,15 +56,9 @@ struct RegionManager : SharedMutexLockWrap RegionsRangeIndex & index; }; - RegionReadLock genReadLock() const - { - return {genSharedLock(), regions, region_range_index}; - } + RegionReadLock genReadLock() const { return {genSharedLock(), regions, region_range_index}; } - RegionWriteLock genWriteLock() - { - return {genUniqueLock(), regions, region_range_index}; - } + RegionWriteLock genWriteLock() { return {genUniqueLock(), regions, region_range_index}; } /// Encapsulate the task lock for region RegionTaskLock genRegionTaskLock(RegionID region_id) const; diff --git a/dbms/src/Storages/Transaction/RegionMeta.cpp b/dbms/src/Storages/Transaction/RegionMeta.cpp index 4fe8936ce7e..a8f6207bbf8 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.cpp +++ b/dbms/src/Storages/Transaction/RegionMeta.cpp @@ -166,7 +166,8 @@ void RegionMeta::setPeerState(const raft_serverpb::PeerState peer_state_) region_state.setState(peer_state_); } -WaitIndexResult RegionMeta::waitIndex(UInt64 index, const UInt64 timeout_ms, std::function && check_running) const +WaitIndexResult RegionMeta::waitIndex(UInt64 index, const UInt64 timeout_ms, std::function && check_running) + const { std::unique_lock lock(mutex); WaitIndexResult status = WaitIndexResult::Finished; @@ -287,11 +288,20 @@ RegionMergeResult MetaRaftCommandDelegate::checkBeforeCommitMerge( case raft_serverpb::PeerState::Normal: break; default: - throw Exception(fmt::format("{}: unexpected state {} of source {}", __FUNCTION__, raft_serverpb::PeerState_Name(state), regionId(), toString(false)), ErrorCodes::LOGICAL_ERROR); + throw Exception( + fmt::format( + "{}: unexpected state {} of source {}", + __FUNCTION__, + raft_serverpb::PeerState_Name(state), + regionId(), + toString(false)), + ErrorCodes::LOGICAL_ERROR); } if (!(source_region == source_meta.region_state.getRegion())) - throw Exception(fmt::format("{}: source region not match exist region meta", __FUNCTION__), ErrorCodes::LOGICAL_ERROR); + throw Exception( + fmt::format("{}: source region not match exist region meta", __FUNCTION__), + ErrorCodes::LOGICAL_ERROR); return computeRegionMergeResult(source_region, region_state.getRegion()); } @@ -302,7 +312,11 @@ void CheckRegionForMergeCmd(const raft_cmdpb::AdminResponse & response, const Re { if (response.has_split() && !(response.split().left() == region_state.getRegion())) throw Exception( - fmt::format("{}: current region meta: {}, expect: {}", __FUNCTION__, region_state.getRegion().ShortDebugString(), response.split().left().ShortDebugString()), + fmt::format( + "{}: current region meta: {}, expect: {}", + __FUNCTION__, + region_state.getRegion().ShortDebugString(), + response.split().left().ShortDebugString()), ErrorCodes::LOGICAL_ERROR); } #pragma GCC diagnostic pop @@ -317,11 +331,19 @@ void MetaRaftCommandDelegate::execRollbackMerge( if (region_state.getState() != raft_serverpb::PeerState::Merging) throw Exception( - fmt::format("{}: region state is {}, expect {}", __FUNCTION__, raft_serverpb::PeerState_Name(region_state.getState()), raft_serverpb::PeerState_Name(raft_serverpb::PeerState::Merging)), + fmt::format( + "{}: region state is {}, expect {}", + __FUNCTION__, + raft_serverpb::PeerState_Name(region_state.getState()), + raft_serverpb::PeerState_Name(raft_serverpb::PeerState::Merging)), ErrorCodes::LOGICAL_ERROR); if (region_state.getMergeState().commit() != rollback_request.commit()) throw Exception( - fmt::format("{}: merge commit index is {}, expect {}", __FUNCTION__, region_state.getMergeState().commit(), rollback_request.commit()), + fmt::format( + "{}: merge commit index is {}, expect {}", + __FUNCTION__, + region_state.getMergeState().commit(), + rollback_request.commit()), ErrorCodes::LOGICAL_ERROR); std::lock_guard lock(mutex); @@ -406,8 +428,8 @@ bool operator==(const RegionMeta & meta1, const RegionMeta & meta2) std::lock_guard lock1(meta1.mutex); std::lock_guard lock2(meta2.mutex); - return meta1.peer == meta2.peer && meta1.apply_state == meta2.apply_state && meta1.applied_term == meta2.applied_term - && meta1.region_state == meta2.region_state; + return meta1.peer == meta2.peer && meta1.apply_state == meta2.apply_state + && meta1.applied_term == meta2.applied_term && meta1.region_state == meta2.region_state; } RegionMetaSnapshot RegionMeta::dumpRegionMetaSnapshot() const @@ -431,7 +453,11 @@ const RegionState & MetaRaftCommandDelegate::regionState() const return region_state; } -RegionMeta::RegionMeta(metapb::Peer peer_, raft_serverpb::RaftApplyState apply_state_, const UInt64 applied_term_, raft_serverpb::RegionLocalState region_state_) +RegionMeta::RegionMeta( + metapb::Peer peer_, + raft_serverpb::RaftApplyState apply_state_, + const UInt64 applied_term_, + raft_serverpb::RegionLocalState region_state_) : peer(std::move(peer_)) , apply_state(std::move(apply_state_)) , applied_term(applied_term_) diff --git a/dbms/src/Storages/Transaction/RegionMeta.h b/dbms/src/Storages/Transaction/RegionMeta.h index a6db236d9c5..a2c8769bf0a 100644 --- a/dbms/src/Storages/Transaction/RegionMeta.h +++ b/dbms/src/Storages/Transaction/RegionMeta.h @@ -58,10 +58,7 @@ class RegionMeta const UInt64 applied_term_, raft_serverpb::RegionLocalState region_state_); - RegionMeta( - metapb::Peer peer_, - metapb::Region region, - raft_serverpb::RaftApplyState apply_state_); + RegionMeta(metapb::Peer peer_, metapb::Region region, raft_serverpb::RaftApplyState apply_state_); RegionMeta(RegionMeta && rhs); diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index abca4971df1..07d2cde857c 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -91,7 +91,10 @@ void RegionPersister::doPersist(const Region & region, const RegionTaskLock * lo doPersist(region_buffer, region_manager.genRegionTaskLock(region.id()), region); } -void RegionPersister::doPersist(RegionCacheWriteElement & region_write_buffer, const RegionTaskLock &, const Region & region) +void RegionPersister::doPersist( + RegionCacheWriteElement & region_write_buffer, + const RegionTaskLock &, + const Region & region) { auto & [region_id, buffer, region_size, applied_index] = region_write_buffer; @@ -141,17 +144,19 @@ void RegionPersister::forceTransformKVStoreV2toV3() const auto & page_transform_entry = page_reader->getPageEntry(page.page_id); if (!page_transform_entry.field_offsets.empty()) { - throw Exception(fmt::format("Can't transform kvstore from V2 to V3, [page_id={}] {}", - page.page_id, - page_transform_entry.toDebugString()), - ErrorCodes::LOGICAL_ERROR); + throw Exception( + fmt::format( + "Can't transform kvstore from V2 to V3, [page_id={}] {}", + page.page_id, + page_transform_entry.toDebugString()), + ErrorCodes::LOGICAL_ERROR); } - write_batch_transform.putPage(page.page_id, // - page_transform_entry.tag, - std::make_shared(page.data.begin(), - page.data.size()), - page.data.size()); + write_batch_transform.putPage( + page.page_id, // + page_transform_entry.tag, + std::make_shared(page.data.begin(), page.data.size()), + page.data.size()); // Will rewrite into V3 one by one. // The region data is big. It is not a good idea to combine pages. @@ -167,7 +172,10 @@ void RegionPersister::forceTransformKVStoreV2toV3() page_writer->writeIntoV2(std::move(write_batch_del_v2), nullptr); } -RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyHelper * proxy_helper, PageStorageConfig config) +RegionMap RegionPersister::restore( + PathPool & path_pool, + const TiFlashRaftProxyHelper * proxy_helper, + PageStorageConfig config) { { auto delegator = path_pool.getPSDiskDelegatorRaft(); @@ -190,8 +198,21 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH provider, global_context.getPSBackgroundPool()); page_storage_v2->restore(); - page_writer = std::make_shared(run_mode, StorageType::KVStore, page_storage_v2, /*storage_v3_*/ nullptr, /*uni_ps_*/ nullptr); - page_reader = std::make_shared(run_mode, NullspaceID, StorageType::KVStore, ns_id, page_storage_v2, /*storage_v3_*/ nullptr, /*uni_ps_*/ nullptr, /*readlimiter*/ global_context.getReadLimiter()); + page_writer = std::make_shared( + run_mode, + StorageType::KVStore, + page_storage_v2, + /*storage_v3_*/ nullptr, + /*uni_ps_*/ nullptr); + page_reader = std::make_shared( + run_mode, + NullspaceID, + StorageType::KVStore, + ns_id, + page_storage_v2, + /*storage_v3_*/ nullptr, + /*uni_ps_*/ nullptr, + /*readlimiter*/ global_context.getReadLimiter()); break; } case PageStorageRunMode::ONLY_V3: @@ -204,8 +225,21 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH config, provider); page_storage_v3->restore(); - page_writer = std::make_shared(run_mode, StorageType::KVStore, /*storage_v2_*/ nullptr, page_storage_v3, /*uni_ps_*/ nullptr); - page_reader = std::make_shared(run_mode, NullspaceID, StorageType::KVStore, ns_id, /*storage_v2_*/ nullptr, page_storage_v3, /*uni_ps_*/ nullptr, global_context.getReadLimiter()); + page_writer = std::make_shared( + run_mode, + StorageType::KVStore, + /*storage_v2_*/ nullptr, + page_storage_v3, + /*uni_ps_*/ nullptr); + page_reader = std::make_shared( + run_mode, + NullspaceID, + StorageType::KVStore, + ns_id, + /*storage_v2_*/ nullptr, + page_storage_v3, + /*uni_ps_*/ nullptr, + global_context.getReadLimiter()); break; } case PageStorageRunMode::MIX_MODE: @@ -231,21 +265,42 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH if (const auto & kvstore_remain_pages = page_storage_v2->getNumberOfPages(); kvstore_remain_pages != 0) { - page_writer = std::make_shared(run_mode, StorageType::KVStore, page_storage_v2, page_storage_v3, /*uni_ps_*/ nullptr); - page_reader = std::make_shared(run_mode, NullspaceID, StorageType::KVStore, ns_id, page_storage_v2, page_storage_v3, /*uni_ps_*/ nullptr, global_context.getReadLimiter()); - - LOG_INFO(log, "Current kvstore transform to V3 begin [pages_before_transform={}]", kvstore_remain_pages); + page_writer = std::make_shared( + run_mode, + StorageType::KVStore, + page_storage_v2, + page_storage_v3, + /*uni_ps_*/ nullptr); + page_reader = std::make_shared( + run_mode, + NullspaceID, + StorageType::KVStore, + ns_id, + page_storage_v2, + page_storage_v3, + /*uni_ps_*/ nullptr, + global_context.getReadLimiter()); + + LOG_INFO( + log, + "Current kvstore transform to V3 begin [pages_before_transform={}]", + kvstore_remain_pages); forceTransformKVStoreV2toV3(); const auto & kvstore_remain_pages_after_transform = page_storage_v2->getNumberOfPages(); - LOG_INFO(log, "Current kvstore transform to V3 finished. [ns_id={}] [done={}] [pages_before_transform={}] [pages_after_transform={}]", // - ns_id, - kvstore_remain_pages_after_transform == 0, - kvstore_remain_pages, - kvstore_remain_pages_after_transform); + LOG_INFO( + log, + "Current kvstore transform to V3 finished. [ns_id={}] [done={}] [pages_before_transform={}] " + "[pages_after_transform={}]", // + ns_id, + kvstore_remain_pages_after_transform == 0, + kvstore_remain_pages, + kvstore_remain_pages_after_transform); if (kvstore_remain_pages_after_transform != 0) { - throw Exception("KVStore transform failed. Still have some data exist in V2", ErrorCodes::LOGICAL_ERROR); + throw Exception( + "KVStore transform failed. Still have some data exist in V2", + ErrorCodes::LOGICAL_ERROR); } } else // no need do transform @@ -259,8 +314,21 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH page_storage_v2 = nullptr; // Must use PageStorageRunMode::ONLY_V3 here. - page_writer = std::make_shared(PageStorageRunMode::ONLY_V3, StorageType::KVStore, /*storage_v2_*/ nullptr, page_storage_v3, /*uni_ps_*/ nullptr); - page_reader = std::make_shared(PageStorageRunMode::ONLY_V3, NullspaceID, StorageType::KVStore, ns_id, /*storage_v2_*/ nullptr, page_storage_v3, /*uni_ps_*/ nullptr, global_context.getReadLimiter()); + page_writer = std::make_shared( + PageStorageRunMode::ONLY_V3, + StorageType::KVStore, + /*storage_v2_*/ nullptr, + page_storage_v3, + /*uni_ps_*/ nullptr); + page_reader = std::make_shared( + PageStorageRunMode::ONLY_V3, + NullspaceID, + StorageType::KVStore, + ns_id, + /*storage_v2_*/ nullptr, + page_storage_v3, + /*uni_ps_*/ nullptr, + global_context.getReadLimiter()); run_mode = PageStorageRunMode::ONLY_V3; break; @@ -268,8 +336,21 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH case PageStorageRunMode::UNI_PS: { auto uni_ps = global_context.getWriteNodePageStorage(); - page_writer = std::make_shared(run_mode, StorageType::KVStore, /*storage_v2_*/ nullptr, /*storage_v3_*/ nullptr, uni_ps); - page_reader = std::make_shared(run_mode, NullspaceID, StorageType::KVStore, ns_id, /*storage_v2_*/ nullptr, /*storage_v3_*/ nullptr, uni_ps, global_context.getReadLimiter()); + page_writer = std::make_shared( + run_mode, + StorageType::KVStore, + /*storage_v2_*/ nullptr, + /*storage_v3_*/ nullptr, + uni_ps); + page_reader = std::make_shared( + run_mode, + NullspaceID, + StorageType::KVStore, + ns_id, + /*storage_v2_*/ nullptr, + /*storage_v3_*/ nullptr, + uni_ps, + global_context.getReadLimiter()); break; } } diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index bd1d58c34ec..04c45518685 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -76,7 +76,9 @@ class RegionTable : private boost::noncopyable public: struct InternalRegion { - InternalRegion(const RegionID region_id_, const std::pair & range_in_table_) + InternalRegion( + const RegionID region_id_, + const std::pair & range_in_table_) : region_id(region_id_) , range_in_table(range_in_table_) {} @@ -120,30 +122,35 @@ class RegionTable : private boost::noncopyable // The original name for this function is tryFlushRegion. RegionDataReadInfoList tryWriteBlockByRegionAndFlush(const RegionPtrWithBlock & region); - void handleInternalRegionsByTable(KeyspaceID keyspace_id, TableID table_id, std::function && callback) const; + void handleInternalRegionsByTable( + KeyspaceID keyspace_id, + TableID table_id, + std::function && callback) const; std::vector> getRegionsByTable(KeyspaceID keyspace_id, TableID table_id) const; /// Write the data of the given region into the table with the given table ID, fill the data list for outer to remove. /// Will trigger schema sync on read error for only once, /// assuming that newer schema can always apply to older data by setting force_decode to true in RegionBlockReader::read. /// Note that table schema must be keep unchanged throughout the process of read then write, we take good care of the lock. - static DM::WriteResult writeBlockByRegion(Context & context, - const RegionPtrWithBlock & region, - RegionDataReadInfoList & data_list_to_remove, - const LoggerPtr & log, - bool lock_region = true); + static DM::WriteResult writeBlockByRegion( + Context & context, + const RegionPtrWithBlock & region, + RegionDataReadInfoList & data_list_to_remove, + const LoggerPtr & log, + bool lock_region = true); /// Check transaction locks in region, and write committed data in it into storage engine if check passed. Otherwise throw an LockException. /// The write logic is the same as #writeBlockByRegion, with some extra checks about region version and conf_version. using ResolveLocksAndWriteRegionRes = std::variant; - static ResolveLocksAndWriteRegionRes resolveLocksAndWriteRegion(TMTContext & tmt, - const TiDB::TableID table_id, - const RegionPtr & region, - const Timestamp start_ts, - const std::unordered_set * bypass_lock_ts, - RegionVersion region_version, - RegionVersion conf_version, - const LoggerPtr & log); + static ResolveLocksAndWriteRegionRes resolveLocksAndWriteRegion( + TMTContext & tmt, + const TiDB::TableID table_id, + const RegionPtr & region, + const Timestamp start_ts, + const std::unordered_set * bypass_lock_ts, + RegionVersion region_version, + RegionVersion conf_version, + const LoggerPtr & log); public: // safe ts is maintained by check_leader RPC (https://github.com/tikv/tikv/blob/1ea26a2ac8761af356cc5c0825eb89a0b8fc9749/components/resolved_ts/src/advance.rs#L262), @@ -247,9 +254,7 @@ struct RegionPtrWithSnapshotFiles using Base = RegionPtr; /// can accept const ref of RegionPtr without cache - RegionPtrWithSnapshotFiles( - const Base & base_, - std::vector && external_files_ = {}); + RegionPtrWithSnapshotFiles(const Base & base_, std::vector && external_files_ = {}); /// to be compatible with usage as RegionPtr. Base::element_type * operator->() const { return base.operator->(); } diff --git a/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp b/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp index 09ee7bdfc37..bd5d0df7eb5 100644 --- a/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp +++ b/dbms/src/Storages/Transaction/RegionsRangeIndex.cpp @@ -55,7 +55,9 @@ void RegionsRangeIndex::remove(const RegionRange & range, RegionID region_id) for (auto it = begin_it; it != end_it; ++it) { if (it->second.region_map.erase(region_id) == 0) - throw Exception(std::string(__PRETTY_FUNCTION__) + ": not found region_id=" + toString(region_id), ErrorCodes::LOGICAL_ERROR); + throw Exception( + std::string(__PRETTY_FUNCTION__) + ": not found region_id=" + toString(region_id), + ErrorCodes::LOGICAL_ERROR); } tryMergeEmpty(begin_it); } @@ -73,7 +75,8 @@ RegionMap RegionsRangeIndex::findByRangeOverlap(const RegionRange & range) const return res; } -std::variant RegionsRangeIndex::findByRangeChecked(const RegionRange & range) const +std::variant RegionsRangeIndex::findByRangeChecked( + const RegionRange & range) const { auto begin_it = root.lower_bound(range.first); auto end_it = root.lower_bound(range.second); diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index 18a386e6118..a0e9f8b5b5f 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -50,7 +50,11 @@ const int64_t DEFAULT_READ_INDEX_WORKER_TICK_MS = 10; namespace { -std::shared_ptr createSchemaSyncer(bool exist_pd_addr, bool for_unit_test, const KVClusterPtr & cluster, bool disaggregated_compute_mode) +std::shared_ptr createSchemaSyncer( + bool exist_pd_addr, + bool for_unit_test, + const KVClusterPtr & cluster, + bool disaggregated_compute_mode) { // Doesn't need SchemaSyncer for tiflash_compute mode. if (disaggregated_compute_mode) @@ -116,25 +120,38 @@ void monitorMPPTasks(std::shared_ptr monitor) void startMonitorMPPTaskThread(const MPPTaskManagerPtr & manager) { - newThreadManager()->scheduleThenDetach(false, "MPPTask-Moniter", [monitor = manager->getMPPTaskMonitor()] { monitorMPPTasks(monitor); }); + newThreadManager()->scheduleThenDetach(false, "MPPTask-Moniter", [monitor = manager->getMPPTaskMonitor()] { + monitorMPPTasks(monitor); + }); } } // namespace -TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config, const pingcap::ClusterConfig & cluster_config) +TMTContext::TMTContext( + Context & context_, + const TiFlashRaftConfig & raft_config, + const pingcap::ClusterConfig & cluster_config) : context(context_) - , kvstore(context_.getSharedContextDisagg()->isDisaggregatedComputeMode() && context_.getSharedContextDisagg()->use_autoscaler ? nullptr : std::make_shared(context)) + , kvstore( + context_.getSharedContextDisagg()->isDisaggregatedComputeMode() + && context_.getSharedContextDisagg()->use_autoscaler + ? nullptr + : std::make_shared(context)) , region_table(context) , background_service(nullptr) , gc_manager(context) - , cluster(raft_config.pd_addrs.empty() ? std::make_shared() - : std::make_shared(raft_config.pd_addrs, cluster_config)) + , cluster( + raft_config.pd_addrs.empty() ? std::make_shared() + : std::make_shared(raft_config.pd_addrs, cluster_config)) , ignore_databases(raft_config.ignore_databases) - , schema_sync_manager(createSchemaSyncer(!raft_config.pd_addrs.empty(), raft_config.for_unit_test, cluster, context_.getSharedContextDisagg()->isDisaggregatedComputeMode())) - , mpp_task_manager(std::make_shared( - std::make_unique( - context.getSettingsRef().task_scheduler_thread_soft_limit, - context.getSettingsRef().task_scheduler_thread_hard_limit, - context.getSettingsRef().task_scheduler_active_set_soft_limit))) + , schema_sync_manager(createSchemaSyncer( + !raft_config.pd_addrs.empty(), + raft_config.for_unit_test, + cluster, + context_.getSharedContextDisagg()->isDisaggregatedComputeMode())) + , mpp_task_manager(std::make_shared(std::make_unique( + context.getSettingsRef().task_scheduler_thread_soft_limit, + context.getSettingsRef().task_scheduler_thread_hard_limit, + context.getSettingsRef().task_scheduler_active_set_soft_limit))) , engine(raft_config.engine) , batch_read_index_timeout_ms(DEFAULT_BATCH_READ_INDEX_TIMEOUT_MS) , wait_index_timeout_ms(DEFAULT_WAIT_INDEX_TIMEOUT_MS) @@ -143,7 +160,8 @@ TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config { startMonitorMPPTaskThread(mpp_task_manager); - if (!raft_config.pd_addrs.empty() && S3::ClientFactory::instance().isEnabled() && !context.getSharedContextDisagg()->isDisaggregatedComputeMode()) + if (!raft_config.pd_addrs.empty() && S3::ClientFactory::instance().isEnabled() + && !context.getSharedContextDisagg()->isDisaggregatedComputeMode()) { etcd_client = Etcd::Client::create(cluster->pd_client, cluster_config); s3gc_owner = OwnerManager::createS3GCOwner(context, /*id*/ raft_config.advertise_engine_addr, etcd_client); @@ -165,20 +183,32 @@ TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config } else { - LOG_WARNING(Logger::get(), "Unknown remote gc method from settings, using default method, value={} remote_gc_method={}", gc_method_int, magic_enum::enum_name(remote_gc_config.method)); + LOG_WARNING( + Logger::get(), + "Unknown remote gc method from settings, using default method, value={} remote_gc_method={}", + gc_method_int, + magic_enum::enum_name(remote_gc_config.method)); } } - remote_gc_config.interval_seconds = context.getSettingsRef().remote_gc_interval_seconds; // TODO: make it reloadable + remote_gc_config.interval_seconds + = context.getSettingsRef().remote_gc_interval_seconds; // TODO: make it reloadable remote_gc_config.verify_locks = context.getSettingsRef().remote_gc_verify_consistency > 0; // set the gc_method so that S3LockService can set tagging when create delmark S3::ClientFactory::instance().gc_method = remote_gc_config.method; - s3gc_manager = std::make_unique(context, cluster->pd_client, s3gc_owner, s3lock_client, remote_gc_config); + s3gc_manager = std::make_unique( + context, + cluster->pd_client, + s3gc_owner, + s3lock_client, + remote_gc_config); } } TMTContext::~TMTContext() = default; -void TMTContext::updateSecurityConfig(const TiFlashRaftConfig & raft_config, const pingcap::ClusterConfig & cluster_config) +void TMTContext::updateSecurityConfig( + const TiFlashRaftConfig & raft_config, + const pingcap::ClusterConfig & cluster_config) { if (!raft_config.pd_addrs.empty()) { @@ -195,7 +225,8 @@ void TMTContext::updateSecurityConfig(const TiFlashRaftConfig & raft_config, con void TMTContext::restore(PathPool & path_pool, const TiFlashRaftProxyHelper * proxy_helper) { // For tiflash_compute mode, kvstore should be nullptr, no need to restore region_table. - if (context.getSharedContextDisagg()->isDisaggregatedComputeMode() && context.getSharedContextDisagg()->use_autoscaler) + if (context.getSharedContextDisagg()->isDisaggregatedComputeMode() + && context.getSharedContextDisagg()->use_autoscaler) return; kvstore->restore(path_pool, proxy_helper); @@ -335,7 +366,8 @@ const std::unordered_set & TMTContext::getIgnoreDatabases() const void TMTContext::reloadConfig(const Poco::Util::AbstractConfiguration & config) { - if (context.getSharedContextDisagg()->isDisaggregatedComputeMode() && context.getSharedContextDisagg()->use_autoscaler) + if (context.getSharedContextDisagg()->isDisaggregatedComputeMode() + && context.getSharedContextDisagg()->use_autoscaler) return; static constexpr const char * COMPACT_LOG_MIN_PERIOD = "flash.compact_log_min_period"; @@ -348,12 +380,14 @@ void TMTContext::reloadConfig(const Poco::Util::AbstractConfiguration & config) static constexpr const char * READ_INDEX_WORKER_TICK_MS = "flash.read_index_worker_tick_ms"; // default config about compact-log: period 120s, rows 40k, bytes 32MB. - getKVStore()->setRegionCompactLogConfig(std::max(config.getUInt64(COMPACT_LOG_MIN_PERIOD, 120), 1), - std::max(config.getUInt64(COMPACT_LOG_MIN_ROWS, 40 * 1024), 1), - std::max(config.getUInt64(COMPACT_LOG_MIN_BYTES, 32 * 1024 * 1024), 1), - std::max(config.getUInt64(COMPACT_LOG_MIN_GAP, 200), 1)); + getKVStore()->setRegionCompactLogConfig( + std::max(config.getUInt64(COMPACT_LOG_MIN_PERIOD, 120), 1), + std::max(config.getUInt64(COMPACT_LOG_MIN_ROWS, 40 * 1024), 1), + std::max(config.getUInt64(COMPACT_LOG_MIN_BYTES, 32 * 1024 * 1024), 1), + std::max(config.getUInt64(COMPACT_LOG_MIN_GAP, 200), 1)); { - batch_read_index_timeout_ms = config.getUInt64(BATCH_READ_INDEX_TIMEOUT_MS, DEFAULT_BATCH_READ_INDEX_TIMEOUT_MS); + batch_read_index_timeout_ms + = config.getUInt64(BATCH_READ_INDEX_TIMEOUT_MS, DEFAULT_BATCH_READ_INDEX_TIMEOUT_MS); wait_index_timeout_ms = config.getUInt64(WAIT_INDEX_TIMEOUT_MS, DEFAULT_WAIT_INDEX_TIMEOUT_MS); wait_region_ready_timeout_sec = ({ int64_t t = config.getInt64(WAIT_REGION_READY_TIMEOUT_SEC, /*20min*/ DEFAULT_WAIT_REGION_READY_TIMEOUT_SEC); @@ -365,7 +399,8 @@ void TMTContext::reloadConfig(const Poco::Util::AbstractConfiguration & config) { LOG_INFO( Logger::get(), - "read-index timeout: {}ms; wait-index timeout: {}ms; wait-region-ready timeout: {}s; read-index-worker-tick: {}ms", + "read-index timeout: {}ms; wait-index timeout: {}ms; wait-region-ready timeout: {}s; " + "read-index-worker-tick: {}ms", batchReadIndexTimeout(), waitIndexTimeout(), waitRegionReadyTimeout(), @@ -430,7 +465,8 @@ const std::string & IntoStoreStatusName(TMTContext::StoreStatus status) }; static const std::string Unknown = "Unknown"; auto idx = static_cast(status); - return idx > static_cast(TMTContext::StoreStatus::_MIN) && idx < static_cast(TMTContext::StoreStatus::_MAX) + return idx > static_cast(TMTContext::StoreStatus::_MIN) + && idx < static_cast(TMTContext::StoreStatus::_MAX) ? StoreStatusName[idx - 1] : Unknown; } diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 42833283ab9..759421e2fe6 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -44,7 +44,9 @@ try // CompactLog always returns true now, even if we can't do a flush. // We use a tryFlushData to pre-filter. raft_cmdpb::AdminResponse response; - ASSERT_EQ(kvs.handleAdminRaftCmd(std::move(request), std::move(response), 1, 5, 1, ctx.getTMTContext()), EngineStoreApplyRes::Persist); + ASSERT_EQ( + kvs.handleAdminRaftCmd(std::move(request), std::move(response), 1, 5, 1, ctx.getTMTContext()), + EngineStoreApplyRes::Persist); // Filter ASSERT_EQ(kvs.tryFlushRegionData(1, false, false, ctx.getTMTContext(), 0, 0, 0, 0), false); @@ -59,9 +61,7 @@ TEST_F(RegionKVStoreTest, ReadIndex) // Start mock proxy in other thread std::atomic_bool over{false}; - auto proxy_runner = std::thread([&]() { - proxy_instance->testRunNormal(over); - }); + auto proxy_runner = std::thread([&]() { proxy_instance->testRunNormal(over); }); KVStore & kvs = getKVS(); ASSERT_EQ(kvs.getProxyHelper(), proxy_helper.get()); @@ -94,23 +94,21 @@ TEST_F(RegionKVStoreTest, ReadIndex) ASSERT_EQ(e.message(), "`fn_handle_batch_read_index` is deprecated"); } } - kvs.initReadIndexWorkers( - []() { - return std::chrono::milliseconds(10); - }, - 1); + kvs.initReadIndexWorkers([]() { return std::chrono::milliseconds(10); }, 1); ASSERT_NE(kvs.read_index_worker_manager, nullptr); } { { // Normal async notifier kvs.asyncRunReadIndexWorkers(); - SCOPE_EXIT({ - kvs.stopReadIndexWorkers(); - }); + SCOPE_EXIT({ kvs.stopReadIndexWorkers(); }); UInt64 tar_region_id = 9; - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {tar_region_id}, {{{RecordKVFormat::genKey(2, 0), RecordKVFormat::genKey(2, 10)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {tar_region_id}, + {{{RecordKVFormat::genKey(2, 0), RecordKVFormat::genKey(2, 10)}}}); { ASSERT_EQ(proxy_instance->regions.at(tar_region_id)->getLatestCommitIndex(), 5); proxy_instance->regions.at(tar_region_id)->updateCommitIndex(66); @@ -129,16 +127,12 @@ TEST_F(RegionKVStoreTest, ReadIndex) ASSERT_EQ(notifier.blockedWaitFor(std::chrono::milliseconds(1000 * 3600)), AsyncNotifier::Status::Normal); std::this_thread::sleep_for(std::chrono::milliseconds(2)); auto tar = kvs.getRegion(tar_region_id); - ASSERT_EQ( - tar->handleWriteRaftCmd({}, 66, 6, ctx.getTMTContext()).first, - EngineStoreApplyRes::None); + ASSERT_EQ(tar->handleWriteRaftCmd({}, 66, 6, ctx.getTMTContext()).first, EngineStoreApplyRes::None); } { // Async notifier error kvs.asyncRunReadIndexWorkers(); - SCOPE_EXIT({ - kvs.stopReadIndexWorkers(); - }); + SCOPE_EXIT({ kvs.stopReadIndexWorkers(); }); auto tar_region_id = 9; { @@ -153,11 +147,15 @@ TEST_F(RegionKVStoreTest, ReadIndex) const std::atomic_size_t terminate_signals_counter{}; std::thread t([&]() { notifier.wake(); - WaitCheckRegionReady(ctx.getTMTContext(), kvs, terminate_signals_counter, 1 / 1000.0, 2 / 1000.0, 5 / 1000.0); - }); - SCOPE_EXIT({ - t.join(); + WaitCheckRegionReady( + ctx.getTMTContext(), + kvs, + terminate_signals_counter, + 1 / 1000.0, + 2 / 1000.0, + 5 / 1000.0); }); + SCOPE_EXIT({ t.join(); }); ASSERT_EQ(notifier.blockedWaitFor(std::chrono::milliseconds(1000 * 3600)), AsyncNotifier::Status::Normal); } @@ -165,9 +163,7 @@ TEST_F(RegionKVStoreTest, ReadIndex) // Note `batchReadIndex` always returns latest committed index in our mock class. // See `RawMockReadIndexTask::poll`. kvs.asyncRunReadIndexWorkers(); - SCOPE_EXIT({ - kvs.stopReadIndexWorkers(); - }); + SCOPE_EXIT({ kvs.stopReadIndexWorkers(); }); { auto region = kvs.getRegion(1); @@ -217,10 +213,10 @@ TEST_F(RegionKVStoreTest, ReadIndex) auto r = region->waitIndex(667 + 1, 100000, []() { return true; }); ASSERT_EQ(std::get<0>(r), WaitIndexResult::Finished); }); - SCOPE_EXIT({ - t.join(); - }); - ASSERT_EQ(notifier.blockedWaitFor(std::chrono::milliseconds(1000 * 3600)), AsyncNotifier::Status::Normal); + SCOPE_EXIT({ t.join(); }); + ASSERT_EQ( + notifier.blockedWaitFor(std::chrono::milliseconds(1000 * 3600)), + AsyncNotifier::Status::Normal); std::this_thread::sleep_for(std::chrono::milliseconds(2)); region->handleWriteRaftCmd({}, 667 + 1, 6, ctx.getTMTContext()); } @@ -243,13 +239,9 @@ void RegionKVStoreTest::testRaftMergeRollback(KVStore & kvs, TMTContext & tmt) auto target_region = kvs.getRegion(1); - auto && [request, response] = MockRaftStoreProxy::composePrepareMerge(target_region->cloneMetaRegion(), source_region->appliedIndex()); - kvs.handleAdminRaftCmd(std::move(request), - std::move(response), - region_id, - 31, - 6, - tmt); + auto && [request, response] + = MockRaftStoreProxy::composePrepareMerge(target_region->cloneMetaRegion(), source_region->appliedIndex()); + kvs.handleAdminRaftCmd(std::move(request), std::move(response), region_id, 31, 6, tmt); ASSERT_TRUE(source_region->isMerging()); } { @@ -261,12 +253,7 @@ void RegionKVStoreTest::testRaftMergeRollback(KVStore & kvs, TMTContext & tmt) { raft_cmdpb::AdminRequest first_request = request; raft_cmdpb::AdminResponse first_response = response; - kvs.handleAdminRaftCmd(std::move(first_request), - std::move(first_response), - region_id, - 32, - 6, - tmt); + kvs.handleAdminRaftCmd(std::move(first_request), std::move(first_response), region_id, 32, 6, tmt); ASSERT_TRUE(false); } catch (Exception & e) @@ -279,12 +266,7 @@ void RegionKVStoreTest::testRaftMergeRollback(KVStore & kvs, TMTContext & tmt) region->meta.region_state.getMutMergeState().set_commit(1234); try { - kvs.handleAdminRaftCmd(std::move(request), - std::move(response), - region_id, - 32, - 6, - tmt); + kvs.handleAdminRaftCmd(std::move(request), std::move(response), region_id, 32, 6, tmt); ASSERT_TRUE(false); } catch (Exception & e) @@ -296,12 +278,7 @@ void RegionKVStoreTest::testRaftMergeRollback(KVStore & kvs, TMTContext & tmt) { auto region = kvs.getRegion(region_id); auto && [request, response] = MockRaftStoreProxy::composeRollbackMerge(region->getMergeState().commit()); - kvs.handleAdminRaftCmd(std::move(request), - std::move(response), - region_id, - 32, - 6, - tmt); + kvs.handleAdminRaftCmd(std::move(request), std::move(response), region_id, 32, 6, tmt); ASSERT_EQ(region->peerState(), raft_serverpb::PeerState::Normal); } } @@ -311,13 +288,25 @@ static void testRaftSplit(KVStore & kvs, TMTContext & tmt, std::unique_ptrinsert("lock", RecordKVFormat::genKey(table_id, 3), RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); + region->insert( + "lock", + RecordKVFormat::genKey(table_id, 3), + RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); region->insert("default", RecordKVFormat::genKey(table_id, 3, 5), TiKVValue("value1")); - region->insert("write", RecordKVFormat::genKey(table_id, 3, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); - - region->insert("lock", RecordKVFormat::genKey(table_id, 8), RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 3, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + + region->insert( + "lock", + RecordKVFormat::genKey(table_id, 8), + RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); region->insert("default", RecordKVFormat::genKey(table_id, 8, 5), TiKVValue("value1")); - region->insert("write", RecordKVFormat::genKey(table_id, 8, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 8, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); ASSERT_EQ(region->dataInfo(), "[write 2 lock 2 default 2 ]"); } @@ -327,9 +316,14 @@ static void testRaftSplit(KVStore & kvs, TMTContext & tmt, std::unique_ptrmutMeta().getMetaRegion().region_epoch(); const auto & ori_source_range = source_region->getRange()->comparableKeys(); - RegionRangeKeys::RegionRange new_source_range = RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 5), RecordKVFormat::genKey(1, 10)); - RegionRangeKeys::RegionRange new_target_range = RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5)); - auto && [request, response] = MockRaftStoreProxy::composeBatchSplit({region_id, region_id2}, regionRangeToEncodeKeys(new_source_range, new_target_range), old_epoch); + RegionRangeKeys::RegionRange new_source_range + = RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 5), RecordKVFormat::genKey(1, 10)); + RegionRangeKeys::RegionRange new_target_range + = RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5)); + auto && [request, response] = MockRaftStoreProxy::composeBatchSplit( + {region_id, region_id2}, + regionRangeToEncodeKeys(new_source_range, new_target_range), + old_epoch); kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest(request), raft_cmdpb::AdminResponse(response), 1, 20, 5, tmt); { auto mmp = kvs.getRegionsByRangeOverlap(new_target_range); @@ -354,12 +348,24 @@ static void testRaftSplit(KVStore & kvs, TMTContext & tmt, std::unique_ptrinsert("lock", RecordKVFormat::genKey(table_id, 3), RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); + region->insert( + "lock", + RecordKVFormat::genKey(table_id, 3), + RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); region->insert("default", RecordKVFormat::genKey(table_id, 3, 5), TiKVValue("value1")); - region->insert("write", RecordKVFormat::genKey(table_id, 3, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); - region->insert("lock", RecordKVFormat::genKey(table_id, 8), RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 3, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + region->insert( + "lock", + RecordKVFormat::genKey(table_id, 8), + RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); region->insert("default", RecordKVFormat::genKey(table_id, 8, 5), TiKVValue("value1")); - region->insert("write", RecordKVFormat::genKey(table_id, 8, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 8, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); ASSERT_EQ(region->dataInfo(), "[write 2 lock 2 default 2 ]"); } @@ -398,17 +404,29 @@ void RegionKVStoreTest::testRaftMerge(KVStore & kvs, TMTContext & tmt) { auto region = kvs.getRegion(1); auto table_id = 1; - region->insert("lock", RecordKVFormat::genKey(table_id, 6), RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); + region->insert( + "lock", + RecordKVFormat::genKey(table_id, 6), + RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); region->insert("default", RecordKVFormat::genKey(table_id, 6, 5), TiKVValue("value1")); - region->insert("write", RecordKVFormat::genKey(table_id, 6, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 6, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); ASSERT_EQ(region->dataInfo(), "[write 1 lock 1 default 1 ]"); } { auto region = kvs.getRegion(region_id); auto table_id = 1; - region->insert("lock", RecordKVFormat::genKey(table_id, 2), RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); + region->insert( + "lock", + RecordKVFormat::genKey(table_id, 2), + RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); region->insert("default", RecordKVFormat::genKey(table_id, 2, 5), TiKVValue("value1")); - region->insert("write", RecordKVFormat::genKey(table_id, 2, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 2, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); ASSERT_EQ(region->dataInfo(), "[write 1 lock 1 default 1 ]"); } } @@ -417,13 +435,9 @@ void RegionKVStoreTest::testRaftMerge(KVStore & kvs, TMTContext & tmt) auto region_id = 7; auto source_region = kvs.getRegion(region_id); auto target_region = kvs.getRegion(1); - auto && [request, response] = MockRaftStoreProxy::composePrepareMerge(target_region->cloneMetaRegion(), source_region->appliedIndex()); - kvs.handleAdminRaftCmd(std::move(request), - std::move(response), - source_region->id(), - 35, - 6, - tmt); + auto && [request, response] + = MockRaftStoreProxy::composePrepareMerge(target_region->cloneMetaRegion(), source_region->appliedIndex()); + kvs.handleAdminRaftCmd(std::move(request), std::move(response), source_region->id(), 35, 6, tmt); ASSERT_EQ(source_region->peerState(), raft_serverpb::PeerState::Merging); } @@ -431,13 +445,16 @@ void RegionKVStoreTest::testRaftMerge(KVStore & kvs, TMTContext & tmt) auto source_id = 7, target_id = 1; auto source_region = kvs.getRegion(source_id); - auto && [request, response] = MockRaftStoreProxy::composeCommitMerge(source_region->cloneMetaRegion(), source_region->appliedIndex()); + auto && [request, response] + = MockRaftStoreProxy::composeCommitMerge(source_region->cloneMetaRegion(), source_region->appliedIndex()); source_region->setStateApplying(); source_region->makeRaftCommandDelegate(kvs.genTaskLock()); const auto & source_region_meta_delegate = source_region->meta.makeRaftCommandDelegate(); try { - kvs.getRegion(target_id)->meta.makeRaftCommandDelegate().checkBeforeCommitMerge(request, source_region_meta_delegate); + kvs.getRegion(target_id)->meta.makeRaftCommandDelegate().checkBeforeCommitMerge( + request, + source_region_meta_delegate); ASSERT_TRUE(false); } catch (Exception & e) @@ -450,7 +467,9 @@ void RegionKVStoreTest::testRaftMerge(KVStore & kvs, TMTContext & tmt) } try { - kvs.getRegion(target_id)->meta.makeRaftCommandDelegate().checkBeforeCommitMerge(request, source_region_meta_delegate); + kvs.getRegion(target_id)->meta.makeRaftCommandDelegate().checkBeforeCommitMerge( + request, + source_region_meta_delegate); ASSERT_TRUE(false); } catch (Exception & e) @@ -462,23 +481,26 @@ void RegionKVStoreTest::testRaftMerge(KVStore & kvs, TMTContext & tmt) { auto source_id = 7, target_id = 1; auto source_region = kvs.getRegion(source_id); - auto && [request, response] = MockRaftStoreProxy::composeCommitMerge(source_region->cloneMetaRegion(), source_region->appliedIndex()); + auto && [request, response] + = MockRaftStoreProxy::composeCommitMerge(source_region->cloneMetaRegion(), source_region->appliedIndex()); { auto mmp = kvs.getRegionsByRangeOverlap(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); ASSERT_TRUE(mmp.count(target_id) != 0); ASSERT_EQ(mmp.size(), 2); } - kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest(request), - raft_cmdpb::AdminResponse(response), - target_id, - 36, - 6, - tmt); + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest(request), + raft_cmdpb::AdminResponse(response), + target_id, + 36, + 6, + tmt); ASSERT_EQ(kvs.getRegion(source_id), nullptr); { - auto mmp = kvs.getRegionsByRangeOverlap(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5))); + auto mmp = kvs.getRegionsByRangeOverlap( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5))); ASSERT_TRUE(mmp.count(1) != 0); ASSERT_EQ(mmp.size(), 1); } @@ -486,24 +508,28 @@ void RegionKVStoreTest::testRaftMerge(KVStore & kvs, TMTContext & tmt) // add 7 back auto task_lock = kvs.genTaskLock(); auto lock = kvs.genRegionMgrWriteLock(task_lock); - auto region = makeRegion(7, RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5), kvs.getProxyHelper()); + auto region + = makeRegion(7, RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5), kvs.getProxyHelper()); lock.regions.emplace(7, region); lock.index.add(region); } { - auto mmp = kvs.getRegionsByRangeOverlap(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5))); + auto mmp = kvs.getRegionsByRangeOverlap( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5))); ASSERT_TRUE(mmp.count(7) != 0); ASSERT_TRUE(mmp.count(1) != 0); ASSERT_EQ(mmp.size(), 2); } - kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest(request), - raft_cmdpb::AdminResponse(response), - target_id, - 36, - 6, - tmt); + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest(request), + raft_cmdpb::AdminResponse(response), + target_id, + 36, + 6, + tmt); { - auto mmp = kvs.getRegionsByRangeOverlap(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5))); + auto mmp = kvs.getRegionsByRangeOverlap( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5))); ASSERT_TRUE(mmp.count(1) != 0); ASSERT_EQ(mmp.size(), 1); } @@ -517,11 +543,19 @@ TEST_F(RegionKVStoreTest, RegionReadWrite) TableID table_id = 100; KVStore & kvs = getKVS(); UInt64 region_id = 1; - proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::make_optional(std::make_pair(RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, 1000)))); + proxy_instance->bootstrapWithRegion( + kvs, + ctx.getTMTContext(), + region_id, + std::make_optional( + std::make_pair(RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, 1000)))); auto region = kvs.getRegion(region_id); { // Test create RegionMeta. - auto meta = RegionMeta(createPeer(2, true), createRegionInfo(666, RecordKVFormat::genKey(0, 0), RecordKVFormat::genKey(0, 1000)), initialApplyState()); + auto meta = RegionMeta( + createPeer(2, true), + createRegionInfo(666, RecordKVFormat::genKey(0, 0), RecordKVFormat::genKey(0, 1000)), + initialApplyState()); ASSERT_EQ(meta.peerId(), 2); } { @@ -531,16 +565,21 @@ TEST_F(RegionKVStoreTest, RegionReadWrite) auto req = GenRegionReadIndexReq(*region, start_ts); ASSERT_EQ(req.ranges().size(), 1); ASSERT_EQ(req.start_ts(), start_ts); - ASSERT_EQ(region->getMetaRegion().region_epoch().DebugString(), - req.context().region_epoch().DebugString()); + ASSERT_EQ(region->getMetaRegion().region_epoch().DebugString(), req.context().region_epoch().DebugString()); ASSERT_EQ(region->getRange()->comparableKeys().first.key, req.ranges()[0].start_key()); ASSERT_EQ(region->getRange()->comparableKeys().second.key, req.ranges()[0].end_key()); } { // Test read committed and lock with CommittedScanner. - region->insert("lock", RecordKVFormat::genKey(table_id, 3), RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); + region->insert( + "lock", + RecordKVFormat::genKey(table_id, 3), + RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20)); region->insert("default", RecordKVFormat::genKey(table_id, 3, 5), TiKVValue("value1")); - region->insert("write", RecordKVFormat::genKey(table_id, 3, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 3, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); ASSERT_EQ(1, region->writeCFCount()); ASSERT_EQ(region->dataInfo(), "[write 1 lock 1 default 1 ]"); { @@ -569,19 +608,27 @@ TEST_F(RegionKVStoreTest, RegionReadWrite) } { // Test duplicate and tryCompactionFilter - region->insert("write", RecordKVFormat::genKey(table_id, 3, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 3, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); ASSERT_EQ(region->dataInfo(), "[write 1 ]"); auto ori_size = region->dataSize(); try { // insert duplicate records - region->insert("write", RecordKVFormat::genKey(table_id, 3, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 3, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5)); ASSERT_TRUE(false); } catch (Exception & e) { - ASSERT_EQ(e.message(), "Found existing key in hex: 7480000000000000FF645F728000000000FF0000030000000000FAFFFFFFFFFFFFFFF7"); + ASSERT_EQ( + e.message(), + "Found existing key in hex: 7480000000000000FF645F728000000000FF0000030000000000FAFFFFFFFFFFFFFFF7"); } ASSERT_EQ(ori_size, region->dataSize()); @@ -590,7 +637,10 @@ TEST_F(RegionKVStoreTest, RegionReadWrite) } { // Test read and delete committed Del record. - region->insert("write", RecordKVFormat::genKey(table_id, 4, 8), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::DelFlag, 5)); + region->insert( + "write", + RecordKVFormat::genKey(table_id, 4, 8), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::DelFlag, 5)); ASSERT_EQ(1, region->writeCFCount()); region->remove("write", RecordKVFormat::genKey(table_id, 4, 8)); ASSERT_EQ(1, region->writeCFCount()); @@ -622,11 +672,7 @@ TEST_F(RegionKVStoreTest, Writes) KVStore & kvs = getKVS(); { // Run without read-index workers - kvs.initReadIndexWorkers( - []() { - return std::chrono::milliseconds(10); - }, - 0); + kvs.initReadIndexWorkers([]() { return std::chrono::milliseconds(10); }, 0); ASSERT_EQ(kvs.read_index_worker_manager, nullptr); kvs.asyncRunReadIndexWorkers(); kvs.stopReadIndexWorkers(); @@ -641,7 +687,13 @@ TEST_F(RegionKVStoreTest, Writes) } { ASSERT_EQ(kvs.getRegion(0), nullptr); - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {1, 2, 3}, {{{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}, {RecordKVFormat::genKey(1, 10), RecordKVFormat::genKey(1, 20)}, {RecordKVFormat::genKey(1, 30), RecordKVFormat::genKey(1, 40)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {1, 2, 3}, + {{{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}, + {RecordKVFormat::genKey(1, 10), RecordKVFormat::genKey(1, 20)}, + {RecordKVFormat::genKey(1, 30), RecordKVFormat::genKey(1, 40)}}}); } { // Test gc region persister @@ -651,12 +703,14 @@ TEST_F(RegionKVStoreTest, Writes) { // Check region range ASSERT_EQ(kvs.regionSize(), 3); - auto mmp = kvs.getRegionsByRangeOverlap(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 15), TiKVKey(""))); + auto mmp = kvs.getRegionsByRangeOverlap( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 15), TiKVKey(""))); ASSERT_EQ(mmp.size(), 2); kvs.handleDestroy(3, ctx.getTMTContext()); kvs.handleDestroy(3, ctx.getTMTContext()); - RegionMap mmp2 = kvs.getRegionsByRangeOverlap(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 15), TiKVKey(""))); + RegionMap mmp2 = kvs.getRegionsByRangeOverlap( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 15), TiKVKey(""))); ASSERT_EQ(mmp2.size(), 1); ASSERT_EQ(mmp2.at(2)->id(), 2); } @@ -673,13 +727,18 @@ TEST_F(RegionKVStoreTest, Writes) } try { - ASSERT_EQ(RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 6, 6, ctx.getTMTContext()), - EngineStoreApplyRes::None); + ASSERT_EQ( + RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 6, 6, ctx.getTMTContext()), + EngineStoreApplyRes::None); ASSERT_TRUE(false); } catch (Exception & e) { - ASSERT_EQ(e.message(), "Raw TiDB PK: 800000000000091D, Prewrite ts: 2333 can not found in default cf for key: 7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE, region_id: 1, applied_index: 5: (applied_term: 5)"); + ASSERT_EQ( + e.message(), + "Raw TiDB PK: 800000000000091D, Prewrite ts: 2333 can not found in default cf for key: " + "7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE, region_id: 1, " + "applied_index: 5: (applied_term: 5)"); ASSERT_EQ(kvs.getRegion(1)->dataInfo(), "[write 1 lock 1 ]"); kvs.getRegion(1)->tryCompactionFilter(1000); } @@ -695,13 +754,17 @@ TEST_F(RegionKVStoreTest, Writes) auto key = RecordKVFormat::genKey(1, 2333, 1); RegionBench::setupPutRequest(request.add_requests(), ColumnFamilyName::Default, key, "v1"); } - ASSERT_EQ(RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 6, 6, ctx.getTMTContext()), - EngineStoreApplyRes::None); + ASSERT_EQ( + RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 6, 6, ctx.getTMTContext()), + EngineStoreApplyRes::None); ASSERT_TRUE(false); } catch (Exception & e) { - ASSERT_EQ(e.message(), "Found existing key in hex: 7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE"); + ASSERT_EQ( + e.message(), + "Found existing key in hex: " + "7480000000000000FF015F728000000000FF00091D0000000000FAFFFFFFFFFFFFFFFE"); } ASSERT_EQ(kvs.getRegion(1)->dataInfo(), "[lock 1 default 1 ]"); kvs.getRegion(1)->remove("default", RecordKVFormat::genKey(1, 2333, 1)); @@ -709,10 +772,15 @@ TEST_F(RegionKVStoreTest, Writes) { raft_cmdpb::RaftCmdRequest request; { - RegionBench::setupPutRequest(request.add_requests(), ColumnFamilyName::Default, std::string("k1"), "v1"); + RegionBench::setupPutRequest( + request.add_requests(), + ColumnFamilyName::Default, + std::string("k1"), + "v1"); } - ASSERT_EQ(RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 6, 6, ctx.getTMTContext()), - EngineStoreApplyRes::None); + ASSERT_EQ( + RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 6, 6, ctx.getTMTContext()), + EngineStoreApplyRes::None); ASSERT_TRUE(false); } catch (Exception & e) @@ -723,8 +791,9 @@ TEST_F(RegionKVStoreTest, Writes) { raft_cmdpb::RaftCmdRequest request; request.add_requests()->set_cmd_type(::raft_cmdpb::CmdType::Invalid); - ASSERT_EQ(RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 10, 6, ctx.getTMTContext()), - EngineStoreApplyRes::None); + ASSERT_EQ( + RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 10, 6, ctx.getTMTContext()), + EngineStoreApplyRes::None); ASSERT_TRUE(false); } catch (Exception & e) @@ -741,21 +810,24 @@ TEST_F(RegionKVStoreTest, Writes) RegionBench::setupDelRequest(request.add_requests(), ColumnFamilyName::Lock, lock_key); } raft_cmdpb::RaftCmdRequest first_request = request; - ASSERT_EQ(RegionBench::applyWriteRaftCmd(kvs, std::move(first_request), 1, 7, 6, ctx.getTMTContext()), - EngineStoreApplyRes::None); + ASSERT_EQ( + RegionBench::applyWriteRaftCmd(kvs, std::move(first_request), 1, 7, 6, ctx.getTMTContext()), + EngineStoreApplyRes::None); RegionBench::setupDelRequest(request.add_requests(), ColumnFamilyName::Write, TiKVKey("illegal key")); // index <= appliedIndex(), ignore raft_cmdpb::RaftCmdRequest second_request; - ASSERT_EQ(RegionBench::applyWriteRaftCmd(kvs, std::move(second_request), 1, 7, 6, ctx.getTMTContext()), - EngineStoreApplyRes::None); + ASSERT_EQ( + RegionBench::applyWriteRaftCmd(kvs, std::move(second_request), 1, 7, 6, ctx.getTMTContext()), + EngineStoreApplyRes::None); try { // request.clear_requests(); RegionBench::setupDelRequest(request.add_requests(), ColumnFamilyName::Write, TiKVKey("illegal key")); - ASSERT_EQ(RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 9, 6, ctx.getTMTContext()), - EngineStoreApplyRes::None); + ASSERT_EQ( + RegionBench::applyWriteRaftCmd(kvs, std::move(request), 1, 9, 6, ctx.getTMTContext()), + EngineStoreApplyRes::None); ASSERT_TRUE(false); } catch (Exception & e) @@ -783,10 +855,22 @@ TEST_F(RegionKVStoreTest, AdminSplit) createDefaultRegions(); auto ctx = TiFlashTestEnv::getGlobalContext(); KVStore & kvs = getKVS(); - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {1}, {{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {1}, + {{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}}); { testRaftSplit(kvs, ctx.getTMTContext(), proxy_instance); - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{}, raft_cmdpb::AdminResponse{}, 8192, 5, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound); + ASSERT_EQ( + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest{}, + raft_cmdpb::AdminResponse{}, + 8192, + 5, + 6, + ctx.getTMTContext()), + EngineStoreApplyRes::NotFound); } } @@ -795,7 +879,12 @@ TEST_F(RegionKVStoreTest, AdminMerge) createDefaultRegions(); auto ctx = TiFlashTestEnv::getGlobalContext(); KVStore & kvs = getKVS(); - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {1, 7}, {{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5)}, {RecordKVFormat::genKey(1, 5), RecordKVFormat::genKey(1, 10)}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {1, 7}, + {{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5)}, + {RecordKVFormat::genKey(1, 5), RecordKVFormat::genKey(1, 10)}}); { testRaftMergeRollback(kvs, ctx.getTMTContext()); @@ -810,7 +899,11 @@ TEST_F(RegionKVStoreTest, AdminChangePeer) auto ctx = TiFlashTestEnv::getGlobalContext(); auto & kvs = getKVS(); { - proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::make_optional(std::make_pair(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 100)))); + proxy_instance->bootstrapWithRegion( + kvs, + ctx.getTMTContext(), + region_id, + std::make_optional(std::make_pair(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 100)))); } { auto meta = kvs.getRegion(region_id)->cloneMetaRegion(); @@ -833,19 +926,13 @@ class ApplySnapshotTest , public testing::WithParamInterface { public: - ApplySnapshotTest() - { - ingest_using_split = GetParam(); - } + ApplySnapshotTest() { ingest_using_split = GetParam(); } protected: bool ingest_using_split{}; }; -INSTANTIATE_TEST_CASE_P( - ByIngestUsingSplit, - ApplySnapshotTest, - testing::Bool()); +INSTANTIATE_TEST_CASE_P(ByIngestUsingSplit, ApplySnapshotTest, testing::Bool()); TEST_P(ApplySnapshotTest, WithNewRegionRange) try @@ -877,14 +964,15 @@ try auto columns = DMTestEnv::getDefaultTableColumns(); auto table_info = DMTestEnv::getMinimalTableInfo(table_id); auto astptr = DMTestEnv::getPrimaryKeyExpr("test_table"); - storage = StorageDeltaMerge::create("TiFlash", - "default" /* db_name */, - "test_table" /* table_name */, - table_info, - ColumnsDescription{columns}, - astptr, - 0, - ctx); + storage = StorageDeltaMerge::create( + "TiFlash", + "default" /* db_name */, + "test_table" /* table_name */, + table_info, + ColumnsDescription{columns}, + astptr, + 0, + ctx); storage->startup(); } SCOPE_EXIT({ @@ -893,7 +981,11 @@ try }); // Initially region_19 range is [0, 10000) { - auto region = makeRegion(region_id, RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, 10000), kvs.getProxyHelper()); + auto region = makeRegion( + region_id, + RecordKVFormat::genKey(table_id, 0), + RecordKVFormat::genKey(table_id, 10000), + kvs.getProxyHelper()); GenMockSSTData(DMTestEnv::getMinimalTableInfo(table_id), table_id, region_id_str, 20, 100, 0); std::vector sst_views{ SSTView{ @@ -928,7 +1020,11 @@ try } // Later, its range is changed to [20000, 50000) { - auto region = makeRegion(region_id, RecordKVFormat::genKey(table_id, 20000), RecordKVFormat::genKey(table_id, 50000), kvs.getProxyHelper()); + auto region = makeRegion( + region_id, + RecordKVFormat::genKey(table_id, 20000), + RecordKVFormat::genKey(table_id, 50000), + kvs.getProxyHelper()); GenMockSSTData(DMTestEnv::getMinimalTableInfo(table_id), table_id, region_id_str, 20100, 20200, 0); std::vector sst_views{ SSTView{ @@ -994,7 +1090,8 @@ try ASSERT_NE(proxy_helper->sst_reader_interfaces.fn_key, nullptr); { auto region_id = 19; - auto region = makeRegion(region_id, RecordKVFormat::genKey(1, 50), RecordKVFormat::genKey(1, 60), kvs.getProxyHelper()); + auto region + = makeRegion(region_id, RecordKVFormat::genKey(1, 50), RecordKVFormat::genKey(1, 60), kvs.getProxyHelper()); auto region_id_str = std::to_string(region_id); auto & mmp = MockSSTReader::getMockSSTData(); MockSSTReader::getMockSSTData().clear(); @@ -1035,62 +1132,69 @@ try } catch (Exception & e) { - ASSERT_EQ(e.message(), fmt::format("try to apply with older index, region_id={} applied_index={} new_index={}", region_id, 8, 6)); + ASSERT_EQ( + e.message(), + fmt::format( + "try to apply with older index, region_id={} applied_index={} new_index={}", + region_id, + 8, + 6)); } } { // Snapshot will be rejected if region overlaps. { - auto region = makeRegion(22, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100), kvs.getProxyHelper()); - auto ingest_ids = kvs.preHandleSnapshotToFiles( - region, - {}, - 9, - 5, - std::nullopt, + auto region = makeRegion( + 22, + RecordKVFormat::genKey(55, 50), + RecordKVFormat::genKey(55, 100), + kvs.getProxyHelper()); + auto ingest_ids = kvs.preHandleSnapshotToFiles(region, {}, 9, 5, std::nullopt, ctx.getTMTContext()); + kvs.checkAndApplyPreHandledSnapshot( + RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); - kvs.checkAndApplyPreHandledSnapshot(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); } try { - auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100), kvs.getProxyHelper()); - auto ingest_ids = kvs.preHandleSnapshotToFiles( - region, - {}, - 9, - 5, - std::nullopt, - ctx.getTMTContext()); - kvs.checkAndApplyPreHandledSnapshot(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); // overlap, but not tombstone + auto region = makeRegion( + 20, + RecordKVFormat::genKey(55, 50), + RecordKVFormat::genKey(55, 100), + kvs.getProxyHelper()); + auto ingest_ids = kvs.preHandleSnapshotToFiles(region, {}, 9, 5, std::nullopt, ctx.getTMTContext()); + kvs.checkAndApplyPreHandledSnapshot( + RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, + ctx.getTMTContext()); // overlap, but not tombstone ASSERT_TRUE(false); } catch (Exception & e) { - ASSERT_EQ(e.message(), "range of region_id=20 is overlapped with region_id=22, state: region { id: 22 }"); + ASSERT_EQ( + e.message(), + "range of region_id=20 is overlapped with region_id=22, state: region { id: 22 }"); } } { { // Applying snapshot will throw if proxy is not inited. const auto * ori_ptr = proxy_helper->proxy_ptr.inner; - SCOPE_EXIT({ - proxy_helper->proxy_ptr.inner = ori_ptr; - }); + SCOPE_EXIT({ proxy_helper->proxy_ptr.inner = ori_ptr; }); try { - auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100), kvs.getProxyHelper()); + auto region = makeRegion( + 20, + RecordKVFormat::genKey(55, 50), + RecordKVFormat::genKey(55, 100), + kvs.getProxyHelper()); // preHandleSnapshotToFiles will assert proxy_ptr is not null. - auto ingest_ids = kvs.preHandleSnapshotToFiles( - region, - {}, - 10, - 5, - std::nullopt, - ctx.getTMTContext()); + auto ingest_ids + = kvs.preHandleSnapshotToFiles(region, {}, 10, 5, std::nullopt, ctx.getTMTContext()); proxy_helper->proxy_ptr.inner = nullptr; - kvs.checkAndApplyPreHandledSnapshot(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); + kvs.checkAndApplyPreHandledSnapshot( + RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, + ctx.getTMTContext()); ASSERT_TRUE(false); } catch (Exception & e) @@ -1106,15 +1210,15 @@ try s.set_state(::raft_serverpb::PeerState::Tombstone); s; })); - auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100), kvs.getProxyHelper()); - auto ingest_ids = kvs.preHandleSnapshotToFiles( - region, - {}, - 10, - 5, - std::nullopt, - ctx.getTMTContext()); - kvs.checkAndApplyPreHandledSnapshot(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); // overlap, tombstone, remove previous one + auto region = makeRegion( + 20, + RecordKVFormat::genKey(55, 50), + RecordKVFormat::genKey(55, 100), + kvs.getProxyHelper()); + auto ingest_ids = kvs.preHandleSnapshotToFiles(region, {}, 10, 5, std::nullopt, ctx.getTMTContext()); + kvs.checkAndApplyPreHandledSnapshot( + RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, + ctx.getTMTContext()); // overlap, tombstone, remove previous one auto state = proxy_helper->getRegionLocalState(8192); ASSERT_EQ(state.state(), raft_serverpb::PeerState::Tombstone); @@ -1146,12 +1250,7 @@ try ColumnFamilyType::Default, BaseBuffView{region_id_str.data(), region_id_str.length()}, }); - kvs.handleIngestSST( - region_id, - SSTViewVec{sst_views.data(), sst_views.size()}, - 100, - 1, - ctx.getTMTContext()); + kvs.handleIngestSST(region_id, SSTViewVec{sst_views.data(), sst_views.size()}, 100, 1, ctx.getTMTContext()); ASSERT_EQ(kvs.getRegion(region_id)->checkIndex(100), true); } } @@ -1165,7 +1264,13 @@ TEST_F(RegionKVStoreTest, Restore) KVStore & kvs = getKVS(); { ASSERT_EQ(kvs.getRegion(0), nullptr); - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {1, 2, 3}, {{{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}, {RecordKVFormat::genKey(1, 10), RecordKVFormat::genKey(1, 20)}, {RecordKVFormat::genKey(1, 30), RecordKVFormat::genKey(1, 40)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {1, 2, 3}, + {{{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}, + {RecordKVFormat::genKey(1, 10), RecordKVFormat::genKey(1, 20)}, + {RecordKVFormat::genKey(1, 30), RecordKVFormat::genKey(1, 40)}}}); } tryPersistRegion(kvs, 1); tryPersistRegion(kvs, 2); @@ -1217,7 +1322,8 @@ TEST_F(RegionKVStoreTest, RegionRange) res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); ASSERT_EQ(res.size(), 4); - res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 1), TiKVKey(""))); + res = region_index.findByRangeOverlap( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 1), TiKVKey(""))); ASSERT_EQ(res.size(), 3); ASSERT_TRUE(res.find(1) != res.end()); ASSERT_TRUE(res.find(2) != res.end()); @@ -1230,19 +1336,27 @@ TEST_F(RegionKVStoreTest, RegionRange) ASSERT_TRUE(res.find(2) != res.end()); ASSERT_TRUE(res.find(4) != res.end()); - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 1), RecordKVFormat::genKey(1, 4)), 4); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 1), RecordKVFormat::genKey(1, 4)), + 4); res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); ASSERT_EQ(res.size(), 3); - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 1)), 3); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 1)), + 3); res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); ASSERT_EQ(res.size(), 2); - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 3)), 2); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 3)), + 2); res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); ASSERT_EQ(res.size(), 1); - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)), 1); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)), + 1); res = region_index.findByRangeOverlap(RegionRangeKeys::makeComparableKeys(TiKVKey(""), TiKVKey(""))); ASSERT_TRUE(res.empty()); @@ -1261,41 +1375,56 @@ TEST_F(RegionKVStoreTest, RegionRange) catch (Exception & e) { const auto & res = e.message(); - ASSERT_EQ(res, "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): not found region_id=1"); + ASSERT_EQ( + res, + "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): not found region_id=1"); } region_index.add(makeRegion(2, RecordKVFormat::genKey(1, 3), RecordKVFormat::genKey(1, 5))); try { - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 4), RecordKVFormat::genKey(1, 5)), 2); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 4), RecordKVFormat::genKey(1, 5)), + 2); assert(false); } catch (Exception & e) { const auto & res = e.message(); - ASSERT_EQ(res, "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): not found start key"); + ASSERT_EQ( + res, + "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): not found start key"); } try { - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 3), RecordKVFormat::genKey(1, 4)), 2); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 3), RecordKVFormat::genKey(1, 4)), + 2); assert(false); } catch (Exception & e) { const auto & res = e.message(); - ASSERT_EQ(res, "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): not found end key"); + ASSERT_EQ( + res, + "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): not found end key"); } try { - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 3), RecordKVFormat::genKey(1, 3)), 2); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 3), RecordKVFormat::genKey(1, 3)), + 2); assert(false); } catch (Exception & e) { const auto & res = e.message(); - ASSERT_EQ(res, "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): range of region_id=2 is empty"); + ASSERT_EQ( + res, + "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): range of region_id=2 is " + "empty"); } try @@ -1306,7 +1435,9 @@ TEST_F(RegionKVStoreTest, RegionRange) catch (Exception & e) { const auto & res = e.message(); - ASSERT_EQ(res, "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): not found region_id=2"); + ASSERT_EQ( + res, + "void DB::RegionsRangeIndex::remove(const DB::RegionRange &, DB::RegionID): not found region_id=2"); } region_index.clear(); @@ -1331,23 +1462,49 @@ TEST_F(RegionKVStoreTest, RegionRange) region_index.add(makeRegion(3, RecordKVFormat::genKey(1, 2), RecordKVFormat::genKey(1, 3))); ASSERT_EQ(root_map.size(), 6); - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 2), RecordKVFormat::genKey(1, 3)), 3); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 2), RecordKVFormat::genKey(1, 3)), + 3); ASSERT_EQ(root_map.size(), 5); - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 1)), 1); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 1)), + 1); ASSERT_EQ(root_map.size(), 4); - region_index.remove(RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 1), RecordKVFormat::genKey(1, 2)), 2); + region_index.remove( + RegionRangeKeys::makeComparableKeys(RecordKVFormat::genKey(1, 1), RecordKVFormat::genKey(1, 2)), + 2); ASSERT_EQ(root_map.size(), 2); } // Test region range with merge. { { // Compute `source_at_left` by region range. - ASSERT_EQ(MetaRaftCommandDelegate::computeRegionMergeResult(createRegionInfo(1, "x", ""), createRegionInfo(1000, "", "x")).source_at_left, false); - ASSERT_EQ(MetaRaftCommandDelegate::computeRegionMergeResult(createRegionInfo(1, "", "x"), createRegionInfo(1000, "x", "")).source_at_left, true); - ASSERT_EQ(MetaRaftCommandDelegate::computeRegionMergeResult(createRegionInfo(1, "x", "y"), createRegionInfo(1000, "y", "z")).source_at_left, true); - ASSERT_EQ(MetaRaftCommandDelegate::computeRegionMergeResult(createRegionInfo(1, "y", "z"), createRegionInfo(1000, "x", "y")).source_at_left, false); + ASSERT_EQ( + MetaRaftCommandDelegate::computeRegionMergeResult( + createRegionInfo(1, "x", ""), + createRegionInfo(1000, "", "x")) + .source_at_left, + false); + ASSERT_EQ( + MetaRaftCommandDelegate::computeRegionMergeResult( + createRegionInfo(1, "", "x"), + createRegionInfo(1000, "x", "")) + .source_at_left, + true); + ASSERT_EQ( + MetaRaftCommandDelegate::computeRegionMergeResult( + createRegionInfo(1, "x", "y"), + createRegionInfo(1000, "y", "z")) + .source_at_left, + true); + ASSERT_EQ( + MetaRaftCommandDelegate::computeRegionMergeResult( + createRegionInfo(1, "y", "z"), + createRegionInfo(1000, "x", "y")) + .source_at_left, + false); } { RegionState region_state; diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp index c8112e880be..b06fb7c3fd8 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore_fast_add_peer.cpp @@ -50,7 +50,9 @@ class RegionKVStoreTestFAP : public RegionKVStoreTest if (global_context.getSharedContextDisagg()->remote_data_store == nullptr) { already_initialize_data_store = false; - global_context.getSharedContextDisagg()->initRemoteDataStore(global_context.getFileProvider(), /*s3_enabled*/ true); + global_context.getSharedContextDisagg()->initRemoteDataStore( + global_context.getFileProvider(), + /*s3_enabled*/ true); ASSERT_TRUE(global_context.getSharedContextDisagg()->remote_data_store != nullptr); } else @@ -128,9 +130,7 @@ try std::vector f(total, false); while (true) { - auto count = std::accumulate(f.begin(), f.end(), 0, [&](int a, bool b) -> int { - return a + int(b); - }); + auto count = std::accumulate(f.begin(), f.end(), 0, [&](int a, bool b) -> int { return a + int(b); }); if (count >= total) { break; @@ -168,7 +168,13 @@ try } CATCH -void persistAfterWrite(Context & ctx, KVStore & kvs, std::unique_ptr & proxy_instance, UniversalPageStoragePtr page_storage, uint64_t region_id, uint64_t index) +void persistAfterWrite( + Context & ctx, + KVStore & kvs, + std::unique_ptr & proxy_instance, + UniversalPageStoragePtr page_storage, + uint64_t region_id, + uint64_t index) { MockRaftStoreProxy::FailCond cond; proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); @@ -195,7 +201,8 @@ try region->addPeer(store_id, peer_id, metapb::PeerRole::Learner); // Write some data, and persist meta. - auto [index, term] = proxy_instance->normalWrite(region_id, {34}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance->normalWrite(region_id, {34}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); kvs.setRegionCompactLogConfig(0, 0, 0, 0); persistAfterWrite(global_context, kvs, proxy_instance, page_storage, region_id, index); @@ -228,7 +235,8 @@ try } { - auto [data_seq, checkpoint_data_holder] = fap_context->getNewerCheckpointData(global_context, store_id, upload_sequence); + auto [data_seq, checkpoint_data_holder] + = fap_context->getNewerCheckpointData(global_context, store_id, upload_sequence); ASSERT_EQ(data_seq, upload_sequence); ASSERT_TRUE(checkpoint_data_holder == nullptr); } diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index 500d2b64814..a1ce880f3b7 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -31,14 +31,20 @@ try { MockRaftStoreProxy::FailCond cond; - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {1}, {{{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {1}, + {{{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}}}); auto kvr1 = kvs.getRegion(region_id); auto r1 = proxy_instance->getRegion(region_id); ASSERT_NE(r1, nullptr); ASSERT_NE(kvr1, nullptr); applied_index = r1->getLatestAppliedIndex(); ASSERT_EQ(r1->getLatestAppliedIndex(), kvr1->appliedIndex()); - auto [index, term] = proxy_instance->normalWrite(region_id, {33}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {33}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 1); ASSERT_EQ(kvr1->appliedIndex(), applied_index + 1); @@ -58,7 +64,11 @@ try auto applied_index = 0; auto region_id = 2; { - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {2}, {{{RecordKVFormat::genKey(1, 10), RecordKVFormat::genKey(1, 20)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {2}, + {{{RecordKVFormat::genKey(1, 10), RecordKVFormat::genKey(1, 20)}}}); MockRaftStoreProxy::FailCond cond; cond.type = MockRaftStoreProxy::FailCond::Type::BEFORE_KVSTORE_WRITE; @@ -66,7 +76,9 @@ try auto r1 = proxy_instance->getRegion(region_id); applied_index = r1->getLatestAppliedIndex(); ASSERT_EQ(r1->getLatestAppliedIndex(), kvr1->appliedIndex()); - auto [index, term] = proxy_instance->normalWrite(region_id, {34}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {34}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); // KVStore failed before write and advance. proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index); @@ -87,7 +99,8 @@ try ASSERT_EQ(kvr1->appliedIndex(), applied_index + 1); auto && [req, res] = MockRaftStoreProxy::composeCompactLog(r1, kvr1->appliedIndex()); - auto [indexc, termc] = proxy_instance->adminCommand(region_id, std::move(req), std::move(res), std::nullopt); + auto [indexc, termc] + = proxy_instance->adminCommand(region_id, std::move(req), std::move(res), std::nullopt); // Reject compact log. kvs.setRegionCompactLogConfig(10000000, 10000000, 10000000, 10000000); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, indexc); @@ -99,7 +112,11 @@ try auto applied_index = 0; auto region_id = 3; { - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {3}, {{{RecordKVFormat::genKey(1, 30), RecordKVFormat::genKey(1, 40)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {3}, + {{{RecordKVFormat::genKey(1, 30), RecordKVFormat::genKey(1, 40)}}}); MockRaftStoreProxy::FailCond cond; cond.type = MockRaftStoreProxy::FailCond::Type::BEFORE_KVSTORE_ADVANCE; @@ -107,7 +124,9 @@ try auto r1 = proxy_instance->getRegion(region_id); applied_index = r1->getLatestAppliedIndex(); ASSERT_EQ(r1->getLatestAppliedIndex(), kvr1->appliedIndex()); - auto [index, term] = proxy_instance->normalWrite(region_id, {34}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {34}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); // KVStore failed before advance. proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index); @@ -123,7 +142,9 @@ try ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index); ASSERT_EQ(kvr1->appliedIndex(), applied_index); ASSERT_EQ(kvr1->appliedIndex(), r1->getLatestCommitIndex() - 1); - EXPECT_THROW(proxy_instance->replay(kvs, ctx.getTMTContext(), region_id, r1->getLatestCommitIndex()), Exception); + EXPECT_THROW( + proxy_instance->replay(kvs, ctx.getTMTContext(), region_id, r1->getLatestCommitIndex()), + Exception); } } @@ -131,7 +152,11 @@ try auto applied_index = 0; auto region_id = 4; { - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {4}, {{{RecordKVFormat::genKey(1, 50), RecordKVFormat::genKey(1, 60)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {4}, + {{{RecordKVFormat::genKey(1, 50), RecordKVFormat::genKey(1, 60)}}}); MockRaftStoreProxy::FailCond cond; cond.type = MockRaftStoreProxy::FailCond::Type::BEFORE_PROXY_ADVANCE; @@ -140,7 +165,9 @@ try applied_index = r1->getLatestAppliedIndex(); ASSERT_EQ(r1->getLatestAppliedIndex(), kvr1->appliedIndex()); LOG_INFO(Logger::get(), "applied_index {}", applied_index); - auto [index, term] = proxy_instance->normalWrite(region_id, {35}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {35}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); // KVStore succeed. Proxy failed before advance. proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index); @@ -158,7 +185,9 @@ try // Proxy shall replay from handle 35. proxy_instance->replay(kvs, ctx.getTMTContext(), region_id, r1->getLatestCommitIndex()); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 1); - auto [index, term] = proxy_instance->normalWrite(region_id, {36}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {36}, {"v2"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 2); } @@ -193,7 +222,9 @@ try auto str_key = Redact::hexStringToKey(k.data(), k.size()); auto str_val = Redact::hexStringToKey(v.data(), v.size()); - auto [index, term] = proxy_instance->rawWrite(region_id, {str_key}, {str_val}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + auto [index, term] + = proxy_instance + ->rawWrite(region_id, {str_key}, {str_val}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); EXPECT_THROW(proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index), Exception); UNUSED(term); EXPECT_THROW(ReadRegionCommitCache(kvr1, true), Exception); @@ -210,7 +241,11 @@ try { KVStore & kvs = getKVS(); UInt64 region_id = 2; - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {region_id}, {{{RecordKVFormat::genKey(1, 10), RecordKVFormat::genKey(1, 20)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {region_id}, + {{{RecordKVFormat::genKey(1, 10), RecordKVFormat::genKey(1, 20)}}}); // InvalidAdmin raft_cmdpb::AdminRequest request; @@ -231,7 +266,11 @@ try // All "useless" commands. KVStore & kvs = getKVS(); UInt64 region_id = 3; - proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {region_id}, {{{RecordKVFormat::genKey(1, 20), RecordKVFormat::genKey(1, 30)}}}); + proxy_instance->debugAddRegions( + kvs, + ctx.getTMTContext(), + {region_id}, + {{{RecordKVFormat::genKey(1, 20), RecordKVFormat::genKey(1, 30)}}}); raft_cmdpb::AdminRequest request; raft_cmdpb::AdminResponse response2; raft_cmdpb::AdminResponse response; @@ -239,45 +278,97 @@ try request.mutable_compact_log(); request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); response = response2; - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), region_id, 22, 6, ctx.getTMTContext()), EngineStoreApplyRes::Persist); + ASSERT_EQ( + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest{request}, + std::move(response), + region_id, + 22, + 6, + ctx.getTMTContext()), + EngineStoreApplyRes::Persist); response = response2; - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), region_id, 23, 6, ctx.getTMTContext()), EngineStoreApplyRes::Persist); + ASSERT_EQ( + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest{request}, + std::move(response), + region_id, + 23, + 6, + ctx.getTMTContext()), + EngineStoreApplyRes::Persist); response = response2; - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 8192, 5, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound); + ASSERT_EQ( + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest{request}, + std::move(response), + 8192, + 5, + 6, + ctx.getTMTContext()), + EngineStoreApplyRes::NotFound); request.set_cmd_type(::raft_cmdpb::AdminCmdType::ComputeHash); response = response2; - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), region_id, 24, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); + ASSERT_EQ( + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest{request}, + std::move(response), + region_id, + 24, + 6, + ctx.getTMTContext()), + EngineStoreApplyRes::None); request.set_cmd_type(::raft_cmdpb::AdminCmdType::VerifyHash); response = response2; - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), region_id, 25, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); + ASSERT_EQ( + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest{request}, + std::move(response), + region_id, + 25, + 6, + ctx.getTMTContext()), + EngineStoreApplyRes::None); { kvs.setRegionCompactLogConfig(0, 0, 0, 0); request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); - ASSERT_EQ(kvs.handleAdminRaftCmd(std::move(request), std::move(response2), region_id, 26, 6, ctx.getTMTContext()), EngineStoreApplyRes::Persist); + ASSERT_EQ( + kvs.handleAdminRaftCmd(std::move(request), std::move(response2), region_id, 26, 6, ctx.getTMTContext()), + EngineStoreApplyRes::Persist); } } } CATCH -static void validate(KVStore & kvs, std::unique_ptr & proxy_instance, UInt64 region_id, MockRaftStoreProxy::Cf & cf_data, ColumnFamilyType cf, int sst_size, int key_count) +static void validate( + KVStore & kvs, + std::unique_ptr & proxy_instance, + UInt64 region_id, + MockRaftStoreProxy::Cf & cf_data, + ColumnFamilyType cf, + int sst_size, + int key_count) { auto kvr1 = kvs.getRegion(region_id); auto r1 = proxy_instance->getRegion(region_id); auto proxy_helper = proxy_instance->generateProxyHelper(); auto ssts = cf_data.ssts(); ASSERT_EQ(ssts.size(), sst_size); - auto make_inner_func = [](const TiFlashRaftProxyHelper * proxy_helper, SSTView snap, SSTReader::RegionRangeFilter range) -> std::unique_ptr { + auto make_inner_func = [](const TiFlashRaftProxyHelper * proxy_helper, + SSTView snap, + SSTReader::RegionRangeFilter range) -> std::unique_ptr { auto parsed_kind = MockRaftStoreProxy::parseSSTViewKind(buffToStrView(snap.path)); auto reader = std::make_unique(proxy_helper, snap, range); assert(reader->sst_format_kind() == parsed_kind); return reader; }; - MultiSSTReader reader{proxy_helper.get(), cf, make_inner_func, ssts, Logger::get(), kvr1->getRange()}; + MultiSSTReader + reader{proxy_helper.get(), cf, make_inner_func, ssts, Logger::get(), kvr1->getRange()}; size_t counter = 0; while (reader.remained()) @@ -369,12 +460,16 @@ try MockRaftStoreProxy::FailCond cond; { - auto [index, term] = proxy_instance->normalWrite(region_id, {9}, {"v9"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {9}, {"v9"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); } { // Test if write succeed. - auto [index, term] = proxy_instance->normalWrite(region_id, {1}, {"fv1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {1}, {"fv1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); EXPECT_THROW(proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index), Exception); } } @@ -392,12 +487,16 @@ try MockRaftStoreProxy::FailCond cond; { - auto [index, term] = proxy_instance->normalWrite(region_id, {19}, {"v19"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {19}, {"v19"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); } { // Test if write succeed. - auto [index, term] = proxy_instance->normalWrite(region_id, {10}, {"v10"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {10}, {"v10"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); EXPECT_THROW(proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index), Exception); } } @@ -416,12 +515,15 @@ try write_cf.freeze(); kvs.mutProxyHelperUnsafe()->sst_reader_interfaces = make_mock_sst_reader_interface(); - proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); + proxy_instance + ->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); MockRaftStoreProxy::FailCond cond; { // Test if write succeed. - auto [index, term] = proxy_instance->normalWrite(region_id, {20}, {"v20"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {20}, {"v20"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); // Found existing key ... EXPECT_THROW(proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index), Exception); } @@ -442,7 +544,8 @@ try kvs.mutProxyHelperUnsafe()->sst_reader_interfaces = make_mock_sst_reader_interface(); // Shall not panic. - proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); + proxy_instance + ->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); } { // Test of ingesting duplicated key with different values. @@ -460,7 +563,10 @@ try kvs.mutProxyHelperUnsafe()->sst_reader_interfaces = make_mock_sst_reader_interface(); // Found existing key ... - EXPECT_THROW(proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt), Exception); + EXPECT_THROW( + proxy_instance + ->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt), + Exception); } { // Test of cancel prehandled. @@ -477,7 +583,15 @@ try write_cf.freeze(); kvs.mutProxyHelperUnsafe()->sst_reader_interfaces = make_mock_sst_reader_interface(); - auto r = proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt, /*cancel_after_prehandle=*/true); + auto r = proxy_instance->snapshot( + kvs, + ctx.getTMTContext(), + region_id, + {default_cf, write_cf}, + 0, + 0, + std::nullopt, + /*cancel_after_prehandle=*/true); } } } @@ -567,19 +681,27 @@ try proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); MockRaftStoreProxy::FailCond cond; { - auto [index, term] = proxy_instance->rawWrite(region_id, {klo}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->rawWrite(region_id, {klo}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); } { - auto [index, term] = proxy_instance->rawWrite(region_id, {klo2}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->rawWrite(region_id, {klo2}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); } { - auto [index, term] = proxy_instance->rawWrite(region_id, {kro}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->rawWrite(region_id, {kro}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); } { - auto [index, term] = proxy_instance->rawWrite(region_id, {kro2}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->rawWrite(region_id, {kro2}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); } } @@ -599,35 +721,39 @@ try ctx.getTMTContext().debugSetWaitIndexTimeout(1); startReadIndexUtils(ctx); - SCOPE_EXIT({ - stopReadIndexUtils(); - }); + SCOPE_EXIT({ stopReadIndexUtils(); }); auto table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::nullopt); auto kvr1 = kvs.getRegion(region_id); ctx.getTMTContext().getRegionTable().updateRegion(*kvr1); - std::vector keys{RecordKVFormat::genKey(table_id, 3).toString(), RecordKVFormat::genKey(table_id, 3, 5).toString(), RecordKVFormat::genKey(table_id, 3, 8).toString()}; - std::vector vals({RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20).toString(), TiKVValue("value1").toString(), RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5).toString()}); + std::vector keys{ + RecordKVFormat::genKey(table_id, 3).toString(), + RecordKVFormat::genKey(table_id, 3, 5).toString(), + RecordKVFormat::genKey(table_id, 3, 8).toString()}; + std::vector vals( + {RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20).toString(), + TiKVValue("value1").toString(), + RecordKVFormat::encodeWriteCfValue(RecordKVFormat::CFModifyFlag::PutFlag, 5).toString()}); auto ops = std::vector{ ColumnFamilyType::Lock, ColumnFamilyType::Default, ColumnFamilyType::Write, }; - auto [index, term] = proxy_instance->rawWrite(region_id, std::move(keys), std::move(vals), {WriteCmdType::Put, WriteCmdType::Put, WriteCmdType::Put}, std::move(ops)); + auto [index, term] = proxy_instance->rawWrite( + region_id, + std::move(keys), + std::move(vals), + {WriteCmdType::Put, WriteCmdType::Put, WriteCmdType::Put}, + std::move(ops)); ASSERT_EQ(index, 6); ASSERT_EQ(kvr1->appliedIndex(), 5); ASSERT_EQ(term, 5); auto mvcc_query_info = MvccQueryInfo(false, 10); auto f = [&] { - auto discard = doLearnerRead( - table_id, - mvcc_query_info, - false, - ctx, - log); + auto discard = doLearnerRead(table_id, mvcc_query_info, false, ctx, log); UNUSED(discard); }; EXPECT_THROW(f(), RegionException); @@ -636,18 +762,14 @@ try auto r1 = proxy_instance->getRegion(region_id); r1->updateAppliedIndex(index); kvr1->setApplied(index, term); - auto regions_snapshot = doLearnerRead( - table_id, - mvcc_query_info, - false, - ctx, - log); + auto regions_snapshot = doLearnerRead(table_id, mvcc_query_info, false, ctx, log); // 0 unavailable regions ASSERT_EQ(regions_snapshot.size(), 1); // No throw auto mvcc_query_info2 = MvccQueryInfo(false, 10); - mvcc_query_info2.regions_query_info.emplace_back(1, kvr1->version(), kvr1->confVer(), table_id, kvr1->getRange()->rawKeys()); + mvcc_query_info2.regions_query_info + .emplace_back(1, kvr1->version(), kvr1->confVer(), table_id, kvr1->getRange()->rawKeys()); validateQueryInfo(mvcc_query_info2, regions_snapshot, ctx.getTMTContext(), log); } CATCH @@ -667,7 +789,11 @@ try table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); auto start = RecordKVFormat::genKey(table_id, 0); auto end = RecordKVFormat::genKey(table_id, 10); - proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::make_pair(start.toString(), end.toString())); + proxy_instance->bootstrapWithRegion( + kvs, + ctx.getTMTContext(), + region_id, + std::make_pair(start.toString(), end.toString())); auto r1 = proxy_instance->getRegion(region_id); // See `decodeWriteCfValue`. @@ -687,7 +813,8 @@ try write_cf.finish_file(SSTFormatKind::KIND_TABLET); write_cf.freeze(); - auto kvr1 = proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); + auto kvr1 = proxy_instance + ->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); ASSERT_EQ(kvr1->orphanKeysInfo().remainedKeyCount(), 1); ASSERT_EQ(kvr1->writeCFCount(), 1); // k2 } @@ -696,8 +823,12 @@ try // Add a new key to trigger another row2col transform. auto kvr1 = kvs.getRegion(region_id); auto k = RecordKVFormat::genKey(table_id, 3, 111); - auto [index, term] = proxy_instance->rawWrite(region_id, {k}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); - auto [index2, term2] = proxy_instance->rawWrite(region_id, {k}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + auto [index, term] + = proxy_instance + ->rawWrite(region_id, {k}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index2, term2] + = proxy_instance + ->rawWrite(region_id, {k}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); ASSERT_EQ(kvr1->orphanKeysInfo().remainedKeyCount(), 1); @@ -709,11 +840,15 @@ try // A normal write to "save" the orphan key. auto k2 = RecordKVFormat::genKey(table_id, 2, 111); auto kvr1 = kvs.getRegion(region_id); - auto [index, term] = proxy_instance->rawWrite(region_id, {k2}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->rawWrite(region_id, {k2}, {value_default}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); // After applied this log, the write record is not orphan any more. ASSERT_EQ(kvr1->writeCFCount(), 0); - auto [index2, term2] = proxy_instance->rawWrite(region_id, {k2}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + auto [index2, term2] + = proxy_instance + ->rawWrite(region_id, {k2}, {value_write}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index2); ASSERT_EQ(kvr1->writeCFCount(), 0); ASSERT_EQ(kvr1->orphanKeysInfo().remainedKeyCount(), 0); @@ -747,7 +882,11 @@ try table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); auto start = RecordKVFormat::genKey(table_id, 0); auto end = RecordKVFormat::genKey(table_id, 10); - proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), region_id, std::make_pair(start.toString(), end.toString())); + proxy_instance->bootstrapWithRegion( + kvs, + ctx.getTMTContext(), + region_id, + std::make_pair(start.toString(), end.toString())); auto r1 = proxy_instance->getRegion(region_id); // See `decodeWriteCfValue`. @@ -764,7 +903,8 @@ try write_cf.finish_file(SSTFormatKind::KIND_TABLET); write_cf.freeze(); - auto kvr1 = proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); + auto kvr1 = proxy_instance + ->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 0, 0, std::nullopt); ASSERT_EQ(kvr1->orphanKeysInfo().remainedKeyCount(), 1); } { @@ -786,11 +926,23 @@ try { auto k3 = RecordKVFormat::genKey(table_id, 3, 111); auto kvr1 = kvs.getRegion(region_id); - proxy_instance->rawWrite(region_id, {k3, k3}, {value_default, value_write}, {WriteCmdType::Put, WriteCmdType::Put}, {ColumnFamilyType::Default, ColumnFamilyType::Write}, 8); + proxy_instance->rawWrite( + region_id, + {k3, k3}, + {value_default, value_write}, + {WriteCmdType::Put, WriteCmdType::Put}, + {ColumnFamilyType::Default, ColumnFamilyType::Write}, + 8); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, 8); auto k4 = RecordKVFormat::genKey(table_id, 4, 111); - proxy_instance->rawWrite(region_id, {k4, k4}, {value_default, value_write}, {WriteCmdType::Put, WriteCmdType::Put}, {ColumnFamilyType::Default, ColumnFamilyType::Write}, 10); + proxy_instance->rawWrite( + region_id, + {k4, k4}, + {value_default, value_write}, + {WriteCmdType::Put, WriteCmdType::Put}, + {ColumnFamilyType::Default, ColumnFamilyType::Write}, + 10); // Remaining orphan keys of k2. EXPECT_THROW(proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, 10), Exception); } @@ -805,7 +957,8 @@ try write_cf.finish_file(SSTFormatKind::KIND_TABLET); write_cf.freeze(); - auto kvr1 = proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 15, 0, 20); + auto kvr1 + = proxy_instance->snapshot(kvs, ctx.getTMTContext(), region_id, {default_cf, write_cf}, 15, 0, 20); ASSERT_EQ(kvr1->orphanKeysInfo().remainedKeyCount(), 1); } { diff --git a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp index 8eb94f27b9a..056c1d82117 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_proactive_flush.cpp @@ -38,7 +38,9 @@ try ASSERT_NE(kvr1, nullptr); applied_index = r1->getLatestAppliedIndex(); ASSERT_EQ(r1->getLatestAppliedIndex(), kvr1->appliedIndex()); - auto [index, term] = proxy_instance->normalWrite(region_id, {33}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); + auto [index, term] + = proxy_instance + ->normalWrite(region_id, {33}, {"v1"}, {WriteCmdType::Put}, {ColumnFamilyType::Default}); proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index); ASSERT_EQ(r1->getLatestAppliedIndex(), applied_index + 1); ASSERT_EQ(kvr1->appliedIndex(), applied_index + 1); @@ -71,7 +73,15 @@ try raft_cmdpb::AdminResponse response; request.mutable_compact_log(); request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 1999, 22, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound); + ASSERT_EQ( + kvs.handleAdminRaftCmd( + raft_cmdpb::AdminRequest{request}, + std::move(response), + 1999, + 22, + 6, + ctx.getTMTContext()), + EngineStoreApplyRes::NotFound); } } } diff --git a/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp b/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp index 2302e1cc484..169c586b025 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp @@ -31,11 +31,9 @@ namespace tests class ReadIndexTest : public ::testing::Test { public: - ReadIndexTest() - = default; + ReadIndexTest() = default; - void SetUp() override - {} + void SetUp() override {} static size_t computeCntUseHistoryTasks(ReadIndexWorkerManager & manager); static void testBasic(); @@ -54,21 +52,15 @@ void ReadIndexTest::testError() proxy_helper = MockRaftStoreProxy::SetRaftStoreProxyFFIHelper(RaftStoreProxyPtr{&proxy_instance}); proxy_instance.init(10); } - auto manager = ReadIndexWorkerManager::newReadIndexWorkerManager( - proxy_helper, - 5, - [&]() { - return std::chrono::milliseconds(10); - }); + auto manager = ReadIndexWorkerManager::newReadIndexWorkerManager(proxy_helper, 5, [&]() { + return std::chrono::milliseconds(10); + }); { std::vector reqs; std::vector resps; std::list futures; { - reqs = { - make_read_index_reqs(2, 10), - make_read_index_reqs(2, 12), - make_read_index_reqs(2, 13)}; + reqs = {make_read_index_reqs(2, 10), make_read_index_reqs(2, 12), make_read_index_reqs(2, 13)}; for (const auto & req : reqs) { auto future = manager->genReadIndexFuture(req); @@ -112,10 +104,7 @@ void ReadIndexTest::testError() } { - reqs = { - make_read_index_reqs(2, 10), - make_read_index_reqs(2, 12), - make_read_index_reqs(2, 13)}; + reqs = {make_read_index_reqs(2, 10), make_read_index_reqs(2, 12), make_read_index_reqs(2, 13)}; for (const auto & req : reqs) { auto future = manager->genReadIndexFuture(req); @@ -279,9 +268,7 @@ void ReadIndexTest::testNormal() auto manager = ReadIndexWorkerManager::newReadIndexWorkerManager( proxy_helper, 5, - [&]() { - return std::chrono::milliseconds(10); - }, + [&]() { return std::chrono::milliseconds(10); }, 3); for (size_t id = 0; id < manager->workers.size(); ++id) @@ -376,9 +363,7 @@ void ReadIndexTest::testNormal() } { // set region id to let mock proxy drop all related tasks. - proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { - proxy.region_id_to_drop.emplace(1); - }); + proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { proxy.region_id_to_drop.emplace(1); }); std::vector reqs; reqs = {make_read_index_reqs(5, 12), make_read_index_reqs(1, 12), make_read_index_reqs(2, 12)}; @@ -392,23 +377,28 @@ void ReadIndexTest::testNormal() ASSERT(!GCMonitor::instance().checkClean()); { // test timeout 0ms - proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { - proxy.region_id_to_drop.emplace(9); - }); + proxy_instance.unsafeInvokeForTest( + [](MockRaftStoreProxy & proxy) { proxy.region_id_to_drop.emplace(9); }); auto resps = manager->batchReadIndex({make_read_index_reqs(9, 12)}, 0); - ASSERT_EQ(resps[0].first.region_error().has_region_not_found(), true); // timeout to region error not found + ASSERT_EQ( + resps[0].first.region_error().has_region_not_found(), + true); // timeout to region error not found } // disable drop region task - proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { - proxy.region_id_to_drop.clear(); - }); - - ReadIndexWorker::setMaxReadIndexTaskTimeout(std::chrono::milliseconds{10}); // set max task timeout in worker - resps = manager->batchReadIndex(reqs, 500); // the old task is still in worker, poll from mock proxy failed, check timeout and set region error `server_is_busy - ASSERT_EQ(resps[1].first.region_error().has_server_is_busy(), true); // meet region error `server_is_busy` for task timeout - - ReadIndexWorker::setMaxReadIndexTaskTimeout(std::chrono::milliseconds{8 * 1000}); // set max task timeout in worker to previous. + proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { proxy.region_id_to_drop.clear(); }); + + ReadIndexWorker::setMaxReadIndexTaskTimeout( + std::chrono::milliseconds{10}); // set max task timeout in worker + resps = manager->batchReadIndex( + reqs, + 500); // the old task is still in worker, poll from mock proxy failed, check timeout and set region error `server_is_busy + ASSERT_EQ( + resps[1].first.region_error().has_server_is_busy(), + true); // meet region error `server_is_busy` for task timeout + + ReadIndexWorker::setMaxReadIndexTaskTimeout( + std::chrono::milliseconds{8 * 1000}); // set max task timeout in worker to previous. resps = manager->batchReadIndex(reqs, 500); ASSERT_EQ(resps[1].first.read_index(), 669); } @@ -424,9 +414,7 @@ void ReadIndexTest::testNormal() } // set region id to let mock proxy drop all related tasks. - proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { - proxy.region_id_to_drop.emplace(1); - }); + proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { proxy.region_id_to_drop.emplace(1); }); resps = proxy_helper.batchReadIndex_v2(reqs, 50); @@ -434,9 +422,7 @@ void ReadIndexTest::testNormal() ASSERT_EQ(resps[1].first.region_error().has_region_not_found(), true); // timeout to region error not found ASSERT_EQ(resps[2].first.read_index(), 669); - proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { - proxy.region_id_to_drop.clear(); - }); + proxy_instance.unsafeInvokeForTest([](MockRaftStoreProxy & proxy) { proxy.region_id_to_drop.clear(); }); } { // test region not exists @@ -461,17 +447,16 @@ void ReadIndexTest::testBatch() proxy_helper = MockRaftStoreProxy::SetRaftStoreProxyFFIHelper(RaftStoreProxyPtr{&proxy_instance}); proxy_instance.init(10); } - auto manager = ReadIndexWorkerManager::newReadIndexWorkerManager( - proxy_helper, - 5, - [&]() { - return std::chrono::milliseconds(10); - }); + auto manager = ReadIndexWorkerManager::newReadIndexWorkerManager(proxy_helper, 5, [&]() { + return std::chrono::milliseconds(10); + }); // DO NOT run manager and mock proxy in other threads. { { // run with empty `waiting_tasks` - manager->getWorkerByRegion(0).data_map.getDataNode(0)->runOneRound(proxy_helper, manager->getWorkerByRegion(0).read_index_notify_ctrl); + manager->getWorkerByRegion(0).data_map.getDataNode(0)->runOneRound( + proxy_helper, + manager->getWorkerByRegion(0).read_index_notify_ctrl); } { // test upsert region @@ -483,16 +468,14 @@ void ReadIndexTest::testBatch() std::vector reqs; std::deque futures; - reqs = { - make_read_index_reqs(0, 1), - make_read_index_reqs(0, 2), - make_read_index_reqs(0, 3), - make_read_index_reqs(1, 2), - make_read_index_reqs(1, 3)}; + reqs + = {make_read_index_reqs(0, 1), + make_read_index_reqs(0, 2), + make_read_index_reqs(0, 3), + make_read_index_reqs(1, 2), + make_read_index_reqs(1, 3)}; // no waiting task - ASSERT_EQ( - manager->getWorkerByRegion(0).data_map.getDataNode(0)->waiting_tasks.size(), - 0); + ASSERT_EQ(manager->getWorkerByRegion(0).data_map.getDataNode(0)->waiting_tasks.size(), 0); // poll failed for (const auto & req : reqs) { @@ -501,17 +484,21 @@ void ReadIndexTest::testBatch() futures.emplace_back(future); } // worker 0 has 3 waiting tasks. - ASSERT_EQ( - manager->getWorkerByRegion(0).data_map.getDataNode(0)->waiting_tasks.size(), - 3); + ASSERT_EQ(manager->getWorkerByRegion(0).data_map.getDataNode(0)->waiting_tasks.size(), 3); // run worker, clean waiting task. manager->runOneRoundAll(); - ASSERT_EQ( - manager->getWorkerByRegion(0).data_map.getDataNode(0)->waiting_tasks.size(), - 0); + ASSERT_EQ(manager->getWorkerByRegion(0).data_map.getDataNode(0)->waiting_tasks.size(), 0); - ASSERT_EQ(1, manager->getWorkerByRegion(0).data_map.getDataNode(0)->running_tasks.size()); // worker 0 has 1 running task. - ASSERT_EQ(1, manager->getWorkerByRegion(1).data_map.getDataNode(1)->running_tasks.size()); // worker 1 has 1 running task. + ASSERT_EQ( + 1, + manager->getWorkerByRegion(0) + .data_map.getDataNode(0) + ->running_tasks.size()); // worker 0 has 1 running task. + ASSERT_EQ( + 1, + manager->getWorkerByRegion(1) + .data_map.getDataNode(1) + ->running_tasks.size()); // worker 1 has 1 running task. { for (auto & r : proxy_instance.regions) { @@ -526,7 +513,8 @@ void ReadIndexTest::testBatch() } { // continuously run same worker, time duration must bigger than setting. - ASSERT_FALSE(manager->getWorkerByRegion(0).lastRunTimeout(std::chrono::milliseconds(500))); // failed to check last run timeout + ASSERT_FALSE(manager->getWorkerByRegion(0).lastRunTimeout( + std::chrono::milliseconds(500))); // failed to check last run timeout } manager->runOneRoundAll(); @@ -554,7 +542,9 @@ void ReadIndexTest::testBatch() manager->runOneRoundAll(); // failed to execute `fn_make_read_index_task`. - ASSERT_EQ(0, manager->getWorkerByRegion(8192).data_map.getDataNode(8192)->running_tasks.size()); // no running task + ASSERT_EQ( + 0, + manager->getWorkerByRegion(8192).data_map.getDataNode(8192)->running_tasks.size()); // no running task auto resp = future->poll(); ASSERT(resp); @@ -571,9 +561,7 @@ void ReadIndexTest::testBatch() std::list futures; std::vector resps; - reqs = { - make_read_index_reqs(0, 5), - make_read_index_reqs(0, 6)}; + reqs = {make_read_index_reqs(0, 5), make_read_index_reqs(0, 6)}; for (const auto & req : reqs) { auto future = manager->genReadIndexFuture(req); @@ -632,7 +620,9 @@ void ReadIndexTest::testBatch() ASSERT_EQ(resp->read_index(), 670); } - ASSERT_EQ(manager->getWorkerByRegion(0).data_map.getDataNode(0)->history_success_tasks->second.read_index(), 670); + ASSERT_EQ( + manager->getWorkerByRegion(0).data_map.getDataNode(0)->history_success_tasks->second.read_index(), + 670); } { MockStressTestCfg::enable = true; diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index 573f666f670..8b9cb63637a 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -42,13 +42,14 @@ namespace DB { -extern void GenMockSSTData(const TiDB::TableInfo & table_info, - TableID table_id, - const String & store_key, - UInt64 start_handle, - UInt64 end_handle, - UInt64 num_fields = 1, - const std::unordered_set & cfs = {ColumnFamilyType::Write, ColumnFamilyType::Default}); +extern void GenMockSSTData( + const TiDB::TableInfo & table_info, + TableID table_id, + const String & store_key, + UInt64 start_handle, + UInt64 end_handle, + UInt64 num_fields = 1, + const std::unordered_set & cfs = {ColumnFamilyType::Write, ColumnFamilyType::Default}); namespace FailPoints { @@ -65,7 +66,10 @@ extern void setupDelRequest(raft_cmdpb::Request *, const std::string &, const Ti } // namespace RegionBench extern void CheckRegionForMergeCmd(const raft_cmdpb::AdminResponse & response, const RegionState & region_state); -extern void ChangeRegionStateRange(RegionState & region_state, bool source_at_left, const RegionState & source_region_state); +extern void ChangeRegionStateRange( + RegionState & region_state, + bool source_at_left, + const RegionState & source_region_state); namespace tests { @@ -73,10 +77,7 @@ namespace tests class RegionKVStoreTest : public ::testing::Test { public: - RegionKVStoreTest() - { - test_path = TiFlashTestEnv::getTemporaryPath("/region_kvs_test"); - } + RegionKVStoreTest() { test_path = TiFlashTestEnv::getTemporaryPath("/region_kvs_test"); } static void SetUpTestCase() {} @@ -99,10 +100,7 @@ class RegionKVStoreTest : public ::testing::Test LOG_INFO(Logger::get("Test"), "Finished setup"); } - void TearDown() override - { - proxy_instance->clear(); - } + void TearDown() override { proxy_instance->clear(); } protected: KVStore & getKVS() { return *kvstore; } @@ -117,10 +115,7 @@ class RegionKVStoreTest : public ::testing::Test kvstore->restore(*path_pool, proxy_helper.get()); return *kvstore; } - void createDefaultRegions() - { - proxy_instance->init(100); - } + void createDefaultRegions() { proxy_instance->init(100); } void initStorages() { bool v = false; @@ -149,15 +144,9 @@ class RegionKVStoreTest : public ::testing::Test over.store(false); ctx.getTMTContext().setStatusRunning(); // Start mock proxy in other thread - proxy_runner.reset(new std::thread([&]() { - proxy_instance->testRunNormal(over); - })); + proxy_runner.reset(new std::thread([&]() { proxy_instance->testRunNormal(over); })); ASSERT_EQ(kvstore->getProxyHelper(), proxy_helper.get()); - kvstore->initReadIndexWorkers( - []() { - return std::chrono::milliseconds(10); - }, - 1); + kvstore->initReadIndexWorkers([]() { return std::chrono::milliseconds(10); }, 1); ASSERT_NE(kvstore->read_index_worker_manager, nullptr); kvstore->asyncRunReadIndexWorkers(); } From 64b14b5b84ecd139c8b64927db2ee37b524d8155 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Wed, 16 Aug 2023 10:26:37 +0800 Subject: [PATCH 65/66] update proxy Signed-off-by: CalvinNeo --- contrib/tiflash-proxy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index edcc0a665ec..2c7dc1ae62e 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit edcc0a665ecb42ef1c2c07ebcf1714dca8e44d1c +Subproject commit 2c7dc1ae62e24b4eaac3e9c19e42d8db3f4e6fde From 1e59422a6b0f19b69c7913a375bb573dbc61f7a0 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Wed, 16 Aug 2023 10:47:12 +0800 Subject: [PATCH 66/66] f Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index ea9050e1916..f2e66416906 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -346,7 +346,7 @@ class KVStore final : private boost::noncopyable std::optional region_task_lock, PersistRegionReason reason, const char * extra_msg) const; - + void releaseReadIndexWorkers(); void handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTaskLock &);