diff --git a/dbms/src/Debug/dbgFuncMisc.cpp b/dbms/src/Debug/dbgFuncMisc.cpp index ed0739879cf..8563aaf7433 100644 --- a/dbms/src/Debug/dbgFuncMisc.cpp +++ b/dbms/src/Debug/dbgFuncMisc.cpp @@ -22,28 +22,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/ColumnFile/ColumnFileBig.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp index 1e99af00295..a48648f7e9c 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.cpp @@ -35,8 +35,21 @@ void ColumnFileBig::calculateStat(const DMContext & context) auto index_cache = context.db_context.getGlobalContext().getMinMaxIndexCache(); auto hash_salt = context.hash_salt; +<<<<<<< HEAD auto pack_filter = DMFilePackFilter::loadFrom(file, index_cache, hash_salt, {segment_range}, EMPTY_FILTER, {}, context.db_context.getFileProvider(), context.getReadLimiter()); +======= + auto pack_filter = DMFilePackFilter::loadFrom( + file, + index_cache, + /*set_cache_if_miss*/ false, + {segment_range}, + EMPTY_FILTER, + {}, + context.db_context.getFileProvider(), + context.getReadLimiter(), + /*tracing_logger*/ nullptr); +>>>>>>> 5e0c2f8f2e (fix empty segment cannot merge after gc and avoid write index data for empty dmfile (#4500)) std::tie(valid_rows, valid_bytes) = pack_filter.validRowsAndBytes(); } diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index f06515f807f..ae5438d93c5 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1593,7 +1593,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(); @@ -1602,43 +1602,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; diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.cpp new file mode 100644 index 00000000000..749423b0bfa --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.cpp @@ -0,0 +1,79 @@ +// Copyright 2022 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 +#include + +namespace DB::DM +{ +DMFileBlockInputStreamBuilder::DMFileBlockInputStreamBuilder(const Context & context) + : file_provider(context.getFileProvider()) + , read_limiter(context.getReadLimiter()) +{ + // init from global context + const auto & global_context = context.getGlobalContext(); + setCaches(global_context.getMarkCache(), global_context.getMinMaxIndexCache()); + // init from settings + setFromSettings(context.getSettingsRef()); +} + +DMFileBlockInputStreamPtr DMFileBlockInputStreamBuilder::build(const DMFilePtr & dmfile, const ColumnDefines & read_columns, const RowKeyRanges & rowkey_ranges) +{ + if (dmfile->getStatus() != DMFile::Status::READABLE) + throw Exception(fmt::format( + "DMFile [{}] is expected to be in READABLE status, but: {}", + dmfile->fileId(), + DMFile::statusString(dmfile->getStatus())), + ErrorCodes::LOGICAL_ERROR); + + // if `rowkey_ranges` is empty, we unconditionally read all packs + // `rowkey_ranges` and `is_common_handle` will only be useful in clean read mode. + // It is safe to ignore them here. + if (unlikely(rowkey_ranges.empty() && enable_clean_read)) + throw Exception("rowkey ranges shouldn't be empty with clean-read enabled", ErrorCodes::LOGICAL_ERROR); + + bool is_common_handle = !rowkey_ranges.empty() && rowkey_ranges[0].is_common_handle; + + DMFilePackFilter pack_filter = DMFilePackFilter::loadFrom( + dmfile, + index_cache, + /*set_cache_if_miss*/ true, + rowkey_ranges, + rs_filter, + read_packs, + file_provider, + read_limiter, + tracing_logger); + + DMFileReader reader( + dmfile, + read_columns, + is_common_handle, + enable_clean_read, + max_data_version, + std::move(pack_filter), + mark_cache, + enable_column_cache, + column_cache, + aio_threshold, + max_read_buffer_size, + file_provider, + read_limiter, + rows_threshold_per_read, + read_one_pack_every_time, + tracing_logger); + + return std::make_shared(std::move(reader)); +} +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h index a58a54b53bd..80964ac380f 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h @@ -39,6 +39,7 @@ using IdSetPtr = std::shared_ptr; class DMFilePackFilter { public: +<<<<<<< HEAD static DMFilePackFilter loadFrom(const DMFilePtr & dmfile, const MinMaxIndexCachePtr & index_cache, UInt64 hash_salt, @@ -49,6 +50,20 @@ class DMFilePackFilter const ReadLimiterPtr & read_limiter) { auto pack_filter = DMFilePackFilter(dmfile, index_cache, hash_salt, rowkey_ranges, filter, read_packs, file_provider, read_limiter); +======= + static DMFilePackFilter loadFrom( + const DMFilePtr & dmfile, + const MinMaxIndexCachePtr & index_cache, + bool set_cache_if_miss, + const RowKeyRanges & rowkey_ranges, + const RSOperatorPtr & filter, + const IdSetPtr & read_packs, + const FileProviderPtr & file_provider, + const ReadLimiterPtr & read_limiter, + const DB::LoggerPtr & tracing_logger) + { + auto pack_filter = DMFilePackFilter(dmfile, index_cache, set_cache_if_miss, rowkey_ranges, filter, read_packs, file_provider, read_limiter, tracing_logger); +>>>>>>> 5e0c2f8f2e (fix empty segment cannot merge after gc and avoid write index data for empty dmfile (#4500)) pack_filter.init(); return pack_filter; } @@ -100,7 +115,11 @@ class DMFilePackFilter private: DMFilePackFilter(const DMFilePtr & dmfile_, const MinMaxIndexCachePtr & index_cache_, +<<<<<<< HEAD UInt64 hash_salt_, +======= + bool set_cache_if_miss_, +>>>>>>> 5e0c2f8f2e (fix empty segment cannot merge after gc and avoid write index data for empty dmfile (#4500)) 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 @@ -108,7 +127,11 @@ class DMFilePackFilter const ReadLimiterPtr & read_limiter_) : dmfile(dmfile_) , index_cache(index_cache_) +<<<<<<< HEAD , hash_salt(hash_salt_) +======= + , set_cache_if_miss(set_cache_if_miss_) +>>>>>>> 5e0c2f8f2e (fix empty segment cannot merge after gc and avoid write index data for empty dmfile (#4500)) , rowkey_ranges(rowkey_ranges_) , filter(filter_) , read_packs(read_packs_) @@ -214,6 +237,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) { @@ -221,13 +245,16 @@ class DMFilePackFilter const auto file_name_base = DMFile::getFileNameBase(col_id); auto load = [&]() { + auto index_file_size = dmfile->colIndexSize(file_name_base); + if (index_file_size == 0) + return std::make_shared(*type); if (!dmfile->configuration) { auto index_buf = ReadBufferFromFileProvider( file_provider, dmfile->colIndexPath(file_name_base), dmfile->encryptionIndexPath(file_name_base), - std::min(static_cast(DBMS_DEFAULT_BUFFER_SIZE), dmfile->colIndexSize(file_name_base)), + std::min(static_cast(DBMS_DEFAULT_BUFFER_SIZE), index_file_size), read_limiter); index_buf.seek(dmfile->colIndexOffset(file_name_base)); return MinMaxIndex::read(*type, index_buf, dmfile->colIndexSize(file_name_base)); @@ -242,21 +269,24 @@ class DMFilePackFilter dmfile->configuration->getChecksumAlgorithm(), dmfile->configuration->getChecksumFrameLength()); index_buf->seek(dmfile->colIndexOffset(file_name_base)); - auto file_size = dmfile->colIndexSize(file_name_base); auto header_size = dmfile->configuration->getChecksumHeaderLength(); auto frame_total_size = dmfile->configuration->getChecksumFrameLength(); - auto frame_count = file_size / frame_total_size + (file_size % frame_total_size != 0); - return MinMaxIndex::read(*type, *index_buf, file_size - header_size * frame_count); + auto frame_count = index_file_size / frame_total_size + (index_file_size % frame_total_size != 0); + return MinMaxIndex::read(*type, *index_buf, index_file_size - header_size * frame_count); } }; 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)); } @@ -269,13 +299,17 @@ 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; +<<<<<<< HEAD UInt64 hash_salt; +======= + bool set_cache_if_miss; +>>>>>>> 5e0c2f8f2e (fix empty segment cannot merge after gc and avoid write index data for empty dmfile (#4500)) RowKeyRanges rowkey_ranges; RSOperatorPtr filter; IdSetPtr read_packs; diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp index 77aca915957..3701e9c6cca 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp @@ -121,6 +121,7 @@ void DMFileWriter::addStreams(ColId col_id, DataTypePtr type, bool do_index) void DMFileWriter::write(const Block & block, const BlockProperty & block_property) { + is_empty_file = false; DMFile::PackStat stat; stat.rows = block.rows(); stat.not_clean = block_property.not_clean_rows; @@ -128,17 +129,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()) @@ -359,7 +360,8 @@ void DMFileWriter::finalizeColumn(ColId col_id, DataTypePtr type) dmfile->encryptionIndexPath(stream_name), false, write_limiter); - stream->minmaxes->write(*type, buf); + if (!is_empty_file) + stream->minmaxes->write(*type, buf); buf.sync(); bytes_written += buf.getMaterializedBytes(); } @@ -372,7 +374,8 @@ void DMFileWriter::finalizeColumn(ColId col_id, DataTypePtr type) write_limiter, dmfile->configuration->getChecksumAlgorithm(), dmfile->configuration->getChecksumFrameLength()); - stream->minmaxes->write(*type, *buf); + if (!is_empty_file) + stream->minmaxes->write(*type, *buf); buf->sync(); bytes_written += buf->getMaterializedBytes(); #ifndef NDEBUG diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h index 2865630eff2..b9868444162 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h @@ -249,6 +249,9 @@ class DMFileWriter FileProviderPtr file_provider; WriteLimiterPtr write_limiter; + + // use to avoid write index data for empty file + bool is_empty_file = true; }; } // namespace DM diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp index 5a9303bec34..294b568c28e 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -50,6 +50,7 @@ void StableValueSpace::setFiles(const DMFiles & files_, const RowKeyRange & rang auto hash_salt = dm_context->hash_salt; for (const auto & file : files_) { +<<<<<<< HEAD auto pack_filter = DMFilePackFilter::loadFrom(file, index_cache, hash_salt, @@ -58,6 +59,18 @@ void StableValueSpace::setFiles(const DMFiles & files_, const RowKeyRange & rang {}, dm_context->db_context.getFileProvider(), dm_context->getReadLimiter()); +======= + auto pack_filter = DMFilePackFilter::loadFrom( + file, + index_cache, + /*set_cache_if_miss*/ true, + {range}, + EMPTY_FILTER, + {}, + dm_context->db_context.getFileProvider(), + dm_context->getReadLimiter(), + /*tracing_logger*/ nullptr); +>>>>>>> 5e0c2f8f2e (fix empty segment cannot merge after gc and avoid write index data for empty dmfile (#4500)) auto [file_valid_rows, file_valid_bytes] = pack_filter.validRowsAndBytes(); rows += file_valid_rows; bytes += file_valid_bytes; @@ -238,6 +251,7 @@ void StableValueSpace::calculateStableProperty(const DMContext & context, const } mvcc_stream->readSuffix(); } +<<<<<<< HEAD auto pack_filter = DMFilePackFilter::loadFrom(file, context.db_context.getGlobalContext().getMinMaxIndexCache(), context.hash_salt, @@ -246,6 +260,18 @@ void StableValueSpace::calculateStableProperty(const DMContext & context, const {}, context.db_context.getFileProvider(), context.getReadLimiter()); +======= + auto pack_filter = DMFilePackFilter::loadFrom( + file, + context.db_context.getGlobalContext().getMinMaxIndexCache(), + /*set_cache_if_miss*/ false, + {rowkey_range}, + EMPTY_FILTER, + {}, + context.db_context.getFileProvider(), + context.getReadLimiter(), + /*tracing_logger*/ nullptr); +>>>>>>> 5e0c2f8f2e (fix empty segment cannot merge after gc and avoid write index data for empty dmfile (#4500)) const auto & use_packs = pack_filter.getUsePacks(); size_t new_pack_properties_index = 0; bool use_new_pack_properties = pack_properties.property_size() == 0; @@ -359,13 +385,12 @@ 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. + // 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 (auto & f : stable->files) { +<<<<<<< HEAD auto filter = DMFilePackFilter::loadFrom(f, // nullptr, context.hash_salt, @@ -374,6 +399,18 @@ RowsAndBytes StableValueSpace::Snapshot::getApproxRowsAndBytes(const DMContext & IdSetPtr{}, context.db_context.getFileProvider(), context.getReadLimiter()); +======= + auto filter = DMFilePackFilter::loadFrom( + f, + context.db_context.getGlobalContext().getMinMaxIndexCache(), + /*set_cache_if_miss*/ false, + {range}, + RSOperatorPtr{}, + IdSetPtr{}, + context.db_context.getFileProvider(), + context.getReadLimiter(), + /*tracing_logger*/ nullptr); +>>>>>>> 5e0c2f8f2e (fix empty segment cannot merge after gc and avoid write index data for empty dmfile (#4500)) const auto & pack_stats = f->getPackStats(); const auto & use_packs = filter.getUsePacks(); for (size_t i = 0; i < pack_stats.size(); ++i) 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 799d770358e..2aa3085e5bb 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 @@ -54,7 +54,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 │ └─────────────────────────────────────────────┘ @@ -64,7 +64,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 │ └─────────────────────────────────────────────┘ @@ -74,7 +74,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 │ └─────────────────────────────────────────────┘ @@ -84,14 +84,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 │ └─────────────────────────────────────────────┘ @@ -101,14 +101,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 │ └─────────────────────────────────────────────┘ @@ -118,14 +118,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 │ └─────────────────────────────────────────────┘ @@ -135,14 +135,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 │ └─────────────────────────────────────────────┘ @@ -152,7 +152,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 a80460d8c30..fc3f5c59ca1 100644 --- a/tests/fullstack-test-dt/expr/timestamp_filter.test +++ b/tests/fullstack-test-dt/expr/timestamp_filter.test @@ -44,7 +44,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 │ └─────────────────────────────────────────────┘ @@ -57,26 +57,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 │ └─────────────────────────────────────────────┘ @@ -86,7 +86,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 │ └─────────────────────────────────────────────┘ @@ -96,7 +96,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 │ └─────────────────────────────────────────────┘ @@ -109,14 +109,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 │ └─────────────────────────────────────────────┘ @@ -155,7 +155,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 │ └─────────────────────────────────────────────┘ @@ -168,7 +168,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 │ └─────────────────────────────────────────────┘