From e87b54b0f09c3050d8add88fbd4179afb08d2a13 Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Thu, 2 Jun 2022 16:52:28 +0800 Subject: [PATCH] fix empty segment cannot merge after gc (#4523) close pingcap/tiflash#4511 --- dbms/src/Debug/dbgFuncMisc.cpp | 63 ++++++- .../DeltaMerge/Delta/DeltaPackFile.cpp | 6 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 164 +++++++++--------- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 6 +- .../DeltaMerge/File/DMFilePackFilter.h | 17 +- .../Storages/DeltaMerge/File/DMFileReader.cpp | 6 +- .../Storages/DeltaMerge/File/DMFileReader.h | 2 +- .../Storages/DeltaMerge/File/DMFileWriter.cpp | 6 +- .../Storages/DeltaMerge/StableValueSpace.cpp | 48 ++--- .../misc/timestamp_rough_set_filter.test | 24 +-- .../expr/timestamp_filter.test | 20 +-- 11 files changed, 211 insertions(+), 151 deletions(-) diff --git a/dbms/src/Debug/dbgFuncMisc.cpp b/dbms/src/Debug/dbgFuncMisc.cpp index 374e6cd4007..105c974716c 100644 --- a/dbms/src/Debug/dbgFuncMisc.cpp +++ b/dbms/src/Debug/dbgFuncMisc.cpp @@ -8,28 +8,73 @@ namespace DB { +inline size_t getThreadIdForLog(const String & line) +{ + auto sub_line = line.substr(line.find("thread_id=")); + std::regex rx(R"((0|[1-9][0-9]*))"); + std::smatch m; + if (regex_search(sub_line, m, rx)) + return std::stoi(m[1]); + else + return 0; +} + +// Usage example: +// The first argument is the key you want to search. +// For example, we want to search the key 'RSFilter exclude rate' in log file, and get the value following it. +// So we can use it as the first argument. +// But many kind of thread can print this keyword, +// so we can use the second argument to specify a keyword that may just be printed by a specific kind of thread. +// Here we use 'Rough set filter' to specify we just want to search read thread. +// And the complete command is the following: +// DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') +// TODO: this is still a too hack way to do test, but cannot think a better way now. void dbgFuncSearchLogForKey(Context & context, const ASTs & args, DBGInvoker::Printer output) { - if (args.size() < 1) - throw Exception("Args not matched, should be: key", ErrorCodes::BAD_ARGUMENTS); + if (args.size() < 2) + throw Exception("Args not matched, should be: key, thread_hint", ErrorCodes::BAD_ARGUMENTS); String key = safeGet(typeid_cast(*args[0]).value); + // the candidate line must be printed by a thread which also print a line contains `thread_hint` + String thread_hint = safeGet(typeid_cast(*args[1]).value); auto log_path = context.getConfigRef().getString("logger.log"); std::ifstream file(log_path); - std::vector line_candidates; - String line; - while (std::getline(file, line)) + // get the lines containing `thread_hint` and `key` + std::vector thread_hint_line_candidates; + std::vector key_line_candidates; { - if ((line.find(key) != String::npos) && (line.find("DBGInvoke") == String::npos)) - line_candidates.emplace_back(line); + String line; + while (std::getline(file, line)) + { + if ((line.find(thread_hint) != String::npos) && (line.find("DBGInvoke") == String::npos)) + thread_hint_line_candidates.emplace_back(line); + else if ((line.find(key) != String::npos) && (line.find("DBGInvoke") == String::npos)) + key_line_candidates.emplace_back(line); + } } - if (line_candidates.empty()) + // get target thread id + if (thread_hint_line_candidates.empty() || key_line_candidates.empty()) { output("Invalid"); return; } - auto & target_line = line_candidates.back(); + size_t target_thread_id = getThreadIdForLog(thread_hint_line_candidates.back()); + if (target_thread_id == 0) + { + output("Invalid"); + return; + } + String target_line; + for (auto iter = key_line_candidates.rbegin(); iter != key_line_candidates.rend(); iter++) + { + if (getThreadIdForLog(*iter) == target_thread_id) + { + target_line = *iter; + break; + } + } + // try parse the first number following the key auto sub_line = target_line.substr(target_line.find(key)); std::regex rx(R"([+-]?([0-9]+([.][0-9]*)?|[.][0-9]+))"); std::smatch m; diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.cpp index 70736b55619..e46ddc20dfb 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.cpp @@ -22,7 +22,7 @@ void DeltaPackFile::calculateStat(const DMContext & context) auto hash_salt = context.hash_salt; auto pack_filter - = DMFilePackFilter::loadFrom(file, index_cache, hash_salt, {segment_range}, EMPTY_FILTER, {}, context.db_context.getFileProvider(), context.getReadLimiter()); + = DMFilePackFilter::loadFrom(file, index_cache, hash_salt, /*set_cache_if_miss*/ false, {segment_range}, EMPTY_FILTER, {}, context.db_context.getFileProvider(), context.getReadLimiter()); std::tie(valid_rows, valid_bytes) = pack_filter.validRowsAndBytes(); } @@ -56,7 +56,7 @@ DeltaPackPtr DeltaPackFile::deserializeMetadata(DMContext & context, // auto dmfile = DMFile::restore(context.db_context.getFileProvider(), file_id, file_ref_id, file_parent_path, DMFile::ReadMetaMode::all()); - auto dp_file = new DeltaPackFile(dmfile, valid_rows, valid_bytes, segment_range); + auto * dp_file = new DeltaPackFile(dmfile, valid_rows, valid_bytes, segment_range); return std::shared_ptr(dp_file); } @@ -132,7 +132,7 @@ size_t DPFileReader::readRowsOnce(MutableColumns & output_cols, // const RowKeyRange * range) { auto read_next_block = [&, this]() -> bool { - rows_before_cur_block += ((bool)cur_block) ? cur_block.rows() : 0; + rows_before_cur_block += (static_cast(cur_block)) ? cur_block.rows() : 0; cur_block_data.clear(); cur_block = file_stream->read(); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index a21d2e0a1c4..e2782252cc8 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -272,7 +272,7 @@ void DeltaMergeStore::setUpBackgroundTask(const DMContextPtr & dm_context) }; auto dmfile_remover = [&](const PageStorage::PathAndIdsVec & path_and_ids_vec, const std::set & valid_ids) { auto delegate = path_pool.getStableDiskDelegator(); - for (auto & [path, ids] : path_and_ids_vec) + for (const auto & [path, ids] : path_and_ids_vec) { for (auto id : ids) { @@ -508,7 +508,7 @@ void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_ if (segment->hasAbandoned()) continue; - auto & rowkey_range = segment->getRowKeyRange(); + const auto & rowkey_range = segment->getRowKeyRange(); auto [cur_offset, cur_limit] = rowkey_range.getPosRange(handle_column, offset, rows - offset); if (unlikely(cur_offset != offset)) @@ -646,7 +646,7 @@ void DeltaMergeStore::ingestFiles( // TODO: If tiflash crash during the middle of ingesting, we may leave some DTFiles on disk and // they can not be deleted. We should find a way to cleanup those files. WriteBatches ingest_wbs(storage_pool, dm_context->getWriteLimiter()); - if (files.size() > 0) + if (!files.empty()) { for (const auto & file : files) { @@ -692,7 +692,7 @@ void DeltaMergeStore::ingestFiles( { /// Generate DMFile instance with a new ref_id pointed to the file_id. auto file_id = file->fileId(); - auto & file_parent_path = file->parentPath(); + const auto & file_parent_path = file->parentPath(); auto ref_id = storage_pool.newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); auto ref_file = DMFile::restore(file_provider, file_id, ref_id, file_parent_path, DMFile::ReadMetaMode::all()); @@ -1057,7 +1057,7 @@ void DeltaMergeStore::waitForWrite(const DMContextPtr & dm_context, const Segmen size_t segment_bytes = segment->getEstimatedBytes(); // The speed of delta merge in a very bad situation we assume. It should be a very conservative value. - size_t _10MB = 10 << 20; + const size_t k10mb = 10 << 20; size_t stop_write_delta_rows = dm_context->db_context.getSettingsRef().dt_segment_stop_write_delta_rows; size_t stop_write_delta_bytes = dm_context->db_context.getSettingsRef().dt_segment_stop_write_delta_size; @@ -1067,7 +1067,7 @@ void DeltaMergeStore::waitForWrite(const DMContextPtr & dm_context, const Segmen if (delta_rows >= stop_write_delta_rows || delta_bytes >= stop_write_delta_bytes) sleep_ms = std::numeric_limits::max(); else - sleep_ms = (double)segment_bytes / _10MB * 1000 * wait_duration_factor; + sleep_ms = static_cast(segment_bytes) / k10mb * 1000 * wait_duration_factor; // checkSegmentUpdate could do foreground merge delta, so call it before sleep. checkSegmentUpdate(dm_context, segment, ThreadType::Write); @@ -1093,7 +1093,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const { if (segment->hasAbandoned()) return; - auto & delta = segment->getDelta(); + const auto & delta = segment->getDelta(); size_t delta_saved_rows = delta->getRows(/* use_unsaved */ false); size_t delta_saved_bytes = delta->getBytes(/* use_unsaved */ false); @@ -1150,7 +1150,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const bool should_merge = segment_rows < segment_limit_rows / 3 && segment_bytes < segment_limit_bytes / 3; // Don't do compact on starting up. - bool should_compact = (thread_type != ThreadType::Init) && std::max((Int64)pack_count - delta_last_try_compact_packs, 0) >= 10; + bool should_compact = (thread_type != ThreadType::Init) && std::max(static_cast(pack_count) - delta_last_try_compact_packs, 0) >= 10; // Don't do background place index if we limit DeltaIndex cache. bool should_place_delta_index = !dm_context->db_context.isDeltaIndexLimited() @@ -1199,7 +1199,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const /// Now start trying structure update. - auto getMergeSibling = [&]() -> SegmentPtr { + auto get_merge_sibling = [&]() -> SegmentPtr { /// For complexity reason, currently we only try to merge with next segment. Normally it is good enough. // The last segment cannot be merged. @@ -1270,16 +1270,13 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto my_should_split = my_segment_rows >= dm_context->segment_limit_rows * 3; if (my_should_split && !my_segment->isSplitForbidden()) { - if (segmentSplit(*dm_context, my_segment, true).first) - return true; - else - return false; + return segmentSplit(*dm_context, my_segment, true).first != nullptr; } return false; }; auto try_bg_merge = [&]() { SegmentPtr merge_sibling; - if (should_merge && (merge_sibling = getMergeSibling())) + if (should_merge && (merge_sibling = get_merge_sibling())) { try_add_background_task(BackgroundTask{TaskType::Merge, dm_context, segment, merge_sibling}); return true; @@ -1442,7 +1439,7 @@ bool shouldCompactStable(const SegmentPtr & seg, DB::Timestamp gc_safepoint, dou if (ratio_threshold < 1.0) return true; - auto & property = seg->getStable()->getStableProperty(); + const auto & property = seg->getStable()->getStableProperty(); LOG_TRACE(log, __PRETTY_FUNCTION__ << property.toDebugString()); // No data older than safe_point to GC. if (property.gc_hint_version > gc_safepoint) @@ -1535,7 +1532,7 @@ UInt64 DeltaMergeStore::onSyncGc(Int64 limit) } assert(segment != nullptr); - if (segment->hasAbandoned() || segment->getLastCheckGCSafePoint() >= gc_safe_point || segment_snap == nullptr) + if (segment->hasAbandoned() || segment_snap == nullptr) continue; const auto segment_id = segment->segmentId(); @@ -1544,43 +1541,52 @@ UInt64 DeltaMergeStore::onSyncGc(Int64 limit) // meet empty segment, try merge it if (segment_snap->getRows() == 0) { + // release segment_snap before checkSegmentUpdate, otherwise this segment is still in update status. + segment_snap = {}; checkSegmentUpdate(dm_context, segment, ThreadType::BG_GC); continue; } - // Avoid recheck this segment when gc_safe_point doesn't change regardless whether we trigger this segment's DeltaMerge or not. - // Because after we calculate StableProperty and compare it with this gc_safe_point, - // there is no need to recheck it again using the same gc_safe_point. - // On the other hand, if it should do DeltaMerge using this gc_safe_point, and the DeltaMerge is interruptted by other process, - // it's still worth to wait another gc_safe_point to check this segment again. - segment->setLastCheckGCSafePoint(gc_safe_point); - dm_context->min_version = gc_safe_point; - - // calculate StableProperty if needed - if (!segment->getStable()->isStablePropertyCached()) - segment->getStable()->calculateStableProperty(*dm_context, segment_range, isCommonHandle()); - try { // Check whether we should apply gc on this segment - const bool should_compact - = GC::shouldCompactStable( - segment, - gc_safe_point, - global_context.getSettingsRef().dt_bg_gc_ratio_threhold_to_trigger_gc, - log) - || GC::shouldCompactDeltaWithStable( - *dm_context, - segment_snap, - segment_range, - global_context.getSettingsRef().dt_bg_gc_delta_delete_ratio_to_trigger_gc, - log); + bool should_compact = false; + if (GC::shouldCompactDeltaWithStable( + *dm_context, + segment_snap, + segment_range, + global_context.getSettingsRef().dt_bg_gc_delta_delete_ratio_to_trigger_gc, + log)) + { + should_compact = true; + } + else if (segment->getLastCheckGCSafePoint() < gc_safe_point) + { + // Avoid recheck this segment when gc_safe_point doesn't change regardless whether we trigger this segment's DeltaMerge or not. + // Because after we calculate StableProperty and compare it with this gc_safe_point, + // there is no need to recheck it again using the same gc_safe_point. + // On the other hand, if it should do DeltaMerge using this gc_safe_point, and the DeltaMerge is interruptted by other process, + // it's still worth to wait another gc_safe_point to check this segment again. + segment->setLastCheckGCSafePoint(gc_safe_point); + dm_context->min_version = gc_safe_point; + + // calculate StableProperty if needed + if (!segment->getStable()->isStablePropertyCached()) + segment->getStable()->calculateStableProperty(*dm_context, segment_range, isCommonHandle()); + + should_compact = GC::shouldCompactStable( + segment, + gc_safe_point, + global_context.getSettingsRef().dt_bg_gc_ratio_threhold_to_trigger_gc, + log); + } bool finish_gc_on_segment = false; if (should_compact) { if (segment = segmentMergeDelta(*dm_context, segment, TaskRunThread::BackgroundGCThread, segment_snap); segment) { // Continue to check whether we need to apply more tasks on this segment + segment_snap = {}; checkSegmentUpdate(dm_context, segment, ThreadType::BG_GC); gc_segments_num++; finish_gc_on_segment = true; @@ -1639,8 +1645,8 @@ SegmentPair DeltaMergeStore::segmentSplit(DMContext & dm_context, const SegmentP } // Not counting the early give up action. - auto delta_bytes = (Int64)segment_snap->delta->getBytes(); - auto delta_rows = (Int64)segment_snap->delta->getRows(); + auto delta_bytes = static_cast(segment_snap->delta->getBytes()); + auto delta_rows = static_cast(segment_snap->delta->getRows()); size_t duplicated_bytes = 0; size_t duplicated_rows = 0; @@ -1767,8 +1773,8 @@ void DeltaMergeStore::segmentMerge(DMContext & dm_context, const SegmentPtr & le } // Not counting the early give up action. - auto delta_bytes = (Int64)left_snap->delta->getBytes() + right_snap->getBytes(); - auto delta_rows = (Int64)left_snap->delta->getRows() + right_snap->getRows(); + auto delta_bytes = static_cast(left_snap->delta->getBytes()) + right_snap->getBytes(); + auto delta_rows = static_cast(left_snap->delta->getRows()) + right_snap->getRows(); CurrentMetrics::Increment cur_dm_segments{CurrentMetrics::DT_SegmentMerge}; GET_METRIC(tiflash_storage_subtask_count, type_seg_merge).Increment(); @@ -1861,12 +1867,12 @@ SegmentPtr DeltaMergeStore::segmentMergeDelta( } // Not counting the early give up action. - auto delta_bytes = (Int64)segment_snap->delta->getBytes(); - auto delta_rows = (Int64)segment_snap->delta->getRows(); + auto delta_bytes = static_cast(segment_snap->delta->getBytes()); + auto delta_rows = static_cast(segment_snap->delta->getRows()); CurrentMetrics::Increment cur_dm_segments{CurrentMetrics::DT_DeltaMerge}; - CurrentMetrics::Increment cur_dm_total_bytes{CurrentMetrics::DT_DeltaMergeTotalBytes, (Int64)segment_snap->getBytes()}; - CurrentMetrics::Increment cur_dm_total_rows{CurrentMetrics::DT_DeltaMergeTotalRows, (Int64)segment_snap->getRows()}; + CurrentMetrics::Increment cur_dm_total_bytes{CurrentMetrics::DT_DeltaMergeTotalBytes, static_cast(segment_snap->getBytes())}; + CurrentMetrics::Increment cur_dm_total_rows{CurrentMetrics::DT_DeltaMergeTotalRows, static_cast(segment_snap->getRows())}; switch (run_thread) { @@ -2097,7 +2103,7 @@ void DeltaMergeStore::restoreStableFiles() auto path_delegate = path_pool.getStableDiskDelegator(); for (const auto & root_path : path_delegate.listPaths()) { - for (auto & file_id : DMFile::listAllInPath(file_provider, root_path, options)) + for (const auto & file_id : DMFile::listAllInPath(file_provider, root_path, options)) { auto dmfile = DMFile::restore(file_provider, file_id, /* ref_id= */ 0, root_path, DMFile::ReadMetaMode::diskSizeOnly()); path_delegate.addDTFile(file_id, dmfile->getBytesOnDisk(), root_path); @@ -2113,15 +2119,15 @@ DeltaMergeStoreStat DeltaMergeStore::getStat() stat.segment_count = segments.size(); - long total_placed_rows = 0; - long total_delta_cache_rows = 0; + Int64 total_placed_rows = 0; + Int64 total_delta_cache_rows = 0; Float64 total_delta_cache_size = 0; - long total_delta_valid_cache_rows = 0; + Int64 total_delta_valid_cache_rows = 0; for (const auto & [handle, segment] : segments) { (void)handle; - auto & delta = segment->getDelta(); - auto & stable = segment->getStable(); + const auto & delta = segment->getDelta(); + const auto & stable = segment->getStable(); total_placed_rows += delta->getPlacedDeltaRows(); @@ -2159,31 +2165,31 @@ DeltaMergeStoreStat DeltaMergeStore::getStat() } } - stat.delta_rate_rows = (Float64)stat.total_delta_rows / stat.total_rows; - stat.delta_rate_segments = (Float64)stat.delta_count / stat.segment_count; + stat.delta_rate_rows = static_cast(stat.total_delta_rows) / stat.total_rows; + stat.delta_rate_segments = static_cast(stat.delta_count) / stat.segment_count; - stat.delta_placed_rate = (Float64)total_placed_rows / stat.total_delta_rows; + stat.delta_placed_rate = static_cast(total_placed_rows) / stat.total_delta_rows; stat.delta_cache_size = total_delta_cache_size; - stat.delta_cache_rate = (Float64)total_delta_valid_cache_rows / stat.total_delta_rows; - stat.delta_cache_wasted_rate = (Float64)(total_delta_cache_rows - total_delta_valid_cache_rows) / total_delta_valid_cache_rows; + stat.delta_cache_rate = static_cast(total_delta_valid_cache_rows) / stat.total_delta_rows; + stat.delta_cache_wasted_rate = static_cast(total_delta_cache_rows - total_delta_valid_cache_rows) / total_delta_valid_cache_rows; - stat.avg_segment_rows = (Float64)stat.total_rows / stat.segment_count; - stat.avg_segment_size = (Float64)stat.total_size / stat.segment_count; + stat.avg_segment_rows = static_cast(stat.total_rows) / stat.segment_count; + stat.avg_segment_size = static_cast(stat.total_size) / stat.segment_count; - stat.avg_delta_rows = (Float64)stat.total_delta_rows / stat.delta_count; - stat.avg_delta_size = (Float64)stat.total_delta_size / stat.delta_count; - stat.avg_delta_delete_ranges = (Float64)stat.total_delete_ranges / stat.delta_count; + stat.avg_delta_rows = static_cast(stat.total_delta_rows) / stat.delta_count; + stat.avg_delta_size = static_cast(stat.total_delta_size) / stat.delta_count; + stat.avg_delta_delete_ranges = static_cast(stat.total_delete_ranges) / stat.delta_count; - stat.avg_stable_rows = (Float64)stat.total_stable_rows / stat.stable_count; - stat.avg_stable_size = (Float64)stat.total_stable_size / stat.stable_count; + stat.avg_stable_rows = static_cast(stat.total_stable_rows) / stat.stable_count; + stat.avg_stable_size = static_cast(stat.total_stable_size) / stat.stable_count; - stat.avg_pack_count_in_delta = (Float64)stat.total_pack_count_in_delta / stat.delta_count; - stat.avg_pack_rows_in_delta = (Float64)stat.total_delta_rows / stat.total_pack_count_in_delta; - stat.avg_pack_size_in_delta = (Float64)stat.total_delta_size / stat.total_pack_count_in_delta; + stat.avg_pack_count_in_delta = static_cast(stat.total_pack_count_in_delta) / stat.delta_count; + stat.avg_pack_rows_in_delta = static_cast(stat.total_delta_rows) / stat.total_pack_count_in_delta; + stat.avg_pack_size_in_delta = static_cast(stat.total_delta_size) / stat.total_pack_count_in_delta; - stat.avg_pack_count_in_stable = (Float64)stat.total_pack_count_in_stable / stat.stable_count; - stat.avg_pack_rows_in_stable = (Float64)stat.total_stable_rows / stat.total_pack_count_in_stable; - stat.avg_pack_size_in_stable = (Float64)stat.total_stable_size / stat.total_pack_count_in_stable; + stat.avg_pack_count_in_stable = static_cast(stat.total_pack_count_in_stable) / stat.stable_count; + stat.avg_pack_rows_in_stable = static_cast(stat.total_stable_rows) / stat.total_pack_count_in_stable; + stat.avg_pack_size_in_stable = static_cast(stat.total_stable_size) / stat.total_pack_count_in_stable; { std::tie(stat.storage_stable_num_snapshots, // @@ -2231,8 +2237,8 @@ SegmentStats DeltaMergeStore::getSegmentStats() (void)handle; SegmentStat stat; - auto & delta = segment->getDelta(); - auto & stable = segment->getStable(); + const auto & delta = segment->getDelta(); + const auto & stable = segment->getStable(); stat.segment_id = segment->segmentId(); stat.range = segment->getRowKeyRange(); @@ -2246,10 +2252,10 @@ SegmentStats DeltaMergeStore::getSegmentStats() stat.delta_pack_count = delta->getPackCount(); stat.stable_pack_count = stable->getPacks(); - stat.avg_delta_pack_rows = (Float64)delta->getRows() / stat.delta_pack_count; - stat.avg_stable_pack_rows = (Float64)stable->getRows() / stat.stable_pack_count; + stat.avg_delta_pack_rows = static_cast(delta->getRows()) / stat.delta_pack_count; + stat.avg_stable_pack_rows = static_cast(stable->getRows()) / stat.stable_pack_count; - stat.delta_rate = (Float64)delta->getRows() / stat.rows; + stat.delta_rate = static_cast(delta->getRows()) / stat.rows; stat.delta_cache_size = delta->getTotalCacheBytes(); stat.delta_index_size = delta->getDeltaIndexBytes(); @@ -2279,8 +2285,8 @@ SegmentReadTasks DeltaMergeStore::getReadTasksByRanges( while (range_it != sorted_ranges.end() && seg_it != segments.end()) { - auto & req_range = *range_it; - auto & seg_range = seg_it->second->getRowKeyRange(); + const auto & req_range = *range_it; + const auto & seg_range = seg_it->second->getRowKeyRange(); if (req_range.intersect(seg_range) && (read_segments.empty() || read_segments.count(seg_it->second->segmentId()))) { if (tasks.empty() || tasks.back()->segment != seg_it->second) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 4b170117c38..7f35357bbd5 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -281,7 +281,7 @@ class DeltaMergeStore : private boost::noncopyable DeltaMergeStore(Context & db_context, // bool data_path_contains_database_name, const String & db_name, - const String & tbl_name, + const String & table_name_, const ColumnDefines & columns, const ColumnDefine & handle, bool is_common_handle_, @@ -303,7 +303,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); + void write(const Context & db_context, const DB::Settings & db_settings, Block && to_write); void deleteRange(const Context & db_context, const DB::Settings & db_settings, const RowKeyRange & delete_range); @@ -329,7 +329,7 @@ class DeltaMergeStore : private boost::noncopyable /// Read all rows without MVCC filtering BlockInputStreams readRaw(const Context & db_context, const DB::Settings & db_settings, - const ColumnDefines & column_defines, + const ColumnDefines & columns_to_read, size_t num_streams, const SegmentIdSet & read_segments = {}); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h index 6adaff61c0a..a6c69c85727 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h @@ -28,13 +28,14 @@ class DMFilePackFilter static DMFilePackFilter loadFrom(const DMFilePtr & dmfile, const MinMaxIndexCachePtr & index_cache, UInt64 hash_salt, + bool set_cache_if_miss, const RowKeyRanges & rowkey_ranges, const RSOperatorPtr & filter, const IdSetPtr & read_packs, const FileProviderPtr & file_provider, const ReadLimiterPtr & read_limiter) { - auto pack_filter = DMFilePackFilter(dmfile, index_cache, hash_salt, rowkey_ranges, filter, read_packs, file_provider, read_limiter); + auto pack_filter = DMFilePackFilter(dmfile, index_cache, hash_salt, set_cache_if_miss, rowkey_ranges, filter, read_packs, file_provider, read_limiter); pack_filter.init(); return pack_filter; } @@ -87,6 +88,7 @@ class DMFilePackFilter DMFilePackFilter(const DMFilePtr & dmfile_, const MinMaxIndexCachePtr & index_cache_, UInt64 hash_salt_, + bool set_cache_if_miss_, const RowKeyRanges & rowkey_ranges_, // filter by handle range const RSOperatorPtr & filter_, // filter by push down where clause const IdSetPtr & read_packs_, // filter by pack index @@ -95,6 +97,7 @@ class DMFilePackFilter : dmfile(dmfile_) , index_cache(index_cache_) , hash_salt(hash_salt_) + , set_cache_if_miss(set_cache_if_miss_) , rowkey_ranges(rowkey_ranges_) , filter(filter_) , read_packs(read_packs_) @@ -199,6 +202,7 @@ class DMFilePackFilter const DMFilePtr & dmfile, const FileProviderPtr & file_provider, const MinMaxIndexCachePtr & index_cache, + bool set_cache_if_miss, ColId col_id, const ReadLimiterPtr & read_limiter) { @@ -235,13 +239,17 @@ class DMFilePackFilter } }; MinMaxIndexPtr minmax_index; - if (index_cache) + if (index_cache && set_cache_if_miss) { minmax_index = index_cache->getOrSet(dmfile->colIndexCacheKey(file_name_base), load); } else { - minmax_index = load(); + // try load from the cache first + if (index_cache) + minmax_index = index_cache->get(dmfile->colIndexCacheKey(file_name_base)); + if (!minmax_index) + minmax_index = load(); } indexes.emplace(col_id, RSIndex(type, minmax_index)); } @@ -254,13 +262,14 @@ class DMFilePackFilter if (!dmfile->isColIndexExist(col_id)) return; - loadIndex(param.indexes, dmfile, file_provider, index_cache, col_id, read_limiter); + loadIndex(param.indexes, dmfile, file_provider, index_cache, set_cache_if_miss, col_id, read_limiter); } private: DMFilePtr dmfile; MinMaxIndexCachePtr index_cache; UInt64 hash_salt; + bool set_cache_if_miss; RowKeyRanges rowkey_ranges; RSOperatorPtr filter; IdSetPtr read_packs; diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp index f5f83693726..28c669d025b 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp @@ -208,7 +208,7 @@ DMFileReader::DMFileReader( , read_columns(read_columns_) , enable_clean_read(enable_clean_read_) , max_read_version(max_read_version_) - , pack_filter(dmfile_, index_cache_, hash_salt_, rowkey_ranges_, filter_, read_packs_, file_provider_, read_limiter) + , pack_filter(dmfile_, index_cache_, hash_salt_, /*set_cache_if_miss*/ true, rowkey_ranges_, filter_, read_packs_, file_provider_, read_limiter) , handle_res(pack_filter.getHandleRes()) , use_packs(pack_filter.getUsePacks()) , skip_packs_by_column(read_columns.size(), 0) @@ -298,7 +298,7 @@ Block DMFileReader::read() // 0 means no limit size_t read_pack_limit = (single_file_mode || read_one_pack_every_time) ? 1 : 0; - auto & pack_stats = dmfile->getPackStats(); + const auto & pack_stats = dmfile->getPackStats(); size_t read_rows = 0; size_t not_clean_rows = 0; @@ -365,7 +365,7 @@ Block DMFileReader::read() } else if (cd.id == TAG_COLUMN_ID) { - column = cd.type->createColumnConst(read_rows, Field((UInt64)(pack_stats[start_pack_id].first_tag))); + column = cd.type->createColumnConst(read_rows, Field(static_cast(pack_stats[start_pack_id].first_tag))); } res.insert(ColumnWithTypeAndName{column, cd.type, cd.name, cd.id}); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.h b/dbms/src/Storages/DeltaMerge/File/DMFileReader.h index f52dc4e929a..432617e9bec 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileReader.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFileReader.h @@ -61,7 +61,7 @@ class DMFileReader // If you have no idea what it means, then simply set it to false. bool enable_clean_read_, // The the MVCC filter version. Used by clean read check. - UInt64 max_data_version_, + UInt64 max_read_version_, // filters const RowKeyRanges & rowkey_ranges_, const RSOperatorPtr & filter_, diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp index 79e1bc1bf8e..66b532cc9c4 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp @@ -107,17 +107,17 @@ void DMFileWriter::write(const Block & block, const BlockProperty & block_proper auto del_mark_column = tryGetByColumnId(block, TAG_COLUMN_ID).column; - const ColumnVector * del_mark = !del_mark_column ? nullptr : (const ColumnVector *)del_mark_column.get(); + const ColumnVector * del_mark = !del_mark_column ? nullptr : static_cast *>(del_mark_column.get()); for (auto & cd : write_columns) { - auto & col = getByColumnId(block, cd.id).column; + const auto & col = getByColumnId(block, cd.id).column; writeColumn(cd.id, *cd.type, *col, del_mark); if (cd.id == VERSION_COLUMN_ID) stat.first_version = col->get64(0); else if (cd.id == TAG_COLUMN_ID) - stat.first_tag = (UInt8)(col->get64(0)); + stat.first_tag = static_cast(col->get64(0)); } if (!options.flags.isSingleFile()) diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp index 2a10f22be37..d23ca7246c3 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -24,7 +24,7 @@ void StableValueSpace::setFiles(const DMFiles & files_, const RowKeyRange & rang if (range.all()) { - for (auto & file : files_) + for (const auto & file : files_) { rows += file->getRows(); bytes += file->getBytes(); @@ -34,11 +34,12 @@ void StableValueSpace::setFiles(const DMFiles & files_, const RowKeyRange & rang { auto index_cache = dm_context->db_context.getGlobalContext().getMinMaxIndexCache(); auto hash_salt = dm_context->hash_salt; - for (auto & file : files_) + for (const auto & file : files_) { auto pack_filter = DMFilePackFilter::loadFrom(file, index_cache, hash_salt, + /*set_cache_if_miss*/ true, {range}, EMPTY_FILTER, {}, @@ -61,7 +62,7 @@ void StableValueSpace::saveMeta(WriteBatch & meta_wb) writeIntBinary(STORAGE_FORMAT_CURRENT.stable, buf); writeIntBinary(valid_rows, buf); writeIntBinary(valid_bytes, buf); - writeIntBinary((UInt64)files.size(), buf); + writeIntBinary(static_cast(files.size()), buf); for (auto & f : files) writeIntBinary(f->refId(), buf); @@ -124,7 +125,7 @@ size_t StableValueSpace::getBytesOnDisk() const size_t StableValueSpace::getPacks() const { size_t packs = 0; - for (auto & file : files) + for (const auto & file : files) packs += file->getPacks(); return packs; } @@ -132,7 +133,7 @@ size_t StableValueSpace::getPacks() const String StableValueSpace::getDMFilesString() { String s; - for (auto & file : files) + for (const auto & file : files) s += "dmf_" + DB::toString(file->fileId()) + ","; if (!s.empty()) s.erase(s.length() - 1); @@ -141,13 +142,13 @@ String StableValueSpace::getDMFilesString() void StableValueSpace::enableDMFilesGC() { - for (auto & file : files) + for (const auto & file : files) file->enableGC(); } void StableValueSpace::recordRemovePacksPages(WriteBatches & wbs) const { - for (auto & file : files) + for (const auto & file : files) { // Here we should remove the ref id instead of file_id. // Because a dmfile could be used by several segments, and only after all ref_ids are removed, then the file_id removed. @@ -161,10 +162,9 @@ void StableValueSpace::calculateStableProperty(const DMContext & context, const property.num_versions = 0; property.num_puts = 0; property.num_rows = 0; - for (size_t i = 0; i < files.size(); i++) + for (const auto & file : files) { - auto & file = files[i]; - auto & pack_stats = file->getPackStats(); + const auto & pack_stats = file->getPackStats(); auto & pack_properties = file->getPackProperties(); if (pack_stats.empty()) continue; @@ -228,12 +228,13 @@ void StableValueSpace::calculateStableProperty(const DMContext & context, const auto pack_filter = DMFilePackFilter::loadFrom(file, context.db_context.getGlobalContext().getMinMaxIndexCache(), context.hash_salt, + /*set_cache_if_miss*/ false, {rowkey_range}, EMPTY_FILTER, {}, context.db_context.getFileProvider(), context.getReadLimiter()); - auto & use_packs = pack_filter.getUsePacks(); + const auto & use_packs = pack_filter.getUsePacks(); size_t new_pack_properties_index = 0; bool use_new_pack_properties = pack_properties.property_size() == 0; if (use_new_pack_properties) @@ -259,14 +260,14 @@ void StableValueSpace::calculateStableProperty(const DMContext & context, const property.num_puts += pack_stats[pack_id].rows - pack_stats[pack_id].not_clean; if (use_new_pack_properties) { - auto & pack_property = new_pack_properties.property(new_pack_properties_index); + const auto & pack_property = new_pack_properties.property(new_pack_properties_index); property.num_rows += pack_property.num_rows(); property.gc_hint_version = std::min(property.gc_hint_version, pack_property.gc_hint_version()); new_pack_properties_index += 1; } else { - auto & pack_property = pack_properties.property(pack_id); + const auto & pack_property = pack_properties.property(pack_id); property.num_rows += pack_property.num_rows(); property.gc_hint_version = std::min(property.gc_hint_version, pack_property.gc_hint_version()); } @@ -346,23 +347,22 @@ RowsAndBytes StableValueSpace::Snapshot::getApproxRowsAndBytes(const DMContext & size_t match_packs = 0; size_t total_match_rows = 0; size_t total_match_bytes = 0; - // Usually, this method will be called for some "cold" key ranges. Loading the index - // into cache may pollute the cache and make the hot index cache invalid. Set the - // index cache to nullptr so that the cache won't be polluted. - // TODO: We can use the cache if the index happens to exist in the cache, but - // don't refill the cache if the index does not exist. - for (auto & f : stable->files) + // Usually, this method will be called for some "cold" key ranges. + // Loading the index into cache may pollute the cache and make the hot index cache invalid. + // So don't refill the cache if the index does not exist. + for (const auto & f : stable->files) { auto filter = DMFilePackFilter::loadFrom(f, // - nullptr, + context.db_context.getGlobalContext().getMinMaxIndexCache(), context.hash_salt, + /*set_cache_if_miss*/ false, {range}, RSOperatorPtr{}, IdSetPtr{}, context.db_context.getFileProvider(), context.getReadLimiter()); - auto & pack_stats = f->getPackStats(); - auto & use_packs = filter.getUsePacks(); + const auto & pack_stats = f->getPackStats(); + const auto & use_packs = filter.getUsePacks(); for (size_t i = 0; i < pack_stats.size(); ++i) { if (use_packs[i]) @@ -375,8 +375,8 @@ RowsAndBytes StableValueSpace::Snapshot::getApproxRowsAndBytes(const DMContext & } if (!total_match_rows || !match_packs) return {0, 0}; - Float64 avg_pack_rows = total_match_rows / match_packs; - Float64 avg_pack_bytes = total_match_bytes / match_packs; + Float64 avg_pack_rows = static_cast(total_match_rows) / match_packs; + Float64 avg_pack_bytes = static_cast(total_match_bytes) / match_packs; // By average, the first and last pack are only half covered by the range. // And if this range only covers one pack, then return the pack's stat. size_t approx_rows = std::max(avg_pack_rows, total_match_rows - avg_pack_rows / 2); diff --git a/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test b/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test index 188b73ae65e..c7df348a58c 100644 --- a/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test +++ b/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test @@ -40,7 +40,7 @@ => DBGInvoke dag('select * from default.test where col_2 < cast_string_datetime(\'2019-06-10 09:00:00.00000\')') -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ @@ -50,7 +50,7 @@ │ 50 │ 2019-06-10 09:00:00.00000 │ └────────────┴───────────────────────────┘ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ @@ -60,7 +60,7 @@ # so '2019-06-10 17:00:00.00000'(tz_offset:28800) below is equal to '2019-06-10 09:00:00.00000' in UTC => DBGInvoke dag('select * from default.test where col_2 < cast_string_datetime(\'2019-06-10 17:00:00.00000\')',4,'encode_type:default,tz_offset:28800') -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ @@ -70,14 +70,14 @@ │ 50 │ 2019-06-10 09:00:00.00000 │ └────────────┴───────────────────────────┘ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ => DBGInvoke dag('select * from default.test where col_2 < cast_string_datetime(\'2019-06-10 04:00:00.00000\')',4,'encode_type:default,tz_name:America/Chicago') -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ @@ -87,14 +87,14 @@ │ 50 │ 2019-06-10 09:00:00.00000 │ └────────────┴───────────────────────────┘ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ => DBGInvoke dag('select * from default.test where col_2 > cast_string_datetime(\'2019-06-13 12:00:01.00000\')') -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ @@ -104,14 +104,14 @@ │ 55 │ 2019-06-13 12:00:01.00000 │ └────────────┴───────────────────────────┘ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ => DBGInvoke dag('select * from default.test where col_2 > cast_string_datetime(\'2019-06-13 20:00:01.00000\')',4,'encode_type:default,tz_offset:28800') -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ @@ -121,14 +121,14 @@ │ 55 │ 2019-06-13 12:00:01.00000 │ └────────────┴───────────────────────────┘ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ => DBGInvoke dag('select * from default.test where col_2 > cast_string_datetime(\'2019-06-13 07:00:01.00000\')',4,'encode_type:default,tz_name:America/Chicago') -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ @@ -138,7 +138,7 @@ │ 55 │ 2019-06-13 12:00:01.00000 │ └────────────┴───────────────────────────┘ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ diff --git a/tests/fullstack-test-dt/expr/timestamp_filter.test b/tests/fullstack-test-dt/expr/timestamp_filter.test index c3639c0dd62..d4f585fe735 100644 --- a/tests/fullstack-test-dt/expr/timestamp_filter.test +++ b/tests/fullstack-test-dt/expr/timestamp_filter.test @@ -30,26 +30,26 @@ mysql> SET time_zone = '+0:00'; set session tidb_isolation_read_engines='tiflash | 1 | 2000-01-01 10:00:00 | +----+---------------------+ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ mysql> SET time_zone = '+0:00'; set session tidb_isolation_read_engines='tiflash'; select * from test.t where ts != '2000-01-01 10:00:00'; -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ ## Tests the direction between column and literal mysql> SET time_zone = '+0:00'; set session tidb_isolation_read_engines='tiflash'; select * from test.t where ts > '2000-01-01 10:00:01'; -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ mysql> SET time_zone = '+0:00'; set session tidb_isolation_read_engines='tiflash'; select * from test.t where '2000-01-01 10:00:01' < ts; -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ @@ -59,7 +59,7 @@ mysql> SET time_zone = '+0:00'; set session tidb_isolation_read_engines='tiflash +----+---------------------+ | 1 | 2000-01-01 10:00:00 | +----+---------------------+ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ @@ -69,7 +69,7 @@ mysql> SET time_zone = '+0:00'; set session tidb_isolation_read_engines='tiflash +----+---------------------+ | 1 | 2000-01-01 10:00:00 | +----+---------------------+ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ @@ -82,14 +82,14 @@ mysql> SET time_zone = '+8:00'; set session tidb_isolation_read_engines='tiflash | 1 | 2000-01-01 18:00:00 | +----+---------------------+ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ mysql> SET time_zone = '+8:00'; set session tidb_isolation_read_engines='tiflash'; select * from test.t where ts != '2000-01-01 18:00:00'; -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 100.00 │ └─────────────────────────────────────────────┘ @@ -128,7 +128,7 @@ mysql> SET time_zone = '+0:00'; set session tidb_isolation_read_engines='tiflash | 1 | 2000-01-01 10:00:00 | +----+---------------------+ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘ @@ -141,7 +141,7 @@ mysql> SET time_zone = '+8:00'; set session tidb_isolation_read_engines='tiflash | 1 | 2000-01-01 18:00:00 | +----+---------------------+ -=> DBGInvoke search_log_for_key('RSFilter exclude rate') +=> DBGInvoke search_log_for_key('RSFilter exclude rate', 'Rough set filter') ┌─search_log_for_key("RSFilter exclude rate")─┐ │ 0.00 │ └─────────────────────────────────────────────┘