From dbfec4a525feabdc6ec05ea1a715ad5b7ebba8ff Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 1 Mar 2023 10:54:07 +0800 Subject: [PATCH 01/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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/32] 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 4366d87d112a2edbbc7d26499d027b91429a27b8 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 7 Aug 2023 16:54:16 +0800 Subject: [PATCH 32/32] rrrr Signed-off-by: CalvinNeo --- dbms/src/Storages/Transaction/KVStore.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index fadfed81774..f82ee6498ce 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -1148,7 +1148,6 @@ void KVStore::proactiveFlushCacheAndRegion(TMTContext & tmt, const DM::RowKeyRan // 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