From 08bd317ce8437590428c213201fb014e39a86b20 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 4 Dec 2024 22:19:01 +0800 Subject: [PATCH 01/48] ci --- dbms/CMakeLists.txt | 4 - dbms/gtest.log | 1 + dbms/src/Interpreters/Settings.h | 1 + .../DeltaMerge/BitmapFilter/BitmapFilter.h | 2 + dbms/src/Storages/DeltaMerge/CMakeLists.txt | 1 + .../DeltaMerge/ColumnFile/ColumnFileBig.h | 2 + .../DeltaMerge/Delta/ColumnFileFlushTask.cpp | 4 +- .../DeltaMerge/Delta/DeltaValueSpace.h | 12 + dbms/src/Storages/DeltaMerge/File/DMFile.h | 6 + .../DeltaMerge/File/DMFilePackFilter.cpp | 66 ++- .../DeltaMerge/File/DMFilePackFilter.h | 8 + dbms/src/Storages/DeltaMerge/Segment.cpp | 22 +- dbms/src/Storages/DeltaMerge/Segment.h | 10 +- .../VersionChain/BuildBitmapFilter.cpp | 95 ++++ .../VersionChain/BuildBitmapFilter.h | 38 ++ .../DeltaMerge/VersionChain/Common.cpp | 129 +++++ .../Storages/DeltaMerge/VersionChain/Common.h | 105 ++++ .../VersionChain/DMFileHandleIndex.h | 241 +++++++++ .../DeltaMerge/VersionChain/DeletedFilter.cpp | 170 ++++++ .../DeltaMerge/VersionChain/DeletedFilter.h | 26 + .../VersionChain/HandleColumnView.h | 109 ++++ .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 306 +++++++++++ .../DeltaMerge/VersionChain/RowKeyFilter.h | 33 ++ .../DeltaMerge/VersionChain/VersionChain.cpp | 230 ++++++++ .../DeltaMerge/VersionChain/VersionChain.h | 85 +++ .../DeltaMerge/VersionChain/VersionFilter.cpp | 276 ++++++++++ .../DeltaMerge/VersionChain/VersionFilter.h | 29 ++ .../VersionChain/tests/bench_btree_hash.cpp | 148 ++++++ .../VersionChain/tests/bench_search.cpp | 188 +++++++ .../tests/bench_version_chain.cpp | 493 ++++++++++++++++++ .../tests/gtest_handle_pack_index.cpp | 31 ++ .../DeltaMerge/tests/bench_vector_index.cpp | 98 ---- .../tests/bench_vector_index_utils.h | 177 ------- .../tests/gtest_dm_delta_merge_store.cpp | 35 +- .../DeltaMerge/tests/gtest_dm_segment.cpp | 31 +- .../DeltaMerge/tests/gtest_segment_bitmap.cpp | 48 +- .../tests/gtest_segment_read_task.cpp | 9 + 37 files changed, 2940 insertions(+), 329 deletions(-) create mode 100644 dbms/gtest.log create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/Common.h create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_btree_hash.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp delete mode 100644 dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp delete mode 100644 dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index fdc5a70fbc5..8116e56400e 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -408,10 +408,6 @@ if (ENABLE_TESTS) tiflash_aggregate_functions kvstore) - if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") - target_link_libraries(bench_dbms tiflash_contrib::highfive) - endif() - add_check(bench_dbms) endif () diff --git a/dbms/gtest.log b/dbms/gtest.log new file mode 100644 index 00000000000..c24670afad1 --- /dev/null +++ b/dbms/gtest.log @@ -0,0 +1 @@ +zsh: no such file or directory: ./dbms/gtests_dbms diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 2368fa17e96..750a0c79151 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -190,6 +190,7 @@ struct Settings M(SettingBool, dt_flush_after_write, false, "[testing] Flush cache or not after write in DeltaTree Engine.") \ M(SettingBool, dt_log_record_version, false, "[testing] Whether log the version of records when write them to storage") \ \ + M(SettingBool, dt_enable_version_chain, true, "") \ /* These PageStorage V2 settings are deprecated since v6.5 */ \ M(SettingUInt64, dt_open_file_max_idle_seconds, 15, "Deprecated.") \ M(SettingUInt64, dt_page_num_max_expect_legacy_files, 100, "Deprecated.") \ diff --git a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h index eb5154bd231..06e580aefda 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h @@ -45,6 +45,8 @@ class BitmapFilter size_t count() const; inline size_t size() const { return filter.size(); } + std::vector & getFilter() { return filter; } + friend class BitmapFilterView; private: diff --git a/dbms/src/Storages/DeltaMerge/CMakeLists.txt b/dbms/src/Storages/DeltaMerge/CMakeLists.txt index 05f731bbfff..cf434670df8 100644 --- a/dbms/src/Storages/DeltaMerge/CMakeLists.txt +++ b/dbms/src/Storages/DeltaMerge/CMakeLists.txt @@ -32,6 +32,7 @@ add_headers_and_sources(delta_merge ./Remote) add_headers_and_sources(delta_merge ./Remote/DataStore) add_headers_and_sources(delta_merge ./Decode) add_headers_and_sources(delta_merge ./StoragePool) +add_headers_and_sources(delta_merge ./VersionChain) add_library(delta_merge ${delta_merge_headers} ${delta_merge_sources}) target_link_libraries(delta_merge PRIVATE dbms page) diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.h b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.h index fbc7611ad4c..bfbb4e0f193 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.h +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileBig.h @@ -73,6 +73,8 @@ class ColumnFileBig : public ColumnFilePersisted auto getFile() const { return file; } + RowKeyRange getRange() const { return segment_range; } + PageIdU64 getDataPageId() { return file->pageId(); } size_t getRows() const override { return valid_rows; } diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp b/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp index 7c11113d583..a4fa8f7e104 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp @@ -43,12 +43,12 @@ DeltaIndex::Updates ColumnFileFlushTask::prepare(WriteBatches & wbs) { if (!task.block_data) continue; - +/* IColumn::Permutation perm; task.sorted = sortBlockByPk(getExtraHandleColumnDefine(context.is_common_handle), task.block_data, perm); if (task.sorted) delta_index_updates.emplace_back(task.deletes_offset, task.rows_offset, perm); - +*/ task.data_page = ColumnFileTiny::writeColumnFileData(context, task.block_data, 0, task.block_data.rows(), wbs); } diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h index ae1d1f6d78e..826c9cb8d39 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -414,6 +414,18 @@ class DeltaValueSnapshot : private boost::noncopyable ColumnFileSetSnapshotPtr getMemTableSetSnapshot() const { return mem_table_snap; } ColumnFileSetSnapshotPtr getPersistedFileSetSnapshot() const { return persisted_files_snap; } + ColumnFiles getColumnFiles() const + { + auto cfs = persisted_files_snap->getColumnFiles(); + const auto & memory_cfs = mem_table_snap->getColumnFiles(); + cfs.insert(cfs.end(), memory_cfs.begin(), memory_cfs.end()); + return cfs; + } + const auto & getDataProvider() const + { + RUNTIME_CHECK(persisted_files_snap->getDataProvider() == mem_table_snap->getDataProvider()); + return persisted_files_snap->getDataProvider(); + } size_t getColumnFileCount() const { return mem_table_snap->getColumnFileCount() + persisted_files_snap->getColumnFileCount(); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.h b/dbms/src/Storages/DeltaMerge/File/DMFile.h index fb9691277b6..2bca2e39f7f 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.h @@ -212,6 +212,10 @@ class DMFile : private boost::noncopyable UInt32 metaVersion() const { return meta->metaVersion(); } bool isColIndexExist(const ColId & col_id) const; + static FileNameBase getFileNameBase(ColId col_id, const IDataType::SubstreamPath & substream = {}) + { + return IDataType::getFileNameForStream(DB::toString(col_id), substream); + } private: DMFile( @@ -322,6 +326,8 @@ class DMFile : private boost::noncopyable { return subFilePath(localIndexFileName(index_id, kind)); } + static String vectorIndexFileName(IndexID index_id) { return fmt::format("idx_{}.vector", index_id); } + String vectorIndexPath(IndexID index_id) const { return subFilePath(vectorIndexFileName(index_id)); } void addPack(const DMFileMeta::PackStat & pack_stat) const { meta->pack_stats.push_back(pack_stat); } diff --git a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.cpp index b0fd3c26b08..adf3c83bbaa 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.cpp @@ -193,30 +193,44 @@ void DMFilePackFilter::loadIndex( const ReadLimiterPtr & read_limiter, const ScanContextPtr & scan_context) { - const auto & type = dmfile->getColumnStat(col_id).type; + auto [type, minmax_index] + = loadIndex(*dmfile, file_provider, index_cache, set_cache_if_miss, col_id, read_limiter, scan_context); + indexes.emplace(col_id, RSIndex(type, minmax_index)); +} + +std::pair DMFilePackFilter::loadIndex( + const DMFile & dmfile, + const FileProviderPtr & file_provider, + const MinMaxIndexCachePtr & index_cache, + bool set_cache_if_miss, + ColId col_id, + const ReadLimiterPtr & read_limiter, + const ScanContextPtr & scan_context) +{ + const auto & type = dmfile.getColumnStat(col_id).type; const auto file_name_base = DMFile::getFileNameBase(col_id); auto load = [&]() { - auto index_file_size = dmfile->colIndexSize(col_id); + auto index_file_size = dmfile.colIndexSize(col_id); if (index_file_size == 0) return std::make_shared(*type); auto index_guard = S3::S3RandomAccessFile::setReadFileInfo({ - .size = dmfile->getReadFileSize(col_id, colIndexFileName(file_name_base)), + .size = dmfile.getReadFileSize(col_id, colIndexFileName(file_name_base)), .scan_context = scan_context, }); - if (!dmfile->getConfiguration()) // v1 + if (!dmfile.getConfiguration()) // v1 { auto index_buf = ReadBufferFromRandomAccessFileBuilder::build( file_provider, - dmfile->colIndexPath(file_name_base), - dmfile->encryptionIndexPath(file_name_base), + dmfile.colIndexPath(file_name_base), + dmfile.encryptionIndexPath(file_name_base), std::min(static_cast(DBMS_DEFAULT_BUFFER_SIZE), index_file_size), read_limiter); return MinMaxIndex::read(*type, index_buf, index_file_size); } - else if (dmfile->useMetaV2()) // v3 + else if (dmfile.useMetaV2()) // v3 { - const auto * dmfile_meta = typeid_cast(dmfile->meta.get()); + const auto * dmfile_meta = typeid_cast(dmfile.meta.get()); assert(dmfile_meta != nullptr); auto info = dmfile_meta->merged_sub_file_infos.find(colIndexFileName(file_name_base)); if (info == dmfile_meta->merged_sub_file_infos.end()) @@ -224,10 +238,10 @@ void DMFilePackFilter::loadIndex( throw Exception( ErrorCodes::LOGICAL_ERROR, "Unknown index file {}", - dmfile->colIndexPath(file_name_base)); + dmfile.colIndexPath(file_name_base)); } - auto file_path = dmfile->meta->mergedPath(info->second.number); + auto file_path = dmfile.meta->mergedPath(info->second.number); auto encryp_path = dmfile_meta->encryptionMergedPath(info->second.number); auto offset = info->second.offset; auto data_size = info->second.size; @@ -236,7 +250,7 @@ void DMFilePackFilter::loadIndex( file_provider, file_path, encryp_path, - dmfile->getConfiguration()->getChecksumFrameLength(), + dmfile.getConfiguration()->getChecksumFrameLength(), read_limiter); buffer.seek(offset); @@ -247,13 +261,13 @@ void DMFilePackFilter::loadIndex( auto buf = ChecksumReadBufferBuilder::build( std::move(raw_data), - dmfile->colIndexPath(file_name_base), // just for debug - dmfile->getConfiguration()->getChecksumFrameLength(), - dmfile->getConfiguration()->getChecksumAlgorithm(), - dmfile->getConfiguration()->getChecksumFrameLength()); + dmfile.colIndexPath(file_name_base), // just for debug + dmfile.getConfiguration()->getChecksumFrameLength(), + dmfile.getConfiguration()->getChecksumAlgorithm(), + dmfile.getConfiguration()->getChecksumFrameLength()); - auto header_size = dmfile->getConfiguration()->getChecksumHeaderLength(); - auto frame_total_size = dmfile->getConfiguration()->getChecksumFrameLength() + header_size; + auto header_size = dmfile.getConfiguration()->getChecksumHeaderLength(); + auto frame_total_size = dmfile.getConfiguration()->getChecksumFrameLength() + header_size; auto frame_count = index_file_size / frame_total_size + (index_file_size % frame_total_size != 0); return MinMaxIndex::read(*type, *buf, index_file_size - header_size * frame_count); @@ -262,14 +276,14 @@ void DMFilePackFilter::loadIndex( { // v2 auto index_buf = ChecksumReadBufferBuilder::build( file_provider, - dmfile->colIndexPath(file_name_base), - dmfile->encryptionIndexPath(file_name_base), + dmfile.colIndexPath(file_name_base), + dmfile.encryptionIndexPath(file_name_base), index_file_size, read_limiter, - dmfile->getConfiguration()->getChecksumAlgorithm(), - dmfile->getConfiguration()->getChecksumFrameLength()); - auto header_size = dmfile->getConfiguration()->getChecksumHeaderLength(); - auto frame_total_size = dmfile->getConfiguration()->getChecksumFrameLength() + header_size; + dmfile.getConfiguration()->getChecksumAlgorithm(), + dmfile.getConfiguration()->getChecksumFrameLength()); + auto header_size = dmfile.getConfiguration()->getChecksumHeaderLength(); + auto frame_total_size = dmfile.getConfiguration()->getChecksumFrameLength() + header_size; 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); } @@ -277,17 +291,17 @@ void DMFilePackFilter::loadIndex( MinMaxIndexPtr minmax_index; if (index_cache && set_cache_if_miss) { - minmax_index = index_cache->getOrSet(dmfile->colIndexCacheKey(file_name_base), load); + minmax_index = index_cache->getOrSet(dmfile.colIndexCacheKey(file_name_base), load); } else { // try load from the cache first if (index_cache) - minmax_index = index_cache->get(dmfile->colIndexCacheKey(file_name_base)); + minmax_index = index_cache->get(dmfile.colIndexCacheKey(file_name_base)); if (minmax_index == nullptr) minmax_index = load(); } - indexes.emplace(col_id, RSIndex(type, minmax_index)); + return {type, minmax_index}; } void DMFilePackFilter::tryLoadIndex(RSCheckParam & param, ColId col_id) diff --git a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h index 4e6a6c4ab35..c94d014fd16 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h @@ -126,6 +126,14 @@ class DMFilePackFilter const DMFiles & dmfiles, const DMFilePackFilterResults & pack_filter_results, UInt64 start_ts); + static std::pair loadIndex( + const DMFile & dmfile, + const FileProviderPtr & file_provider, + const MinMaxIndexCachePtr & index_cache, + bool set_cache_if_miss, + ColId col_id, + const ReadLimiterPtr & read_limiter, + const ScanContextPtr & scan_context); private: DMFilePackFilter( diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 93fa3f8b26b..e568fbe0d7c 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -3078,10 +3078,18 @@ BitmapFilterPtr Segment::buildBitmapFilter( const RowKeyRanges & read_ranges, const DMFilePackFilterResults & pack_filter_results, UInt64 start_ts, - size_t expected_block_size) + size_t expected_block_size, + bool use_version_chain) { RUNTIME_CHECK_MSG(!dm_context.read_delta_only, "Read delta only is unsupported"); sanitizeCheckReadRanges(__FUNCTION__, read_ranges, rowkey_range, log); + RUNTIME_CHECK(!is_common_handle); + + if (use_version_chain) + { + return buildBitmapFilter(dm_context, *segment_snap, read_ranges, filter, start_ts, version_chain); + } + if (dm_context.read_stable_only || (segment_snap->delta->getRows() == 0 && segment_snap->delta->getDeletes() == 0)) { return buildBitmapFilterStableOnly( @@ -3452,18 +3460,23 @@ BlockInputStreamPtr Segment::getLateMaterializationStream( dm_context.tracing_id); } -RowKeyRanges Segment::shrinkRowKeyRanges(const RowKeyRanges & read_ranges) const +RowKeyRanges Segment::shrinkRowKeyRanges(const RowKeyRange & target_range, const RowKeyRanges & read_ranges) { RowKeyRanges real_ranges; for (const auto & read_range : read_ranges) { - auto real_range = rowkey_range.shrink(read_range); + auto real_range = target_range.shrink(read_range); if (!real_range.none()) real_ranges.emplace_back(std::move(real_range)); } return real_ranges; } +RowKeyRanges Segment::shrinkRowKeyRanges(const RowKeyRanges & read_ranges) const +{ + return shrinkRowKeyRanges(rowkey_range, read_ranges); +} + static bool hasCacheableColumn(const ColumnDefines & columns) { return std::find_if(columns.begin(), columns.end(), DMFileReader::isCacheableColumn) != columns.end(); @@ -3488,7 +3501,8 @@ BlockInputStreamPtr Segment::getBitmapFilterInputStream( read_ranges, pack_filter_results, start_ts, - build_bitmap_filter_block_rows); + build_bitmap_filter_block_rows, + dm_context.global_context.getSettingsRef().dt_enable_version_chain); // If we don't need to read the cacheable columns, release column cache as soon as possible. if (!hasCacheableColumn(columns_to_read)) diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index bb331e8cdef..1e666f89d52 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -27,9 +27,10 @@ #include #include #include +#include +#include #include #include - namespace DB { struct GeneralCancelHandle; @@ -663,6 +664,8 @@ class Segment } } + static RowKeyRanges shrinkRowKeyRanges(const RowKeyRange & target_range, const RowKeyRanges & read_ranges); + #ifndef DBMS_PUBLIC_GTEST private: #else @@ -736,7 +739,8 @@ class Segment const RowKeyRanges & read_ranges, const DMFilePackFilterResults & pack_filter_results, UInt64 start_ts, - size_t expected_block_size); + size_t expected_block_size, + bool use_version_chain); BitmapFilterPtr buildBitmapFilterNormal( const DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, @@ -838,6 +842,8 @@ class Segment const LoggerPtr parent_log; // Used when constructing new segments in split const LoggerPtr log; + + VersionChain version_chain; // TODO: support common handle }; void readSegmentMetaInfo(ReadBuffer & buf, Segment::SegmentMetaInfo & segment_info); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp new file mode 100644 index 00000000000..1ac3a64d698 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -0,0 +1,95 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include +#include +#include +#include + +namespace DB::DM +{ +RSResults getDMFileRSFilterResults( + const DMContext & dm_context, + const DMFilePtr & dmfile, + const RSOperatorPtr & rs_operator) +{ + if (!rs_operator) + return RSResults(dmfile->getPacks(), RSResult::Some); + + auto pack_filter = DMFilePackFilter::loadFrom( + dmfile, + dm_context.global_context.getMinMaxIndexCache(), + true, + {}, // read_ranges + rs_operator, + {}, + dm_context.global_context.getFileProvider(), + dm_context.getReadLimiter(), + dm_context.scan_context, + dm_context.tracing_id, + ReadTag::MVCC); + return pack_filter.getPackResConst(); +} + +RSResults getStableRSFilterResults( + const DMContext & dm_context, + const StableValueSpace::Snapshot & stable, + const RSOperatorPtr & rs_operator) +{ + const auto & dmfiles = stable.getDMFiles(); + RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); + return getDMFileRSFilterResults(dm_context, dmfiles[0], rs_operator); +} + +template +BitmapFilterPtr buildBitmapFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const RSOperatorPtr & rs_operator, + const UInt64 read_ts, + VersionChain & version_chain) +{ + const auto base_ver_snap = version_chain.replaySnapshot(dm_context, snapshot); + const auto & delta = *(snapshot.delta); + const auto & stable = *(snapshot.stable); + const UInt32 delta_rows = delta.getRows(); + const UInt32 stable_rows = stable.getDMFilesRows(); + const UInt32 total_rows = delta_rows + stable_rows; + auto bitmap_filter = std::make_shared(total_rows, true); + auto & filter = bitmap_filter->getFilter(); + + // TODO: make these functions return filter out rows. + // TODO: send the pack res to buildVersionFilter and buildDeletedFilter to skip some packs. + auto stable_pack_res = getStableRSFilterResults(dm_context, stable, rs_operator); + buildRowKeyFilter(dm_context, snapshot, read_ranges, stable_pack_res, filter); + buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); + buildDeletedFilter(dm_context, snapshot, filter); + + bitmap_filter->runOptimize(); + return bitmap_filter; +} + +template BitmapFilterPtr buildBitmapFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const RSOperatorPtr & rs_operator, + const UInt64 read_ts, + VersionChain & version_chain); +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h new file mode 100644 index 00000000000..c652b7ce2c1 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h @@ -0,0 +1,38 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +#include +#include + +namespace DB::DM +{ +struct DMContext; +struct SegmentSnapshot; +struct RowKeyRange; +using RowKeyRanges = std::vector; + +class BitmapFilter; +using BitmapFilterPtr = std::shared_ptr; + +template +BitmapFilterPtr buildBitmapFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const RSOperatorPtr & rs_operator, + const UInt64 read_ts, + VersionChain & version_chain); +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp new file mode 100644 index 00000000000..86c822a1ef4 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp @@ -0,0 +1,129 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include + +namespace DB::DM +{ +RSResults getRSResultsByRanges( + const Context & global_context, + const ScanContextPtr & scan_context, + const String & tracing_id, + const DMFilePtr & dmfile, + const RowKeyRanges & ranges) +{ + if (ranges.empty()) + return RSResults(dmfile->getPacks(), RSResult::All); + + auto pack_filter = DMFilePackFilter::loadFrom( + dmfile, + global_context.getMinMaxIndexCache(), + true, + ranges, + EMPTY_RS_OPERATOR, + {}, + global_context.getFileProvider(), + global_context.getReadLimiter(), + scan_context, + tracing_id, + ReadTag::MVCC); + + return pack_filter.getHandleRes(); +} + +namespace +{ +template +T getMaxValue(const MinMaxIndex & minmax_index, size_t i) +{ + if constexpr (std::is_same_v) + return minmax_index.getIntMinMax(i).second; + else if constexpr (std::is_same_v) + return minmax_index.getStringMinMax(i).second; + else if constexpr (std::is_same_v) + return minmax_index.getUInt64MinMax(i).second; + else + static_assert(false, "Not support type"); +} + +std::pair clipRSResults(const RSResults & rs_results) +{ + const auto start = std::find_if(rs_results.begin(), rs_results.end(), [](RSResult r) { return r.isUse(); }); + if (start == rs_results.end()) + return {}; + const auto end = std::find_if(start, rs_results.end(), [](RSResult r) { return !r.isUse(); }); + const auto start_pack_id = start - rs_results.begin(); + return std::make_pair(RSResults(start, end), start_pack_id); +} +} // namespace + +std::pair getClippedRSResultsByRanges( + const Context & global_context, + const ScanContextPtr & scan_context, + const String & tracing_id, + const DMFilePtr & dmfile, + const std::optional & segment_range) +{ + if (!segment_range) + return std::make_pair(RSResults(dmfile->getPacks(), RSResult::All), 0); + + const auto handle_res = getRSResultsByRanges( + global_context, + scan_context, + tracing_id, + dmfile, + {*segment_range}); + + return clipRSResults(handle_res); +} + +template +std::vector loadPackMaxValue(const Context & global_context, const DMFile & dmfile, const ColId col_id) +{ + if (dmfile.getPacks() == 0) + return {}; + + auto [type, minmax_index] = DMFilePackFilter::loadIndex( + dmfile, + global_context.getFileProvider(), + global_context.getMinMaxIndexCache(), + /* set cache*/ true, + col_id, + global_context.getReadLimiter(), + /*scan context*/ nullptr); + + auto pack_count = dmfile.getPacks(); + std::vector pack_max_values; + pack_max_values.reserve(pack_count); + for (size_t i = 0; i < pack_count; ++i) + { + pack_max_values.push_back(getMaxValue(*minmax_index, i)); + } + return pack_max_values; +} + +template std::vector loadPackMaxValue( + const Context & global_context, + const DMFile & dmfile, + const ColId col_id); +template std::vector loadPackMaxValue( + const Context & global_context, + const DMFile & dmfile, + const ColId col_id); +//template std::vector loadPackMaxValue(const Context & global_context, const DMFile & dmfile, const ColId col_id); + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h new file mode 100644 index 00000000000..6026c848626 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h @@ -0,0 +1,105 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +#include +#include +#include +#include +#include + +#include + +namespace DB::DM +{ +struct DMContext; + +using RowID = UInt32; +static constexpr RowID NotExistRowID = std::numeric_limits::max(); +static constexpr RowID UnknownRowID = NotExistRowID - 1; + +template +concept Int64OrString = std::same_as || std::same_as; + +template +concept Int64OrStringView = std::same_as || std::same_as; + +template +ColumnDefine getHandleColumnDefine() +{ + if constexpr (std::is_same_v) + return getExtraIntHandleColumnDefine(); + else if constexpr (std::is_same_v) + return getExtraStringHandleColumnDefine(); + else + static_assert(false, "Not support type"); +} + +// For ColumnFileReader +template +ColumnDefinesPtr getHandleColumnDefinesPtr() +{ + static auto cds_ptr = std::make_shared(1, getHandleColumnDefine()); + return cds_ptr; +} + +inline ColumnDefinesPtr getVersionColumnDefinesPtr() +{ + static auto cds_ptr = std::make_shared(1, getVersionColumnDefine()); + return cds_ptr; +} + +inline ColumnDefinesPtr getTagColumnDefinesPtr() +{ + static auto cds_ptr = std::make_shared(1, getTagColumnDefine()); + return cds_ptr; +} + +template +bool inRowKeyRange(const RowKeyRange & range, HandleView handle) +{ + if constexpr (std::is_same_v) + return range.start.int_value <= handle && handle < range.end.int_value; + else + static_assert(false, "TODO: support common handle"); +} + +template +std::pair convertRowKeyRange(const RowKeyRange & range) +{ + if constexpr (std::is_same_v) + return {range.start.int_value, range.end.int_value}; + else + static_assert(false, "TODO: support common handle"); +} + +RSResults getRSResultsByRanges( + const Context & global_context, + const ScanContextPtr & scan_context, + const String & tracing_id, + const DMFilePtr & dmfile, + const RowKeyRanges & ranges); + +std::pair getClippedRSResultsByRanges( + const Context & global_context, + const ScanContextPtr & scan_context, + const String & tracing_id, + const DMFilePtr & dmfile, + const std::optional & segment_range); + +template +std::vector loadPackMaxValue(const Context & global_context, const DMFile & dmfile, const ColId col_id); + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h new file mode 100644 index 00000000000..aac51b4073e --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h @@ -0,0 +1,241 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB::DM +{ + +template +class DMFileHandleIndex +{ +public: + DMFileHandleIndex( + const Context & global_context_, + const DMFilePtr & dmfile_, + const RowID start_row_id_, + std::optional rowkey_range_) + : global_context(global_context_) + , dmfile(dmfile_) + , start_row_id(start_row_id_) + , rowkey_range(std::move(rowkey_range_)) + , clipped_pack_range(getPackRange()) + , clipped_pack_index(loadPackIndex()) + , clipped_pack_offsets(loadPackOffsets()) + , clipped_handle_packs(clipped_pack_range.count()) + , clipped_need_read_packs(std::vector(clipped_pack_range.count(), 1)) // read all packs by default + { + RUNTIME_CHECK( + clipped_pack_index.size() == clipped_pack_range.count(), + clipped_pack_index.size(), + clipped_pack_range.count()); + RUNTIME_CHECK( + clipped_pack_offsets.size() == clipped_pack_range.count(), + clipped_pack_offsets.size(), + clipped_pack_range.count()); + } + + template + std::optional getBaseVersion(HandleView h) + { + auto clipped_pack_id = getClippedPackId(h); + if (!clipped_pack_id) + return {}; + auto row_id = getBaseVersion(h, *clipped_pack_id); + if (!row_id) + return {}; + return start_row_id + *row_id; + } + + void calculateReadPacks(const std::span handles) + { + std::vector calc_read_packs(clipped_pack_range.count(), 0); + UInt32 calc_read_count = 0; + for (const Handle & h : handles) + { + auto clipped_pack_id = getClippedPackId(h); + if (!clipped_pack_id || calc_read_packs[*clipped_pack_id]) + continue; + + calc_read_packs[*clipped_pack_id] = 1; + ++calc_read_count; + + // Read too many packs, read all by default + if (calc_read_count * 4 >= clipped_pack_range.count()) + return; + } + clipped_need_read_packs->swap(calc_read_packs); + } + + void cleanHandleColumn() + { + // Reset handle column data to save memory. + std::fill(clipped_handle_packs.begin(), clipped_handle_packs.end(), nullptr); + clipped_need_read_packs.emplace(clipped_pack_range.count(), 1); + } + +private: + template + std::optional getClippedPackId(HandleView h) + { + if (unlikely(rowkey_range && !inRowKeyRange(*rowkey_range, h))) + return {}; + + auto itr = std::lower_bound(clipped_pack_index.begin(), clipped_pack_index.end(), h); + if (itr == clipped_pack_index.end()) + return {}; + return itr - clipped_pack_index.begin(); + } + + template + std::optional getBaseVersion(HandleView h, UInt32 clipped_pack_id) + { + loadHandleIfNotLoaded(); + const auto & handle_col = clipped_handle_packs[clipped_pack_id]; + const auto * handles = toColumnVectorDataPtr(handle_col); + RUNTIME_CHECK_MSG(handles != nullptr, "TODO: support common handle"); + auto itr = std::lower_bound(handles->begin(), handles->end(), h); + if (itr != handles->end() && *itr == h) + { + return itr - handles->begin() + clipped_pack_offsets[clipped_pack_id]; + } + return {}; + } + + std::vector loadPackIndex() + { + auto max_values = loadPackMaxValue(global_context, *dmfile, EXTRA_HANDLE_COLUMN_ID); + return std::vector( + max_values.begin() + clipped_pack_range.start_pack_id, + max_values.begin() + clipped_pack_range.end_pack_id); + } + + std::vector loadPackOffsets() + { + const auto & pack_stats = dmfile->getPackStats(); + std::vector pack_offsets(clipped_pack_range.count(), 0); + for (UInt32 clipped_pack_id = 1; clipped_pack_id < pack_offsets.size(); ++clipped_pack_id) + { + UInt32 real_pack_id = clipped_pack_id + clipped_pack_range.start_pack_id; + pack_offsets[clipped_pack_id] = pack_offsets[clipped_pack_id - 1] + pack_stats[real_pack_id - 1].rows; + } + return pack_offsets; + } + + static bool isCommonHandle() { return std::is_same_v; } + + void loadHandleIfNotLoaded() + { + if (likely(!clipped_need_read_packs)) + return; + + auto read_pack_ids = std::make_shared(); + const auto & packs = *clipped_need_read_packs; + for (UInt32 i = 0; i < packs.size(); ++i) + { + if (packs[i]) + read_pack_ids->insert(i + clipped_pack_range.start_pack_id); + } + + auto scan_context = std::make_shared(); + auto pack_filter = DMFilePackFilter::loadFrom( + dmfile, + global_context.getMinMaxIndexCache(), + true, //set_cache_if_miss + {}, // rowkey_ranges, empty means all + nullptr, // RSOperatorPtr + read_pack_ids, + global_context.getFileProvider(), + global_context.getReadLimiter(), + scan_context, + __FILE__, + ReadTag::MVCC); + + DMFileReader reader( + dmfile, + {getHandleColumnDefine()}, + isCommonHandle(), + /*enable_handle_clean_read*/ false, + /*enable_del_clean_read*/ false, + /*is_fast_scan*/ false, + /*max_data_version*/ std::numeric_limits::max(), + std::move(pack_filter), + global_context.getMarkCache(), + /*enable_column_cache*/ false, + /*column_cache*/ nullptr, + global_context.getSettingsRef().max_read_buffer_size, + global_context.getFileProvider(), + global_context.getReadLimiter(), + DEFAULT_MERGE_BLOCK_SIZE, + /*read_one_pack_every_time*/ true, + "DMFileHandleIndex", + /*max_sharing_column_bytes_for_all*/ false, + scan_context, + ReadTag::MVCC); + + + for (UInt32 pack_id : *read_pack_ids) + { + auto block = reader.read(); + clipped_handle_packs[pack_id - clipped_pack_range.start_pack_id] = block.begin()->column; + } + clipped_need_read_packs.reset(); + } + + struct PackRange + { + // [start_pack_id, end_pack_id) + UInt32 start_pack_id; + UInt32 end_pack_id; + + UInt32 count() const { return end_pack_id - start_pack_id; } + }; + + PackRange getPackRange() + { + if (!rowkey_range) + return PackRange{.start_pack_id = 0, .end_pack_id = static_cast(dmfile->getPacks())}; + + const auto [handle_res, start_pack_id] = getClippedRSResultsByRanges( + global_context, + std::make_shared(), + "DMFileHandleIndex", + dmfile, + rowkey_range); + return PackRange{ + .start_pack_id = start_pack_id, + .end_pack_id = start_pack_id + static_cast(handle_res.size())}; + } + + const Context & global_context; + const DMFilePtr dmfile; + const RowID start_row_id; + const std::optional rowkey_range; // Range of ColumnFileBig or nullopt for Stable DMFile + + // Clipped by rowkey_range + const PackRange clipped_pack_range; + const std::vector clipped_pack_index; // max value of each pack + const std::vector clipped_pack_offsets; // offset of each pack + std::vector clipped_handle_packs; // handle column of each pack + std::optional> clipped_need_read_packs; +}; + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp new file mode 100644 index 00000000000..f000cbc7680 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp @@ -0,0 +1,170 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include +#include +#include +#include +#include + +namespace DB::DM +{ +UInt32 buildDeletedFilterBlock( + const DMContext & dm_context, + const IColumnFileDataProviderPtr & data_provider, + const ColumnFile & cf, + const UInt32 start_row_id, + std::vector & filter) +{ + assert(cf.isInMemoryFile() || cf.isTinyFile()); + auto cf_reader = cf.getReader(dm_context, data_provider, getTagColumnDefinesPtr(), ReadTag::MVCC); + auto block = cf_reader->readNextBlock(); + RUNTIME_CHECK_MSG(!cf_reader->readNextBlock(), "{}: read all rows in one block is required!", cf.toString()); + const auto & deleteds = *toColumnVectorDataPtr(block.begin()->column); // Must success. + for (UInt32 i = 0; i < deleteds.size(); ++i) + { + if (deleteds[i]) + filter[start_row_id + i] = 0; + } + return deleteds.size(); +} + +UInt32 buildDeletedFilterDMFile( + const DMContext & dm_context, + const DMFilePtr & dmfile, + const std::optional & segment_range, + const ssize_t start_row_id, + std::vector & filter) +{ + auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges( + dm_context.global_context, + dm_context.scan_context, + dm_context.tracing_id, + dmfile, segment_range); + if (valid_handle_res.empty()) + return 0; + + auto read_packs = std::make_shared(); + UInt32 need_read_rows = 0; + std::unordered_map read_pack_to_start_row_ids; + + const auto & pack_stats = dmfile->getPackStats(); + const auto & pack_properties = dmfile->getPackProperties(); + UInt32 rows = 0; + for (UInt32 i = 0; i < valid_handle_res.size(); ++i) + { + const UInt32 pack_id = valid_start_pack_id + i; + const UInt32 pack_start_row_id = start_row_id + rows; + if (pack_properties.property(pack_id).deleted_rows() > 0) + { + read_packs->insert(pack_id); + read_pack_to_start_row_ids.emplace(pack_id, pack_start_row_id); + need_read_rows += pack_stats[pack_id].rows; + } + rows += pack_stats[pack_id].rows; + } + + if (need_read_rows == 0) + return rows; + + DMFileBlockInputStreamBuilder builder(dm_context.global_context); + builder.onlyReadOnePackEveryTime().setReadPacks(read_packs).setReadTag(ReadTag::MVCC); + auto stream = builder.build(dmfile, {getTagColumnDefine()}, {}, dm_context.scan_context); + UInt32 read_rows = 0; + for (auto pack_id : *read_packs) + { + auto block = stream->read(); + RUNTIME_CHECK(block.rows() == pack_stats[pack_id].rows, block.rows(), pack_stats[pack_id].rows); + const auto & deleteds = *toColumnVectorDataPtr(block.begin()->column); // Must success + + const auto itr = read_pack_to_start_row_ids.find(pack_id); + RUNTIME_CHECK(itr != read_pack_to_start_row_ids.end(), read_pack_to_start_row_ids, pack_id); + const UInt32 pack_start_row_id = itr->second; + + for (UInt32 i = 0; i < deleteds.size(); ++i) + { + if (deleteds[i]) + filter[pack_start_row_id + i] = 0; + } + read_rows += pack_stats[pack_id].rows; + } + RUNTIME_CHECK(read_rows == need_read_rows, read_rows, need_read_rows); + return rows; +} + +UInt32 buildDeletedFilterColumnFileBig( + const DMContext & dm_context, + const ColumnFileBig & cf_big, + const ssize_t start_row_id, + std::vector & filter) +{ + return buildDeletedFilterDMFile(dm_context, cf_big.getFile(), cf_big.getRange(), start_row_id, filter); +} + +UInt32 buildDeletedFilterStable( + const DMContext & dm_context, + const StableValueSpace::Snapshot & stable, + std::vector & filter) +{ + const auto & dmfiles = stable.getDMFiles(); + RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); + return buildDeletedFilterDMFile(dm_context, dmfiles[0], std::nullopt, 0, filter); +} + +void buildDeletedFilter(const DMContext & dm_context, const SegmentSnapshot & snapshot, std::vector & filter) +{ + const auto & delta = *(snapshot.delta); + const auto & stable = *(snapshot.stable); + const UInt32 delta_rows = delta.getRows(); + const UInt32 stable_rows = stable.getDMFilesRows(); + const UInt32 total_rows = delta_rows + stable_rows; + RUNTIME_CHECK(filter.size() == total_rows, filter.size(), total_rows); + + auto read_rows = buildDeletedFilterStable(dm_context, stable, filter); + RUNTIME_CHECK(stable_rows == read_rows, stable_rows, read_rows); + + const auto cfs = delta.getColumnFiles(); + const auto & data_provider = delta.getDataProvider(); + for (auto itr = cfs.begin(); itr != cfs.end(); ++itr) + { + const auto & cf = *itr; + if (cf->isDeleteRange()) + continue; + + const UInt32 cf_rows = cf->getRows(); + const UInt32 start_row_id = read_rows; + read_rows += cf_rows; + + // TODO: add deleted_rows in tiny file + if (cf->isInMemoryFile() || cf->isTinyFile()) + { + const auto n = buildDeletedFilterBlock(dm_context, data_provider, *cf, start_row_id, filter); + RUNTIME_CHECK(cf_rows == n, cf_rows, n); + continue; + } + + if (const auto * cf_big = cf->tryToBigFile(); cf_big) + { + const auto n = buildDeletedFilterColumnFileBig(dm_context, *cf_big, start_row_id, filter); + RUNTIME_CHECK(cf_rows == n, cf_rows, n); + continue; + } + RUNTIME_CHECK_MSG(false, "{}: unknow ColumnFile type", cf->toString()); + } + RUNTIME_CHECK(read_rows == total_rows, read_rows, total_rows); +} +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h new file mode 100644 index 00000000000..84f38d980f1 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h @@ -0,0 +1,26 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +#include + +#include + +namespace DB::DM +{ +struct DMContext; +struct SegmentSnapshot; +void buildDeletedFilter(const DMContext & dm_context, const SegmentSnapshot & snapshot, std::vector & filter); +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h b/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h new file mode 100644 index 00000000000..ccf5ec64e31 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h @@ -0,0 +1,109 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +namespace DB::DM +{ +template +class HandleColumnView +{}; + +template <> +class HandleColumnView +{ +public: + HandleColumnView(const IColumn & col) + : data(toColumnVectorData(col)) + {} + + auto begin() const { return data.begin(); } + + auto end() const { return data.end(); } + +private: + const PaddedPODArray & data; +}; + +template <> +class HandleColumnView +{ +public: + HandleColumnView(const IColumn & col) + : offsets(typeid_cast(col).getOffsets()) + , chars(typeid_cast(col).getChars()) + {} + + class Iterator + { + public: + Iterator(const IColumn::Offsets & offsets, const ColumnString::Chars_t & chars, size_t pos) + : offsets(offsets) + , chars(chars) + , pos(pos) + {} + + std::string_view operator*() const + { + assert(offsets[-1] == 0); + const auto off = offsets[pos]; + const auto size = offsets[pos] - offsets[pos - 1] - 1; + return std::string_view(static_cast(&chars[off]), size); + } + + Iterator & operator+(size_t n) + { + pos += n; + return *this; + } + + Iterator & operator-(size_t n) + { + pos -= n; + return *this; + } + + Iterator & operator++() + { + ++pos; + return *this; + } + + Iterator & operator--() + { + --pos; + return *this; + } + + Iterator & operator++(int) = delete; + Iterator & operator--(int) = delete; + + bool operator!=(const Iterator & other) const { return pos != other.pos; } + + private: + const IColumn::Offsets & offsets; + const ColumnString::Chars_t & chars; + size_t pos = 0; + }; + + auto begin() const { return Iterator(offsets, chars, 0); } + + auto end() const { return Iterator(offsets, chars, offsets.size()); } + +private: + const IColumn::Offsets & offsets; + const ColumnString::Chars_t & chars; +}; + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp new file mode 100644 index 00000000000..f4c1a29e206 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -0,0 +1,306 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include +#include +#include +#include + +namespace DB::DM +{ +namespace +{ + +// TODO: shrinking read_range by segment_range +template +UInt32 buildRowKeyFilterVector( + const PaddedPODArray & handles, + const RowKeyRanges & delete_ranges, + const RowKeyRanges & read_ranges, + const UInt32 start_row_id, + std::vector & filter) +{ + for (UInt32 i = 0; i < handles.size(); ++i) + { + auto in_range = [h = handles[i]](const RowKeyRange & range) { + return inRowKeyRange(range, h); + }; + if (std::any_of(delete_ranges.begin(), delete_ranges.end(), in_range) + || std::none_of(read_ranges.begin(), read_ranges.end(), in_range)) + { + filter[start_row_id + i] = 0; + } + } + return handles.size(); +} + +template +UInt32 buildRowKeyFilterBlock( + const DMContext & dm_context, + const IColumnFileDataProviderPtr & data_provider, + const ColumnFile & cf, + const RowKeyRanges & delete_ranges, + const RowKeyRanges & read_ranges, + const UInt32 start_row_id, + std::vector & filter) +{ + assert(cf.isInMemoryFile() || cf.isTinyFile()); + + if (cf.getRows() == 0) + return 0; + + auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); + auto block = cf_reader->readNextBlock(); + RUNTIME_CHECK_MSG(!cf_reader->readNextBlock(), "{}: MUST read all rows in one block!", cf.toString()); + const auto * handles_ptr = toColumnVectorDataPtr(block.begin()->column); + RUNTIME_CHECK_MSG(handles_ptr != nullptr, "TODO: support common handle"); + const auto & handles = *handles_ptr; + return buildRowKeyFilterVector(handles, delete_ranges, read_ranges, start_row_id, filter); +} + +template +UInt32 buildRowKeyFilterDMFile( + const DMContext & dm_context, + const DMFilePtr & dmfile, + const std::optional & segment_range, + const RowKeyRanges & delete_ranges, + const RowKeyRanges & read_ranges, + const RSResults * stable_pack_res, + const UInt32 start_row_id, + std::vector & filter) +{ + auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges( + dm_context.global_context, + dm_context.scan_context, + dm_context.tracing_id, + dmfile, segment_range); + if (valid_handle_res.empty()) + return 0; + + if (stable_pack_res) + { + const auto & s_pack_res = *stable_pack_res; + RUNTIME_CHECK( + stable_pack_res->size() == valid_handle_res.size(), + stable_pack_res->size(), + valid_handle_res.size()); + for (UInt32 i = 0; i < valid_handle_res.size(); ++i) + if (!s_pack_res[i].isUse()) + valid_handle_res[i] = RSResult::None; + } + + const auto read_ranges_handle_res = getRSResultsByRanges( + dm_context.global_context, + dm_context.scan_context, + dm_context.tracing_id, + dmfile, + read_ranges); + for (UInt32 i = 0; i < valid_handle_res.size(); ++i) + valid_handle_res[i] = valid_handle_res[i] && read_ranges_handle_res[valid_start_pack_id + i]; + + if (!delete_ranges.empty()) + { + const auto delete_ranges_handle_res = getRSResultsByRanges( + dm_context.global_context, + dm_context.scan_context, + dm_context.tracing_id, + dmfile, + delete_ranges); + for (UInt32 i = 0; i < valid_handle_res.size(); ++i) + valid_handle_res[i] = valid_handle_res[i] && !delete_ranges_handle_res[valid_start_pack_id + i]; + } + + auto read_packs = std::make_shared(); + UInt32 need_read_rows = 0; + std::unordered_map read_pack_to_start_row_ids; + + const auto & pack_stats = dmfile->getPackStats(); + UInt32 rows = 0; + for (UInt32 i = 0; i < valid_handle_res.size(); ++i) + { + const auto pack_id = valid_start_pack_id + i; + if (!valid_handle_res[i].isUse()) + { + std::fill_n(filter.begin() + start_row_id + rows, pack_stats[pack_id].rows, false); + } + else if (!valid_handle_res[i].allMatch()) + { + read_packs->insert(pack_id); + read_pack_to_start_row_ids.emplace(pack_id, start_row_id + rows); + need_read_rows += pack_stats[pack_id].rows; + } + rows += pack_stats[pack_id].rows; + } + + if (need_read_rows == 0) + return rows; + + DMFileBlockInputStreamBuilder builder(dm_context.global_context); + builder.onlyReadOnePackEveryTime().setReadPacks(read_packs).setReadTag(ReadTag::MVCC); + auto stream = builder.build(dmfile, {getHandleColumnDefine()}, {}, dm_context.scan_context); + UInt32 read_rows = 0; + for (auto pack_id : *read_packs) + { + auto block = stream->read(); + const auto * handles_ptr = toColumnVectorDataPtr(block.begin()->column); + RUNTIME_CHECK_MSG(handles_ptr != nullptr, "TODO: support common handle"); + const auto & handles = *handles_ptr; + + const auto itr = read_pack_to_start_row_ids.find(pack_id); + RUNTIME_CHECK(itr != read_pack_to_start_row_ids.end(), read_pack_to_start_row_ids, pack_id); + read_rows += buildRowKeyFilterVector( + handles, + delete_ranges, + read_ranges, + itr->second, // start_row_id + filter); + } + RUNTIME_CHECK(read_rows == need_read_rows, read_rows, need_read_rows); + return rows; +} + +template +UInt32 buildRowKeyFilterColumnFileBig( + const DMContext & dm_context, + const ColumnFileBig & cf_big, + const RowKeyRanges & delete_ranges, + const RowKeyRanges & read_ranges, + const UInt32 start_row_id, + std::vector & filter) +{ + if (cf_big.getRows() == 0) + return 0; + return buildRowKeyFilterDMFile( + dm_context, + cf_big.getFile(), + cf_big.getRange(), + delete_ranges, + Segment::shrinkRowKeyRanges(cf_big.getRange(), read_ranges), + nullptr, // stable_pack_res + start_row_id, + filter); +} + +template +UInt32 buildRowKeyFilterStable( + const DMContext & dm_context, + const StableValueSpace::Snapshot & stable, + const RowKeyRanges & delete_ranges, + const RowKeyRanges & read_ranges, + const RSResults & stable_pack_res, + std::vector & filter) +{ + const auto & dmfiles = stable.getDMFiles(); + RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); + const auto & dmfile = dmfiles[0]; + if (dmfile->getPacks() == 0) + return 0; + return buildRowKeyFilterDMFile( + dm_context, + dmfile, + std::nullopt, // segment_range + delete_ranges, + read_ranges, + &stable_pack_res, + 0, // start_row_id + filter); +} + +void buildRowKeyFilterDeleteRange(const ColumnFileDeleteRange & cf_delete_range, RowKeyRanges & delete_ranges) +{ + delete_ranges.push_back(cf_delete_range.getDeleteRange()); +} +} // namespace + +template +void buildRowKeyFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const RSResults & stable_pack_res, + std::vector & filter) +{ + const auto & delta = *(snapshot.delta); + const auto & stable = *(snapshot.stable); + const UInt32 delta_rows = delta.getRows(); + const UInt32 stable_rows = stable.getDMFilesRows(); + const UInt32 total_rows = delta_rows + stable_rows; + RUNTIME_CHECK(filter.size() == total_rows, filter.size(), total_rows); + + const auto cfs = delta.getColumnFiles(); + const auto & data_provider = delta.getDataProvider(); + RowKeyRanges delete_ranges; + UInt32 read_rows = 0; + // Read from new cfs to old cfs to handle delete range + for (auto itr = cfs.rbegin(); itr != cfs.rend(); ++itr) + { + const auto & cf = *itr; + if (const auto * cf_delete_range = cf->tryToDeleteRange(); cf_delete_range) + { + buildRowKeyFilterDeleteRange(*cf_delete_range, delete_ranges); + continue; + } + + const UInt32 cf_rows = cf->getRows(); + RUNTIME_CHECK(delta_rows >= read_rows + cf_rows, delta_rows, read_rows, cf_rows); + const UInt32 start_row_id = total_rows - read_rows - cf_rows; + read_rows += cf_rows; + + // TODO: add min-max value in tiny file to optimize rowkey filter. + if (cf->isInMemoryFile() || cf->isTinyFile()) + { + const auto n = buildRowKeyFilterBlock( + dm_context, + data_provider, + *cf, + delete_ranges, + read_ranges, + start_row_id, + filter); + RUNTIME_CHECK(cf_rows == n, cf_rows, n); + continue; + } + else if (const auto * cf_big = cf->tryToBigFile(); cf_big) + { + const auto n = buildRowKeyFilterColumnFileBig( + dm_context, + *cf_big, + delete_ranges, + read_ranges, + start_row_id, + filter); + RUNTIME_CHECK(cf_rows == n, cf_rows, n); + continue; + } + RUNTIME_CHECK_MSG(false, "{}: unknow ColumnFile type", cf->toString()); + } + RUNTIME_CHECK(read_rows == delta_rows, read_rows, delta_rows); + + const auto n + = buildRowKeyFilterStable(dm_context, stable, delete_ranges, read_ranges, stable_pack_res, filter); + RUNTIME_CHECK(n == stable_rows, n, stable_rows); +} + +template void buildRowKeyFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const RSResults & stable_pack_res, + std::vector & filter); + +// TODO: String +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h new file mode 100644 index 00000000000..05eaac1a679 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h @@ -0,0 +1,33 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +#include + +namespace DB::DM +{ +struct DMContext; +struct SegmentSnapshot; +struct RowKeyRange; +using RowKeyRanges = std::vector; + +template +void buildRowKeyFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const RSResults & stable_pack_res, + std::vector & filter); +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp new file mode 100644 index 00000000000..83a302b5c03 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -0,0 +1,230 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include + +namespace DB::DM +{ + +template +std::shared_ptr> VersionChain::replaySnapshot( + const DMContext & dm_context, + const SegmentSnapshot & snapshot) +{ + // Check Stable + if (dmfile_or_delete_range_list->empty()) + { + const auto & dmfiles = snapshot.stable->getDMFiles(); + RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); + dmfile_or_delete_range_list->push_back( + DMFileHandleIndex{dm_context.global_context, dmfiles[0], /*start_row_id*/ 0, std::nullopt}); + } + + const auto & stable = *(snapshot.stable); + const UInt32 stable_rows = stable.getDMFilesRows(); + const auto & delta = *(snapshot.delta); + const UInt32 delta_rows = delta.getRows(); + const UInt32 delta_delete_ranges = delta.getDeletes(); + std::lock_guard lock(mtx); + if (delta_rows + delta_delete_ranges <= replayed_rows_and_deletes) + { + RUNTIME_CHECK(base_versions->size() >= delta_rows, base_versions->size(), delta_rows); + return base_versions; + } + + base_versions = std::make_shared>(*base_versions); + const auto cfs = delta.getColumnFiles(); + const auto & data_provider = delta.getDataProvider(); + + UInt32 skipped_rows_and_deletes = 0; + auto pos = cfs.begin(); + for (; pos != cfs.end(); ++pos) + { + auto skip_n = (*pos)->isDeleteRange() ? (*pos)->getDeletes() : (*pos)->getRows(); + if (skip_n + skipped_rows_and_deletes > replayed_rows_and_deletes) + break; + skipped_rows_and_deletes += skip_n; + } + + // `pos` points to the first ColumnFile that has data not been replayed + auto offset = replayed_rows_and_deletes - skipped_rows_and_deletes; + // Only ColumnFileInMemory or ColumnFileTiny can be half replayed + RUNTIME_CHECK(offset == 0 || (*pos)->isInMemoryFile() || (*pos)->isTinyFile(), offset, (*pos)->toString()); + + const bool calculate_read_packs = (cfs.end() - pos == 1) && ((*pos)->isInMemoryFile() || (*pos)->isTinyFile()) + && dmfile_or_delete_range_list->size() == 1; + const auto initial_replayed_rows_and_deletes = replayed_rows_and_deletes; + SCOPE_EXIT({ cleanHandleColumn(); }); + for (; pos != cfs.end(); ++pos) + { + const auto & cf = *pos; + + if (cf->isInMemoryFile() || cf->isTinyFile()) + { + replayed_rows_and_deletes + += replayBlock(dm_context, data_provider, *cf, offset, stable_rows, calculate_read_packs); + offset = 0; + } + else if (const auto * cf_delete_range = cf->tryToDeleteRange(); cf_delete_range) + { + replayed_rows_and_deletes += replayDeleteRange(*cf_delete_range); + } + else if (const auto * cf_big = cf->tryToBigFile(); cf_big) + { + replayed_rows_and_deletes += replayColumnFileBig(dm_context, *cf_big, stable_rows); + } + else + { + RUNTIME_CHECK_MSG(false, "Unknow column file: {}", cf->toString()); + } + } + + RUNTIME_CHECK( + replayed_rows_and_deletes == delta_rows + delta_delete_ranges, + replayed_rows_and_deletes, + delta_rows, + delta_delete_ranges); + RUNTIME_CHECK(base_versions->size() == delta_rows, base_versions->size(), delta_rows); + + LOG_INFO( + snapshot.log, + "replays {} rows. initial_replayed_rows_and_deletes={}, delta_rows={}, delta_delete_ranges={}, " + "replayed_rows_and_deletes={}.", + replayed_rows_and_deletes - initial_replayed_rows_and_deletes, + initial_replayed_rows_and_deletes, + delta_rows, + delta_delete_ranges, + replayed_rows_and_deletes); + + return base_versions; +} + +template +UInt32 VersionChain::replayBlock( + const DMContext & dm_context, + const IColumnFileDataProviderPtr & data_provider, + const ColumnFile & cf, + const UInt32 offset, + const UInt32 stable_rows, + const bool calculate_read_packs) +{ + assert(cf.isInMemoryFile() || cf.isTinyFile()); + + auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); + auto block = cf_reader->readNextBlock(); + RUNTIME_CHECK_MSG(!cf_reader->readNextBlock(), "{}: read all rows in one block is required!", cf.toString()); + const auto * handle_col = toColumnVectorDataPtr(block.begin()->column); + RUNTIME_CHECK_MSG(handle_col != nullptr, "TODO: support common handle"); + RUNTIME_CHECK(handle_col->size() > offset, handle_col->size(), offset); + const std::span handles{handle_col->data() + offset, handle_col->size() - offset}; + + if (calculate_read_packs) + calculateReadPacks(handles); + + for (auto h : handles) + { + const RowID curr_row_id = base_versions->size() + stable_rows; + if (auto itr = new_handle_to_row_ids->find(h); itr != new_handle_to_row_ids->end()) + { + base_versions->push_back(itr->second); + continue; + } + if (auto row_id = findBaseVersionFromDMFileOrDeleteRangeList(h); row_id) + { + base_versions->push_back(*row_id); + continue; + } + + new_handle_to_row_ids->insert(std::make_pair(h, curr_row_id)); + base_versions->push_back(NotExistRowID); + } + return handles.size(); +} + +template +UInt32 VersionChain::replayColumnFileBig( + const DMContext & dm_context, + const ColumnFileBig & cf_big, + const UInt32 stable_rows) +{ + const UInt32 rows = cf_big.getRows(); + const UInt32 start_row_id = base_versions->size() + stable_rows; + base_versions->insert(base_versions->end(), rows, NotExistRowID); + + dmfile_or_delete_range_list->push_back( + DMFileHandleIndex{dm_context.global_context, cf_big.getFile(), start_row_id, cf_big.getRange()}); + return rows; +} + +template +UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range) +{ + auto [start, end] = convertRowKeyRange(cf_delete_range.getDeleteRange()); + auto itr = new_handle_to_row_ids->lower_bound(start); + const auto end_itr = new_handle_to_row_ids->lower_bound(end); + std::vector erased_handles; + while (itr != end_itr) + { + erased_handles.push_back(itr->first); + itr = new_handle_to_row_ids->erase(itr); + } + dmfile_or_delete_range_list->push_back(cf_delete_range.getDeleteRange()); + return cf_delete_range.getDeletes(); +} + +template +std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRangeList(Handle h) +{ + for (auto itr = dmfile_or_delete_range_list->rbegin(); itr != dmfile_or_delete_range_list->rend(); ++itr) + { + auto & dmfile_or_delete_range = *itr; + if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) + { + if (auto row_id = dmfile_index->getBaseVersion(h); row_id) + return row_id; + } + else if (auto * delete_range = std::get_if(&dmfile_or_delete_range); delete_range) + { + if (inRowKeyRange(*delete_range, h)) + return {}; + } + } + return {}; +} + +template +void VersionChain::calculateReadPacks(const std::span handles) +{ + assert(dmfile_or_delete_range_list->size() == 1); + auto & dmfile_index = std::get>(dmfile_or_delete_range_list->front()); + dmfile_index.calculateReadPacks(handles); +} + +template +void VersionChain::cleanHandleColumn() +{ + for (auto & dmfile_or_delete_range : *dmfile_or_delete_range_list) + { + if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) + dmfile_index->cleanHandleColumn(); + } +} + +template class VersionChain; +//template class VersionChain; + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h new file mode 100644 index 00000000000..2779b5386b7 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -0,0 +1,85 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +#include +#include +#include +#include + +namespace DB::DM +{ + +struct DMContext; +struct SegmentSnapshot; +class ColumnFile; +class ColumnFileBig; +class ColumnFileDeleteRange; + +template +class VersionChain +{ +public: + VersionChain() + : base_versions(std::make_shared>()) + , new_handle_to_row_ids(std::make_shared>()) + , dmfile_or_delete_range_list(std::make_shared>()) + {} + + [[nodiscard]] std::shared_ptr> replaySnapshot( + const DMContext & dm_context, + const SegmentSnapshot & snapshot); + + std::unique_ptr> deepCopy() + { + auto new_version_chain = std::make_unique>(); + new_version_chain->replayed_rows_and_deletes = replayed_rows_and_deletes; + new_version_chain->base_versions = std::make_shared>(*base_versions); + new_version_chain->new_handle_to_row_ids = std::make_shared>(*new_handle_to_row_ids); + new_version_chain->dmfile_or_delete_range_list + = std::make_shared>(*dmfile_or_delete_range_list); + return new_version_chain; + } + + UInt32 getReplayedRows() const { return base_versions->size(); } + +private: + [[nodiscard]] UInt32 replayBlock( + const DMContext & dm_context, + const IColumnFileDataProviderPtr & data_provider, + const ColumnFile & cf, + const UInt32 offset, + const UInt32 stable_rows, + const bool calculate_read_packs); + [[nodiscard]] UInt32 replayColumnFileBig( + const DMContext & dm_context, + const ColumnFileBig & cf_big, + const UInt32 stable_rows); + [[nodiscard]] UInt32 replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range); + + [[nodiscard]] std::optional findBaseVersionFromDMFileOrDeleteRangeList(Handle h); + void calculateReadPacks(const std::span handles); + void cleanHandleColumn(); + + DISALLOW_COPY_AND_MOVE(VersionChain); + + std::mutex mtx; + UInt32 replayed_rows_and_deletes = 0; // delta.getRows() + delta.getDeletes() + std::shared_ptr> base_versions; // base_versions->size() == delta.getRows() + std::shared_ptr> new_handle_to_row_ids; // TODO: shared_ptr is unneccessary + using DMFileOrDeleteRange = std::variant>; + std::shared_ptr> dmfile_or_delete_range_list; // TODO: shared_ptr is unneccessary +}; +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp new file mode 100644 index 00000000000..f95847e2bc4 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp @@ -0,0 +1,276 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include +#include +#include +#include + +namespace DB::DM +{ +[[nodiscard]] UInt32 buildVersionFilterBlock( + const DMContext & dm_context, + const IColumnFileDataProviderPtr & data_provider, + const ColumnFile & cf, + const UInt64 read_ts, + const std::vector & base_ver_snap, + const UInt32 stable_rows, + const UInt32 start_row_id, + std::vector & filter) +{ + assert(cf.isInMemoryFile() || cf.isTinyFile()); + auto cf_reader = cf.getReader(dm_context, data_provider, getVersionColumnDefinesPtr(), ReadTag::MVCC); + auto block = cf_reader->readNextBlock(); + if (!block) + return 0; + RUNTIME_CHECK_MSG(!cf_reader->readNextBlock(), "{}: read all rows in one block is required!", cf.toString()); + const auto & versions = *toColumnVectorDataPtr(block.begin()->column); // Must success. + + // Traverse data from new to old + for (ssize_t i = versions.size() - 1; i >= 0; --i) + { + const UInt32 row_id = start_row_id + i; + + if (!filter[row_id]) + continue; + + // invisible + if (versions[i] > read_ts) + { + filter[row_id] = 0; + continue; + } + + // visible + + const auto base_row_id = base_ver_snap[row_id - stable_rows]; + // Newer version has been chosen. + if (base_row_id != NotExistRowID && !filter[base_row_id]) + { + filter[row_id] = 0; + continue; + } + // Choose this version + if (base_row_id != NotExistRowID) + filter[base_row_id] = 0; + } + return versions.size(); +} + + +[[nodiscard]] UInt32 buildVersionFilterDMFile( + const DMContext & dm_context, + const DMFilePtr & dmfile, + const std::optional & segment_range, + const UInt64 read_ts, + const ssize_t start_row_id, + std::vector & filter) +{ + auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges( + dm_context.global_context, + dm_context.scan_context, + dm_context.tracing_id, + dmfile, segment_range); + if (valid_handle_res.empty()) + return 0; + + const auto max_versions = loadPackMaxValue(dm_context.global_context, *dmfile, VERSION_COLUMN_ID); + + auto read_packs = std::make_shared(); + UInt32 need_read_rows = 0; + std::unordered_map read_pack_to_start_row_ids; + + const auto & pack_stats = dmfile->getPackStats(); + UInt32 rows = 0; + for (UInt32 i = 0; i < valid_handle_res.size(); ++i) + { + const UInt32 pack_id = valid_start_pack_id + i; + const UInt32 pack_start_row_id = start_row_id + rows; + const auto & stat = pack_stats[pack_id]; + if (stat.not_clean || max_versions[pack_id] > read_ts) + { + read_packs->insert(pack_id); + read_pack_to_start_row_ids.emplace(pack_id, pack_start_row_id); + need_read_rows += stat.rows; + } + rows += stat.rows; + } + + if (need_read_rows == 0) + return rows; + + // If all packs need to read is clean, we can just read version column. + // However, the benefits in general scenarios may not be significant. + + DMFileBlockInputStreamBuilder builder(dm_context.global_context); + builder.onlyReadOnePackEveryTime().setReadPacks(read_packs).setReadTag(ReadTag::MVCC); + auto stream = builder.build( + dmfile, + {getHandleColumnDefine(), getVersionColumnDefine()}, + {}, + dm_context.scan_context); + + UInt32 read_rows = 0; + for (auto pack_id : *read_packs) + { + auto block = stream->read(); + RUNTIME_CHECK(block.rows() == pack_stats[pack_id].rows, block.rows(), pack_stats[pack_id].rows); + read_rows += block.rows(); + const auto * handles_ptr = toColumnVectorDataPtr(block.getByName(EXTRA_HANDLE_COLUMN_NAME).column); + RUNTIME_CHECK_MSG(handles_ptr != nullptr, "TODO: support common handle"); + const auto & handles = *handles_ptr; + const auto & versions + = *toColumnVectorDataPtr(block.getByName(VERSION_COLUMN_NAME).column); // Must success. + + const auto itr = read_pack_to_start_row_ids.find(pack_id); + RUNTIME_CHECK(itr != read_pack_to_start_row_ids.end(), read_pack_to_start_row_ids, pack_id); + const UInt32 pack_start_row_id = itr->second; + + // Filter invisible versions + if (max_versions[pack_id] > read_ts) + { + for (UInt32 i = 0; i < block.rows(); ++i) + { + // TODO: benchmark + // filter[pack_start_row_id + i] = versions[i] <= read_ts; + if unlikely (versions[i] > read_ts) + filter[pack_start_row_id + i] = 0; + } + } + + // Filter multiple versions + if (pack_stats[pack_id].not_clean) + { + // [handle_itr, handle_end) is a pack. + auto handle_itr = handles.begin(); + const auto handle_end = handle_itr + pack_stats[pack_id].rows; + for (;;) + { + auto itr = std::adjacent_find(handle_itr, handle_end); + if (itr == handle_end) + break; + + // Let `handle_itr` point to next different handle. + handle_itr = std::find_if(itr, handle_end, [h = *itr](Int64 a) { return h != a; }); + // [itr, handle_itr) are the same handle of different verions. + auto count = std::distance(itr, handle_itr); + + const UInt32 base_row_id = pack_start_row_id + std::distance(handles.begin(), itr); + if (!filter[base_row_id]) + { + std::fill_n(filter.begin() + base_row_id + 1, count - 1, 0); + continue; + } + else + { + for (UInt32 i = 1; i < count; ++i) + { + if (filter[base_row_id + i]) + filter[base_row_id + i - 1] = 0; + else + break; + } + } + } + } + } + RUNTIME_CHECK(read_rows == need_read_rows, read_rows, need_read_rows); + return rows; +} + +[[nodiscard]] UInt32 buildVersionFilterColumnFileBig( + const DMContext & dm_context, + const ColumnFileBig & cf_big, + const UInt64 read_ts, + const ssize_t start_row_id, + std::vector & filter) +{ + return buildVersionFilterDMFile(dm_context, cf_big.getFile(), cf_big.getRange(), read_ts, start_row_id, filter); +} + +[[nodiscard]] UInt32 buildVersionFilterStable( + const DMContext & dm_context, + const StableValueSpace::Snapshot & stable, + const UInt64 read_ts, + std::vector & filter) +{ + const auto & dmfiles = stable.getDMFiles(); + RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); + return buildVersionFilterDMFile(dm_context, dmfiles[0], std::nullopt, read_ts, 0, filter); +} + +void buildVersionFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const std::vector & base_ver_snap, + const UInt64 read_ts, + std::vector & filter) +{ + const auto & delta = *(snapshot.delta); + const auto & stable = *(snapshot.stable); + const UInt32 delta_rows = delta.getRows(); + const UInt32 stable_rows = stable.getDMFilesRows(); + const UInt32 total_rows = delta_rows + stable_rows; + RUNTIME_CHECK(filter.size() == total_rows, filter.size(), total_rows); + + // Delta MVCC + const auto cfs = delta.getColumnFiles(); + const auto & data_provider = delta.getDataProvider(); + + UInt32 read_rows = 0; + + // Traverse data from new to old + for (auto itr = cfs.rbegin(); itr != cfs.rend(); ++itr) + { + const auto & cf = *itr; + if (cf->isDeleteRange()) + continue; + + const UInt32 cf_rows = cf->getRows(); + RUNTIME_CHECK(delta_rows >= read_rows + cf_rows, delta_rows, read_rows, cf_rows); + const UInt32 start_row_id = total_rows - read_rows - cf_rows; + read_rows += cf_rows; + + // TODO: add clean and max version in tiny file + if (cf->isInMemoryFile() || cf->isTinyFile()) + { + const auto n = buildVersionFilterBlock( + dm_context, + data_provider, + *cf, + read_ts, + base_ver_snap, + stable_rows, + start_row_id, + filter); + RUNTIME_CHECK(cf_rows == n, cf_rows, n); + continue; + } + + if (const auto * cf_big = cf->tryToBigFile(); cf_big) + { + const auto n = buildVersionFilterColumnFileBig(dm_context, *cf_big, read_ts, start_row_id, filter); + RUNTIME_CHECK(cf_rows == n, cf_rows, n); + continue; + } + RUNTIME_CHECK_MSG(false, "{}: unknow ColumnFile type", cf->toString()); + } + RUNTIME_CHECK(read_rows == delta_rows, read_rows, delta_rows); + const auto n = buildVersionFilterStable(dm_context, stable, read_ts, filter); + RUNTIME_CHECK(n == stable_rows, n, stable_rows); +} +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h new file mode 100644 index 00000000000..ff4230ec3d3 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h @@ -0,0 +1,29 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +#include + +namespace DB::DM +{ +struct DMContext; +struct SegmentSnapshot; +void buildVersionFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const std::vector & base_ver_snap, + const UInt64 read_ts, + std::vector & filter); +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_btree_hash.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_btree_hash.cpp new file mode 100644 index 00000000000..9b859bf50a1 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_btree_hash.cpp @@ -0,0 +1,148 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 + +#include +#include + +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wdeprecated-builtins" + +#include +#include + +#pragma clang diagnostic pop + +namespace +{ + +enum class IndexType +{ + BTree = 0, + Hash = 1, +}; + +enum class SearchResult +{ + AlwaysFound = 0, + AlwaysNotFound = 1, +}; + +std::vector> randomData(UInt32 size) +{ + std::vector> v(size); + for (UInt32 i = 0; i < size; ++i) + { + v[i] = {i, i}; + } + std::shuffle(v.begin(), v.end(), std::mt19937(std::random_device()())); + return v; +} + +template +void build(benchmark::State & state, Args &&... args) +{ + const auto [type, size] = std::make_tuple(std::move(args)...); + const auto v = randomData(size); + + if (type == IndexType::BTree) + { + for (auto _ : state) + { + absl::btree_map t(v.begin(), v.end()); + benchmark::DoNotOptimize(t); + } + } + else + { + for (auto _ : state) + { + absl::flat_hash_map t(v.begin(), v.end()); + benchmark::DoNotOptimize(t); + } + } +} + +template +void search(benchmark::State & state, Args &&... args) +{ + const auto [index_type, search_result, size] = std::make_tuple(std::move(args)...); + const auto v = randomData(size); + + std::random_device rd; + std::mt19937 g(rd()); + + auto gen_always_found = [&]() { + return g() % size; + }; + auto gen_always_not_found = [&]() { + return gen_always_found() + size; + }; + std::function gen; + if (search_result == SearchResult::AlwaysFound) + gen = gen_always_found; + else + gen = gen_always_not_found; + + if (index_type == IndexType::BTree) + { + absl::btree_map t(v.begin(), v.end()); + for (auto _ : state) + { + auto itr = t.find(gen()); + benchmark::DoNotOptimize(itr); + } + } + else + { + absl::flat_hash_map t(v.begin(), v.end()); + for (auto _ : state) + { + auto itr = t.find(gen()); + benchmark::DoNotOptimize(itr); + } + } +} + +BENCHMARK_CAPTURE(search, btree_found_10000, IndexType::BTree, SearchResult::AlwaysFound, 10000); +BENCHMARK_CAPTURE(search, btree_found_20000, IndexType::BTree, SearchResult::AlwaysFound, 20000); +BENCHMARK_CAPTURE(search, btree_found_40000, IndexType::BTree, SearchResult::AlwaysFound, 40000); +BENCHMARK_CAPTURE(search, btree_found_80000, IndexType::BTree, SearchResult::AlwaysFound, 80000); +BENCHMARK_CAPTURE(search, hash_found_10000, IndexType::Hash, SearchResult::AlwaysFound, 10000); +BENCHMARK_CAPTURE(search, hash_found_20000, IndexType::Hash, SearchResult::AlwaysFound, 20000); +BENCHMARK_CAPTURE(search, hash_found_40000, IndexType::Hash, SearchResult::AlwaysFound, 40000); +BENCHMARK_CAPTURE(search, hash_found_80000, IndexType::Hash, SearchResult::AlwaysFound, 80000); + +BENCHMARK_CAPTURE(search, btree_not_found_10000, IndexType::BTree, SearchResult::AlwaysNotFound, 10000); +BENCHMARK_CAPTURE(search, btree_not_found_20000, IndexType::BTree, SearchResult::AlwaysNotFound, 20000); +BENCHMARK_CAPTURE(search, btree_not_found_40000, IndexType::BTree, SearchResult::AlwaysNotFound, 40000); +BENCHMARK_CAPTURE(search, btree_not_found_80000, IndexType::BTree, SearchResult::AlwaysNotFound, 80000); +BENCHMARK_CAPTURE(search, hash_not_found_10000, IndexType::Hash, SearchResult::AlwaysNotFound, 10000); +BENCHMARK_CAPTURE(search, hash_not_found_20000, IndexType::Hash, SearchResult::AlwaysNotFound, 20000); +BENCHMARK_CAPTURE(search, hash_not_found_40000, IndexType::Hash, SearchResult::AlwaysNotFound, 40000); +BENCHMARK_CAPTURE(search, hash_not_found_80000, IndexType::Hash, SearchResult::AlwaysNotFound, 80000); + +BENCHMARK_CAPTURE(build, btree, IndexType::BTree, 10000); +BENCHMARK_CAPTURE(build, btree, IndexType::BTree, 20000); +BENCHMARK_CAPTURE(build, btree, IndexType::BTree, 40000); +BENCHMARK_CAPTURE(build, btree, IndexType::BTree, 80000); +BENCHMARK_CAPTURE(build, hash, IndexType::Hash, 10000); +BENCHMARK_CAPTURE(build, hash, IndexType::Hash, 20000); +BENCHMARK_CAPTURE(build, hash, IndexType::Hash, 40000); +BENCHMARK_CAPTURE(build, hash, IndexType::Hash, 80000); + +} // namespace diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp new file mode 100644 index 00000000000..e2e28463a13 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp @@ -0,0 +1,188 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include +namespace +{ + +enum class SearchType +{ + LinearScan = 0, + BinarySearch = 1, +}; + +enum class SearchResult +{ + AlwaysFound = 0, + AlwaysNotFound = 1, +}; + +template +void searchInt64(benchmark::State & state, Args &&... args) +{ + const auto [type, size, result] = std::make_tuple(std::move(args)...); + std::vector v(size); + std::iota(v.begin(), v.end(), 0); + std::random_device rd; + std::mt19937 g(rd()); + + auto gen_always_found = [&]() { + return g() % size; + }; + auto gen_always_not_found = [&]() { + return gen_always_found() + size; + }; + std::function gen; + if (result == SearchResult::AlwaysFound) + gen = gen_always_found; + else + gen = gen_always_not_found; + + if (type == SearchType::BinarySearch) + { + for (auto _ : state) + { + auto itr = std::lower_bound(v.begin(), v.end(), gen()); + benchmark::DoNotOptimize(itr); + } + } + else if (type == SearchType::LinearScan) + { + for (auto _ : state) + { + auto itr = std::find(v.begin(), v.end(), gen()); + benchmark::DoNotOptimize(itr); + } + } + + /* + else if (search_type == SearchType::StdSet) + { + std::set s(v.begin(), v.end()); + for (auto _ : state) + { + auto itr = s.find(gen()); + benchmark::DoNotOptimize(itr); + } + } + else if (search_type == SearchType::AbslSet) + { + absl::btree_set s(v.begin(), v.end()); + for (auto _ : state) + { + auto itr = s.find(gen()); + benchmark::DoNotOptimize(itr); + } + }*/ +} + +struct Entry +{ + std::int64_t id, a, b; +}; + +template +void searchStruct(benchmark::State & state, Args &&... args) +{ + const auto [type, size, result] = std::make_tuple(std::move(args)...); + std::vector v(size); + for (int i = 0; i < size; ++i) + v[i].id = i; + std::random_device rd; + std::mt19937 g(rd()); + + auto gen_always_found = [&]() { + return g() % size; + }; + auto gen_always_not_found = [&]() { + return gen_always_found() + size; + }; + std::function gen; + if (result == SearchResult::AlwaysFound) + gen = gen_always_found; + else + gen = gen_always_not_found; + + if (type == SearchType::BinarySearch) + { + for (auto _ : state) + { + auto itr + = std::lower_bound(v.begin(), v.end(), gen(), [](const Entry & e, std::int64_t v) { return e.id < v; }); + benchmark::DoNotOptimize(itr); + } + } + /* + else if (type == SearchType::LinearScan) + { + for (auto _ : state) + { + auto itr = std::find(v.begin(), v.end(), gen()); + benchmark::DoNotOptimize(itr); + } + } + else if (search_type == SearchType::StdSet) + { + std::set s(v.begin(), v.end()); + for (auto _ : state) + { + auto itr = s.find(gen()); + benchmark::DoNotOptimize(itr); + } + } + else if (search_type == SearchType::AbslSet) + { + absl::btree_set s(v.begin(), v.end()); + for (auto _ : state) + { + auto itr = s.find(gen()); + benchmark::DoNotOptimize(itr); + } + }*/ +} +/* +BENCHMARK_CAPTURE(searchInt64, LinearScan4, SearchType::LinearScan, 4, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, LinearScan8, SearchType::LinearScan, 8, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, LinearScan16, SearchType::LinearScan, 16, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, LinearScan32, SearchType::LinearScan, 32, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, LinearScan64, SearchType::LinearScan, 64, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, LinearScan128, SearchType::LinearScan, 128, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, LinearScan256, SearchType::LinearScan, 256, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, LinearScan512, SearchType::LinearScan, 512, SearchResult::AlwaysFound); +*/ + +BENCHMARK_CAPTURE(searchInt64, BinarySearch4, SearchType::BinarySearch, 4, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, BinarySearch8, SearchType::BinarySearch, 8, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, BinarySearch16, SearchType::BinarySearch, 16, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, BinarySearch32, SearchType::BinarySearch, 32, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, BinarySearch64, SearchType::BinarySearch, 64, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, BinarySearch128, SearchType::BinarySearch, 128, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, BinarySearch256, SearchType::BinarySearch, 256, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchInt64, BinarySearch512, SearchType::BinarySearch, 512, SearchResult::AlwaysFound); + + +BENCHMARK_CAPTURE(searchStruct, BinarySearch4, SearchType::BinarySearch, 4, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchStruct, BinarySearch8, SearchType::BinarySearch, 8, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchStruct, BinarySearch16, SearchType::BinarySearch, 16, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchStruct, BinarySearch32, SearchType::BinarySearch, 32, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchStruct, BinarySearch64, SearchType::BinarySearch, 64, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchStruct, BinarySearch128, SearchType::BinarySearch, 128, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchStruct, BinarySearch256, SearchType::BinarySearch, 256, SearchResult::AlwaysFound); +BENCHMARK_CAPTURE(searchStruct, BinarySearch512, SearchType::BinarySearch, 512, SearchResult::AlwaysFound); + +} // namespace diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp new file mode 100644 index 00000000000..334f9beff9d --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -0,0 +1,493 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include +#include +#include +#include +#include + +#include +#include +using namespace DB; +using namespace DB::tests; +using namespace DB::DM; +using namespace DB::DM::tests; + +namespace CurrentMetrics +{ +extern const Metric DT_SnapshotOfRead; +} // namespace CurrentMetrics + +namespace +{ +ContextPtr context; +DMContextPtr dm_context; +ColumnDefinesPtr cols; + +SegmentPtr segment; +SegmentSnapshotPtr segment_snapshot; + +constexpr const char * log_level = "error"; + +const String db_name = "test"; + +UInt64 version = 1; + +enum class BenchType +{ + None = 0, + DeltaIndex = 1, + VersionChain = 2, +}; + +void initContext(bool is_common_handle, BenchType type) +{ + if (context) + return; + + bool enable_colors = isatty(STDERR_FILENO) && isatty(STDOUT_FILENO); + DB::tests::TiFlashTestEnv::setupLogger(log_level, std::cerr, enable_colors); + + auto table_name = String(magic_enum::enum_name(type)); + UInt64 curr_ns = clock_gettime_ns(); + String testdata_path = fmt::format("/tmp/{}.{}", curr_ns, table_name); + constexpr auto run_mode = DB::PageStorageRunMode::ONLY_V3; + TiFlashTestEnv::initializeGlobalContext({testdata_path}, run_mode); + context = TiFlashTestEnv::getContext(); + + auto storage_path_pool + = std::make_shared(context->getPathPool().withTable(db_name, table_name, false)); + auto storage_pool = std::make_shared( + *context, + NullspaceID, + /*NAMESPACE_ID*/ 100, + *storage_path_pool, + fmt::format("{}.{}", db_name, table_name)); + storage_pool->restore(); + cols = DMTestEnv::getDefaultColumns( + is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID); + dm_context = DMContext::createUnique( + *context, + storage_path_pool, + storage_pool, + /*min_version_*/ 0, + NullspaceID, + /*physical_table_id*/ 100, + /*pk_col_id*/ EXTRA_HANDLE_COLUMN_ID, + is_common_handle, + 1, // rowkey_column_size + context->getSettingsRef()); +} + +SegmentPtr createSegment(bool is_common_handle) +{ + return Segment::newSegment( + Logger::get(), + *dm_context, + cols, + RowKeyRange::newAll(is_common_handle, 1), + DELTA_MERGE_FIRST_SEGMENT_ID, + 0); +} + +constexpr Int64 MaxHandle = 1000000; + +class RandomSequence +{ +public: + RandomSequence(UInt32 n) + : v(randomInt64s(n)) + , pos(v.begin()) + {} + + std::vector get(UInt32 n) + { + std::vector res; + while (res.size() < n) + { + auto copied = std::min(std::distance(pos, v.end()), static_cast(n - res.size())); + res.insert(res.end(), pos, pos + copied); + std::advance(pos, copied); + if (pos == v.end()) + reset(); + } + return res; + } + + void reset() { pos = v.begin(); } + +private: + std::vector randomInt64s(UInt32 n) + { + static constexpr int rnd_seed = 573172; + std::mt19937 g(rnd_seed); + std::vector v(n); + for (UInt32 i = 0; i < n; ++i) + { + v[i] = g() % MaxHandle; + } + return v; + } + + std::vector v; + std::vector::iterator pos; +}; + +RandomSequence random_sequences{10 * MaxHandle}; + +void writeDelta(Segment & seg, UInt32 delta_rows) +{ + for (UInt32 i = 0; i < delta_rows; i += 2048) + { + Block block; + const auto n = std::min(delta_rows - i, 2048U); + const auto v = random_sequences.get(n); + block.insert(createColumn(v, EXTRA_HANDLE_COLUMN_NAME, EXTRA_HANDLE_COLUMN_ID)); + block.insert(createColumn(std::vector(n, version++), VERSION_COLUMN_NAME, VERSION_COLUMN_ID)); + block.insert(createColumn(std::vector(n, /*deleted*/ 0), TAG_COLUMN_NAME, TAG_COLUMN_ID)); + seg.write(*dm_context, block, false); + } +} + +SegmentPtr createSegment(bool is_common_handle, UInt32 delta_rows) +{ + auto seg = createSegment(is_common_handle); + { + auto block = DMTestEnv::prepareSimpleWriteBlock(0, 1000000, false, version++); + seg->write(*dm_context, block, false); + seg = seg->mergeDelta(*dm_context, cols); + } + + writeDelta(*seg, delta_rows); + + return seg; +} + +DeltaIndexPtr buildDeltaIndex(const SegmentSnapshotPtr & snapshot, Segment & segment) +{ + auto pk_ver_col_defs = std::make_shared( + ColumnDefines{getExtraHandleColumnDefine(dm_context->is_common_handle), getVersionColumnDefine()}); + + auto delta_reader = std::make_shared( + *dm_context, + snapshot->delta, + pk_ver_col_defs, + segment.getRowKeyRange(), + ReadTag::MVCC); + + auto [delta_index, fully_indexed] = segment.ensurePlace( + *dm_context, + snapshot, + delta_reader, + {segment.getRowKeyRange()}, + std::numeric_limits::max()); + + if (delta_index == nullptr || !fully_indexed) + std::abort(); + + return delta_index; +} + +void buildVersionChain(const SegmentSnapshot & snapshot, VersionChain & version_chain) +{ + const auto base_ver_snap = version_chain.replaySnapshot(*dm_context, snapshot); + benchmark::DoNotOptimize(base_ver_snap); +} + +void initialize(BenchType type, bool is_common_handle, UInt32 delta_rows) +{ + random_sequences.reset(); + initContext(is_common_handle, type); + segment = createSegment(is_common_handle, delta_rows); + segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); +} + +void shutdown() +{ + segment_snapshot = nullptr; + segment = nullptr; + cols = nullptr; + dm_context = nullptr; + context->shutdown(); + context = nullptr; +} + +template +void MVCCFullBuildIndex(benchmark::State & state, Args &&... args) +try +{ + const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); + initialize(type, is_common_handle, delta_rows); + + if (type == BenchType::DeltaIndex) + { + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); + auto delta_index = buildDeltaIndex(segment_snapshot, *segment); // Warming up + RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); + for (auto _ : state) + { + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); + delta_index = buildDeltaIndex(segment_snapshot, *segment); + RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); + } + } + else if (type == BenchType::VersionChain) + { + { + VersionChain version_chain; + buildVersionChain(*segment_snapshot, version_chain); // Warming up + RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); + } + for (auto _ : state) + { + VersionChain version_chain; + buildVersionChain(*segment_snapshot, version_chain); + RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); + } + } + shutdown(); +} +CATCH + +template +void MVCCIncrementalBuildIndex(benchmark::State & state, Args &&... args) +try +{ + const auto [type, is_common_handle, incremental_delta_rows] = std::make_tuple(std::move(args)...); + constexpr UInt32 prepared_delta_rows = 10000; + initialize(type, is_common_handle, prepared_delta_rows); + + if (type == BenchType::DeltaIndex) + { + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); + auto base_delta_index = buildDeltaIndex(segment_snapshot, *segment); + RUNTIME_ASSERT(base_delta_index->getPlacedStatus().first == prepared_delta_rows); + + segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*base_delta_index); + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == prepared_delta_rows); + + writeDelta(*segment, incremental_delta_rows); + segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == prepared_delta_rows); + RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); + + for (auto _ : state) + { + RUNTIME_ASSERT( + segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == prepared_delta_rows); + auto delta_index = buildDeltaIndex(segment_snapshot, *segment); + RUNTIME_ASSERT(delta_index->getPlacedStatus().first == prepared_delta_rows + incremental_delta_rows); + } + } + else if (type == BenchType::VersionChain) + { + VersionChain base_version_chain; + buildVersionChain(*segment_snapshot, base_version_chain); + RUNTIME_ASSERT(base_version_chain.getReplayedRows() == prepared_delta_rows); + writeDelta(*segment, incremental_delta_rows); + segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); + RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); + for (auto _ : state) + { + auto version_chain = base_version_chain.deepCopy(); + RUNTIME_ASSERT(version_chain->getReplayedRows() == prepared_delta_rows); + buildVersionChain(*segment_snapshot, *version_chain); + RUNTIME_ASSERT(version_chain->getReplayedRows() == prepared_delta_rows + incremental_delta_rows); + } + } + + shutdown(); +} +CATCH + +template +void MVCCBuildBitmap(benchmark::State & state, Args &&... args) +try +{ + const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); + initialize(type, is_common_handle, delta_rows); + + if (type == BenchType::DeltaIndex) + { + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); + auto delta_index = buildDeltaIndex(segment_snapshot, *segment); + RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); + segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*delta_index); + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == delta_rows); + + for (auto _ : state) + { + auto bitmap_filter = segment->buildBitmapFilter( + *dm_context, + segment_snapshot, + {segment->getRowKeyRange()}, + nullptr, + std::numeric_limits::max(), + DEFAULT_BLOCK_SIZE, + false); + benchmark::DoNotOptimize(bitmap_filter); + } + } + else if (type == BenchType::VersionChain) + { + VersionChain version_chain; + buildVersionChain(*segment_snapshot, version_chain); + RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); + for (auto _ : state) + { + auto bitmap_filter = buildBitmapFilter( + *dm_context, + *segment_snapshot, + {segment->getRowKeyRange()}, + nullptr, + std::numeric_limits::max(), + version_chain); + benchmark::DoNotOptimize(bitmap_filter); + } + } + shutdown(); +} +CATCH + +// TODO: move verify to unit-tests. +template +void MVCCBuildBitmapVerify(benchmark::State & state, Args &&... args) +try +{ + const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); + initialize(type, is_common_handle, delta_rows); + + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); + auto delta_index = buildDeltaIndex(segment_snapshot, *segment); + RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); + segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*delta_index); + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == delta_rows); + + VersionChain version_chain; + buildVersionChain(*segment_snapshot, version_chain); + RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); + + + auto bitmap_filter1 = segment->buildBitmapFilter( + *dm_context, + segment_snapshot, + {segment->getRowKeyRange()}, + nullptr, + std::numeric_limits::max(), + DEFAULT_BLOCK_SIZE, + false); + auto bitmap_filter2 = buildBitmapFilter( + *dm_context, + *segment_snapshot, + {segment->getRowKeyRange()}, + nullptr, + std::numeric_limits::max(), + version_chain); + + const auto & filter1 = bitmap_filter1->getFilter(); + const auto & filter2 = bitmap_filter2->getFilter(); + RUNTIME_ASSERT(filter1.size() == filter2.size()); + for (UInt32 i = 0; i < filter1.size(); ++i) + { + if (filter1[i] != filter2[i]) + { + fmt::println("i={}, filter1={}, filter2={}", i, filter1[i], filter2[i]); + std::abort(); + } + } + + for (auto _ : state) {} + + shutdown(); +} +CATCH + + +//constexpr bool IsCommonHandle = true; +constexpr bool IsNotCommonHandle = false; + +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_10, BenchType::DeltaIndex, IsNotCommonHandle, 10u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_100, BenchType::DeltaIndex, IsNotCommonHandle, 100u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_500, BenchType::DeltaIndex, IsNotCommonHandle, 500u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_1000, BenchType::DeltaIndex, IsNotCommonHandle, 1000u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_5000, BenchType::DeltaIndex, IsNotCommonHandle, 5000u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_10000, BenchType::DeltaIndex, IsNotCommonHandle, 10000u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_50000, BenchType::DeltaIndex, IsNotCommonHandle, 50000u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_100000, BenchType::DeltaIndex, IsNotCommonHandle, 100000u); + +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_1, BenchType::VersionChain, IsNotCommonHandle, 1u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_5, BenchType::VersionChain, IsNotCommonHandle, 5u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_10, BenchType::VersionChain, IsNotCommonHandle, 10u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_100, BenchType::VersionChain, IsNotCommonHandle, 100u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_500, BenchType::VersionChain, IsNotCommonHandle, 500u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_1000, BenchType::VersionChain, IsNotCommonHandle, 1000u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_5000, BenchType::VersionChain, IsNotCommonHandle, 5000u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_10000, BenchType::VersionChain, IsNotCommonHandle, 10000u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_50000, BenchType::VersionChain, IsNotCommonHandle, 50000u); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_100000, BenchType::VersionChain, IsNotCommonHandle, 100000u); + +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_10, BenchType::DeltaIndex, IsNotCommonHandle, 10u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_100, BenchType::DeltaIndex, IsNotCommonHandle, 100u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_500, BenchType::DeltaIndex, IsNotCommonHandle, 500u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_1000, BenchType::DeltaIndex, IsNotCommonHandle, 1000u); + +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_1, BenchType::VersionChain, IsNotCommonHandle, 1u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_5, BenchType::VersionChain, IsNotCommonHandle, 5u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_10, BenchType::VersionChain, IsNotCommonHandle, 10u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_100, BenchType::VersionChain, IsNotCommonHandle, 100u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_500, BenchType::VersionChain, IsNotCommonHandle, 500u); +BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_1000, BenchType::VersionChain, IsNotCommonHandle, 1000u); + +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_10, BenchType::DeltaIndex, IsNotCommonHandle, 10u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_100, BenchType::DeltaIndex, IsNotCommonHandle, 100u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_500, BenchType::DeltaIndex, IsNotCommonHandle, 500u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_1000, BenchType::DeltaIndex, IsNotCommonHandle, 1000u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_5000, BenchType::DeltaIndex, IsNotCommonHandle, 5000u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_10000, BenchType::DeltaIndex, IsNotCommonHandle, 10000u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_50000, BenchType::DeltaIndex, IsNotCommonHandle, 50000u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_100000, BenchType::DeltaIndex, IsNotCommonHandle, 100000u); + +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_1, BenchType::VersionChain, IsNotCommonHandle, 1u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_5, BenchType::VersionChain, IsNotCommonHandle, 5u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_10, BenchType::VersionChain, IsNotCommonHandle, 10u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_100, BenchType::VersionChain, IsNotCommonHandle, 100u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_500, BenchType::VersionChain, IsNotCommonHandle, 500u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_1000, BenchType::VersionChain, IsNotCommonHandle, 1000u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_5000, BenchType::VersionChain, IsNotCommonHandle, 5000u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_10000, BenchType::VersionChain, IsNotCommonHandle, 10000u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_50000, BenchType::VersionChain, IsNotCommonHandle, 50000u); +BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_100000, BenchType::VersionChain, IsNotCommonHandle, 100000u); + +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_1, BenchType::None, IsNotCommonHandle, 1u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5, BenchType::None, IsNotCommonHandle, 5u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10, BenchType::None, IsNotCommonHandle, 10u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100, BenchType::None, IsNotCommonHandle, 100u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_500, BenchType::None, IsNotCommonHandle, 500u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_1000, BenchType::None, IsNotCommonHandle, 1000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5000, BenchType::None, IsNotCommonHandle, 5000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10000, BenchType::None, IsNotCommonHandle, 10000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_50000, BenchType::None, IsNotCommonHandle, 50000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100000, BenchType::None, IsNotCommonHandle, 100000u); +} // namespace diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp new file mode 100644 index 00000000000..af141441993 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp @@ -0,0 +1,31 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include + +namespace DB::DM::tests +{ +TEST(HandleIndexTest, Basic) +{ + [[maybe_unused]] VersionChain version_chain_int; +} + +TEST(HandleColumnView, Basic) +{ + [[maybe_unused]] HandleColumnView handle_column_view_int; + [[maybe_unused]] HandleColumnView handle_column_view_string; +} +} // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp b/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp deleted file mode 100644 index 7792664168a..00000000000 --- a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp +++ /dev/null @@ -1,98 +0,0 @@ -// Copyright 2024 PingCAP, Inc. -// -// 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 -#include -#include - -namespace DB::DM::bench -{ - -static void VectorIndexBuild(::benchmark::State & state) -try -{ - const auto & dataset = DatasetMnist::get(); - - auto train_data = dataset.buildDataTrainColumn(/* max_rows= */ 10000); - auto index_def = dataset.createIndexDef(tipb::VectorIndexKind::HNSW); - for (auto _ : state) - { - auto builder = std::make_unique(0, index_def); - builder->addBlock(*train_data, nullptr, []() { return true; }); - } -} -CATCH - -static void VectorIndexSearchTop10(::benchmark::State & state) -try -{ - const auto & dataset = DatasetMnist::get(); - - auto index_path = DB::tests::TiFlashTestEnv::getTemporaryPath("vector_search_top_10/vector_index.idx"); - VectorIndexBenchUtils::saveVectorIndex( // - index_path, - dataset, - /* max_rows= */ 10000); - - auto viewer = VectorIndexBenchUtils::viewVectorIndex(index_path, dataset); - - std::random_device rd; - std::mt19937 rng(rd()); - std::uniform_int_distribution dist(0, dataset.dataTestSize() - 1); - - for (auto _ : state) - { - auto test_index = dist(rng); - const auto & query_vector = DatasetMnist::get().dataTestAt(test_index); - auto keys = VectorIndexBenchUtils::queryTopK(viewer, query_vector, 10, state); - RUNTIME_CHECK(keys.size() == 10); - } -} -CATCH - -static void VectorIndexSearchTop100(::benchmark::State & state) -try -{ - const auto & dataset = DatasetMnist::get(); - - auto index_path = DB::tests::TiFlashTestEnv::getTemporaryPath("vector_search_top_10/vector_index.idx"); - VectorIndexBenchUtils::saveVectorIndex( // - index_path, - dataset, - /* max_rows= */ 10000); - - auto viewer = VectorIndexBenchUtils::viewVectorIndex(index_path, dataset); - - std::random_device rd; - std::mt19937 rng(rd()); - std::uniform_int_distribution dist(0, dataset.dataTestSize() - 1); - - for (auto _ : state) - { - auto test_index = dist(rng); - const auto & query_vector = DatasetMnist::get().dataTestAt(test_index); - auto keys = VectorIndexBenchUtils::queryTopK(viewer, query_vector, 100, state); - RUNTIME_CHECK(keys.size() == 100); - } -} -CATCH - -BENCHMARK(VectorIndexBuild); - -BENCHMARK(VectorIndexSearchTop10); - -BENCHMARK(VectorIndexSearchTop100); - -} // namespace DB::DM::bench diff --git a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h b/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h deleted file mode 100644 index 27e6c3cfa7e..00000000000 --- a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h +++ /dev/null @@ -1,177 +0,0 @@ -// Copyright 2024 PingCAP, Inc. -// -// 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 -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace DB::DM::bench -{ - -/** - * @brief Compatible with datasets on ANN-Benchmark: - * https://github.com/erikbern/ann-benchmarks - */ -class Dataset -{ -public: - explicit Dataset(std::string_view file_name) - { - auto dataset_directory = std::filesystem::path(__FILE__).parent_path().string() + "/bench_dataset"; - auto dataset_path = fmt::format("{}/{}", dataset_directory, file_name); - - if (!std::filesystem::exists(dataset_path)) - { - throw Exception(fmt::format( - "Benchmark cannot run because dataset file {} not found. See {}/README.md for setup instructions.", - dataset_path, - dataset_directory)); - } - - auto file = HighFive::File(dataset_path, HighFive::File::ReadOnly); - - auto dataset_train = file.getDataSet("train"); - dataset_train.read(data_train); - - auto dataset_test = file.getDataSet("test"); - dataset_test.read(data_test); - } - - virtual ~Dataset() = default; - - virtual UInt32 dimension() const = 0; - - virtual tipb::VectorDistanceMetric distanceMetric() const = 0; - -public: - MutableColumnPtr buildDataTrainColumn(std::optional max_rows = std::nullopt) const - { - auto vec_column = ColumnArray::create(ColumnFloat32::create()); - size_t rows = data_train.size(); - if (max_rows.has_value()) - rows = std::min(rows, *max_rows); - for (size_t i = 0; i < rows; ++i) - { - const auto & row = data_train[i]; - vec_column->insertData(reinterpret_cast(row.data()), row.size() * sizeof(Float32)); - } - return vec_column; - } - - size_t dataTestSize() const { return data_test.size(); } - - const std::vector & dataTestAt(size_t index) const { return data_test.at(index); } - - TiDB::VectorIndexDefinitionPtr createIndexDef(tipb::VectorIndexKind kind) const - { - return std::make_shared(TiDB::VectorIndexDefinition{ - .kind = kind, - .dimension = dimension(), - .distance_metric = distanceMetric(), - }); - } - -protected: - std::vector> data_train; - std::vector> data_test; -}; - -class DatasetMnist : public Dataset -{ -public: - DatasetMnist() - : Dataset("fashion-mnist-784-euclidean.hdf5") - { - RUNTIME_CHECK(data_train[0].size() == dimension()); - RUNTIME_CHECK(data_test[0].size() == dimension()); - } - - UInt32 dimension() const override { return 784; } - - tipb::VectorDistanceMetric distanceMetric() const override { return tipb::VectorDistanceMetric::L2; } - - static const DatasetMnist & get() - { - static DatasetMnist dataset; - return dataset; - } -}; - -class VectorIndexBenchUtils -{ -public: - template - static void saveVectorIndex( - std::string_view index_path, - const Dataset & dataset, - std::optional max_rows = std::nullopt) - { - Poco::File(index_path.data()).createDirectories(); - - auto train_data = dataset.buildDataTrainColumn(max_rows); - auto index_def = dataset.createIndexDef(Builder::kind()); - auto builder = std::make_unique(0, index_def); - builder->addBlock(*train_data, nullptr, []() { return true; }); - builder->saveToFile(index_path); - } - - template - static auto viewVectorIndex(std::string_view index_path, const Dataset & dataset) - { - auto index_view_props = dtpb::VectorIndexFileProps(); - index_view_props.set_index_kind(tipb::VectorIndexKind_Name(Viewer::kind())); - index_view_props.set_dimensions(dataset.dimension()); - index_view_props.set_distance_metric(tipb::VectorDistanceMetric_Name(dataset.distanceMetric())); - return Viewer::view(index_view_props, index_path); - } - - static auto queryTopK( - VectorIndexViewerPtr viewer, - const std::vector & ref, - UInt32 top_k, - std::optional> state = std::nullopt) - { - if (state.has_value()) - state->get().PauseTiming(); - - auto ann_query_info = std::make_shared(); - auto distance_metric = tipb::VectorDistanceMetric::INVALID_DISTANCE_METRIC; - tipb::VectorDistanceMetric_Parse(viewer->file_props.distance_metric(), &distance_metric); - ann_query_info->set_distance_metric(distance_metric); - ann_query_info->set_top_k(top_k); - ann_query_info->set_ref_vec_f32(DB::DM::tests::VectorIndexTestUtils::encodeVectorFloat32(ref)); - - auto filter = BitmapFilterView::createWithFilter(viewer->size(), true); - - if (state.has_value()) - state->get().ResumeTiming(); - - return viewer->search(ann_query_info, filter); - } -}; - - -} // namespace DB::DM::bench 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 f499edde1f5..67f7fbcd803 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 @@ -951,7 +951,8 @@ CATCH TEST_P(DeltaMergeStoreRWTest, WriteMultipleBlock) try { - const size_t num_write_rows = 32; + constexpr size_t num_write_rows = 32; + constexpr bool clear_data_in_range = true; // Test write multi blocks without overlap { @@ -979,7 +980,7 @@ try auto file_ids = file_ids1; file_ids.insert(file_ids.cend(), file_ids2.begin(), file_ids2.end()); file_ids.insert(file_ids.cend(), file_ids3.begin(), file_ids3.end()); - store->ingestFiles(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, !clear_data_in_range); break; } case TestMode::PageStorageV2_MemoryAndDisk: @@ -991,7 +992,7 @@ try auto range = range1.merge(range3); auto file_ids = file_ids1; file_ids.insert(file_ids.cend(), file_ids3.begin(), file_ids3.end()); - store->ingestFiles(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, !clear_data_in_range); store->write(*db_context, db_context->getSettingsRef(), block2); break; @@ -1029,7 +1030,7 @@ try // Test write multi blocks with overlap { - UInt64 tso1 = 1; + UInt64 tso1 = 3; // ts of the same key should incre... UInt64 tso2 = 100; Block block1 = DMTestEnv::prepareSimpleWriteBlock(0, 1 * num_write_rows, false, tso1); Block block2 = DMTestEnv::prepareSimpleWriteBlock(1 * num_write_rows, 2 * num_write_rows, false, tso1); @@ -1051,11 +1052,11 @@ try { auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); auto [range1, file_ids1] = genDMFile(*dm_context, block1); - store->ingestFiles(dm_context, range1, {file_ids1}, false); + store->ingestFiles(dm_context, range1, {file_ids1}, clear_data_in_range); auto [range2, file_ids2] = genDMFile(*dm_context, block2); - store->ingestFiles(dm_context, range2, {file_ids2}, false); + store->ingestFiles(dm_context, range2, {file_ids2}, clear_data_in_range); auto [range3, file_ids3] = genDMFile(*dm_context, block3); - store->ingestFiles(dm_context, range3, {file_ids3}, false); + store->ingestFiles(dm_context, range3, {file_ids3}, clear_data_in_range); break; } case TestMode::PageStorageV2_MemoryAndDisk: @@ -1065,9 +1066,9 @@ try auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); auto [range1, file_ids1] = genDMFile(*dm_context, block1); - store->ingestFiles(dm_context, range1, {file_ids1}, false); + store->ingestFiles(dm_context, range1, {file_ids1}, clear_data_in_range); auto [range3, file_ids3] = genDMFile(*dm_context, block3); - store->ingestFiles(dm_context, range3, {file_ids3}, false); + store->ingestFiles(dm_context, range3, {file_ids3}, clear_data_in_range); break; } } @@ -1109,8 +1110,8 @@ try db_context->getSettingsRef(), columns, {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, - /* num_streams= */ 1, - /* start_ts= */ static_cast(1), + /* num_streams= */ 2, + /* start_ts= */ static_cast(2), EMPTY_FILTER, std::vector{}, 0, @@ -1122,7 +1123,7 @@ try in, Strings({DMTestEnv::pk_name}), createColumns({ - createColumn(createNumbers(0, 2 * num_write_rows)), + createColumn(createNumbers(64, 96)), })); } } @@ -3996,6 +3997,16 @@ CATCH void DeltaMergeStoreRWTest::dupHandleVersionAndDeltaIndexAdvancedThanSnapshot() { + // Always use delta index in this case. + auto & global_settings = db_context->getGlobalContext().getSettingsRef(); + bool enable_version_chain = global_settings.dt_enable_version_chain; + if (enable_version_chain) + global_settings.set("dt_enable_version_chain", "false"); + SCOPE_EXIT({ + if (enable_version_chain) + global_settings.set("dt_enable_version_chain", "true"); + }); + auto table_column_defines = DMTestEnv::getDefaultColumns(); store = reload(table_column_defines); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp index 1aac3acc6b8..154559315c0 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp @@ -1028,6 +1028,11 @@ try } CATCH +namespace +{ +constexpr bool use_version_chain = true; +} + TEST_F(SegmentTest, ColumnFileBigRangeGreaterThanSegment) try { @@ -1091,7 +1096,8 @@ try real_ranges, {}, std::numeric_limits::max(), - DEFAULT_BLOCK_SIZE); + DEFAULT_BLOCK_SIZE, + use_version_chain); // the bitmap only contains the overlapped packs of ColumnFileBig. So only 60 here. ASSERT_EQ(bitmap_filter->size(), 60); ASSERT_EQ(bitmap_filter->toDebugString(), "000000000011111111111111111111111111111111111111110000000000"); @@ -1140,6 +1146,29 @@ try // write range [80, 90) Block block2 = DMTestEnv::prepareSimpleWriteBlock(80, 90, false); segment->write(dmContext(), std::move(block2)); + + // test built bitmap filter + auto segment_snap = segment->createSnapshot(dmContext(), false, CurrentMetrics::DT_SnapshotOfRead); + auto read_ranges = {RowKeyRange::newAll(false, 1)}; + auto real_ranges = segment->shrinkRowKeyRanges(read_ranges); + auto bitmap_filter1 = segment->buildBitmapFilter( // + dmContext(), + segment_snap, + real_ranges, + EMPTY_RS_OPERATOR, + std::numeric_limits::max(), + DEFAULT_BLOCK_SIZE, + !use_version_chain); + auto bitmap_filter2 = segment->buildBitmapFilter( // + dmContext(), + segment_snap, + real_ranges, + EMPTY_RS_OPERATOR, + std::numeric_limits::max(), + DEFAULT_BLOCK_SIZE, + use_version_chain); + + ASSERT_EQ(bitmap_filter1->toDebugString(), bitmap_filter2->toDebugString()); } { // test read data with delete-range and new writes diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp index 24ec7be1bba..edc58b04c13 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -179,6 +180,34 @@ class SegmentBitmapFilterTest : public SegmentTestBasic auto expected_handle = genSequence(test_case.expected_handle); ASSERT_TRUE(sequenceEqual(expected_handle.data(), handle->data(), test_case.expected_size)); } + + verifyBitmapFilter(SEG_ID); + } + + inline static constexpr bool use_version_chain = true; + void verifyBitmapFilter(const PageIdU64 seg_id, const UInt64 read_ts = std::numeric_limits::max()) + { + auto [seg, snap] = getSegmentForRead(seg_id); + + auto bitmap_filter1 = seg->buildBitmapFilter( + *dm_context, + snap, + {seg->getRowKeyRange()}, + nullptr, + read_ts, + DEFAULT_BLOCK_SIZE, + !use_version_chain); + + auto bitmap_filter2 = seg->buildBitmapFilter( + *dm_context, + snap, + {seg->getRowKeyRange()}, + nullptr, + read_ts, + DEFAULT_BLOCK_SIZE, + use_version_chain); + + ASSERT_EQ(bitmap_filter1->toDebugString(), bitmap_filter2->toDebugString()); } auto loadPackFilterResults(const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) @@ -276,7 +305,7 @@ TEST_F(SegmentBitmapFilterTest, Big) try { runTestCase(TestCase{ - "d_tiny:[100, 500)|d_big:[250, 1000)|d_mem:[240, 290)", + "d_tiny:[100, 500)|d_dr:[250, 1000)|d_big:[250, 1000)|d_mem:[240, 290)", 900, "[0, 140)|[1150, 1200)|[440, 1150)", "[100, 1000)"}); @@ -348,6 +377,9 @@ try ASSERT_TRUE(new_seg_id.has_value()); ASSERT_TRUE(areSegmentsSharingStable({SEG_ID, *new_seg_id})); + verifyBitmapFilter(SEG_ID); + verifyBitmapFilter(*new_seg_id); + auto left_handle = getSegmentHandle(SEG_ID, {}); const auto * left_h = toColumnVectorDataPtr(left_handle); auto expected_left_handle = genSequence("[0, 128)|[200, 255)|[256, 305)|[310, 512)"); @@ -393,6 +425,7 @@ TEST_F(SegmentBitmapFilterTest, CleanStable) std::string expect_result; expect_result.append(std::string(25000, '1')); ASSERT_EQ(bitmap_filter->toDebugString(), expect_result); + verifyBitmapFilter(SEG_ID); } TEST_F(SegmentBitmapFilterTest, NotCleanStable) @@ -420,6 +453,7 @@ TEST_F(SegmentBitmapFilterTest, NotCleanStable) } expect_result.append(std::string(5000, '1')); ASSERT_EQ(bitmap_filter->toDebugString(), expect_result); + verifyBitmapFilter(SEG_ID); } { // Stale read @@ -440,6 +474,7 @@ TEST_F(SegmentBitmapFilterTest, NotCleanStable) } expect_result.append(std::string(5000, '0')); ASSERT_EQ(bitmap_filter->toDebugString(), expect_result); + verifyBitmapFilter(SEG_ID, 1); } } @@ -466,6 +501,8 @@ TEST_F(SegmentBitmapFilterTest, StableRange) expect_result.append(std::string(10000, '0')); expect_result.append(std::string(40000, '1')); ASSERT_EQ(bitmap_filter->toDebugString(), expect_result); + + verifyBitmapFilter(SEG_ID); } TEST_F(SegmentBitmapFilterTest, StableLogicalSplit) @@ -483,6 +520,9 @@ try ASSERT_TRUE(new_seg_id.has_value()); ASSERT_TRUE(areSegmentsSharingStable({SEG_ID, *new_seg_id})); + verifyBitmapFilter(SEG_ID); + verifyBitmapFilter(*new_seg_id); + auto left_handle = getSegmentHandle(SEG_ID, {}); const auto * left_h = toColumnVectorDataPtr(left_handle); auto expected_left_handle = genSequence("[0, 25000)"); @@ -528,7 +568,8 @@ try {seg->getRowKeyRange()}, loadPackFilterResults(snap, {seg->getRowKeyRange()}), std::numeric_limits::max(), - DEFAULT_BLOCK_SIZE); + DEFAULT_BLOCK_SIZE, + use_version_chain); ASSERT_EQ(bitmap_filter->size(), 30); ASSERT_EQ(bitmap_filter->count(), 20); // `count()` returns the number of bit has been set. ASSERT_EQ(bitmap_filter->toDebugString(), "000001111111111111111111100000"); @@ -558,7 +599,8 @@ try {seg->getRowKeyRange()}, loadPackFilterResults(snap, {seg->getRowKeyRange()}), std::numeric_limits::max(), - DEFAULT_BLOCK_SIZE); + DEFAULT_BLOCK_SIZE, + use_version_chain); ASSERT_EQ(bitmap_filter->size(), 750); ASSERT_EQ(bitmap_filter->count(), 20); // `count()` returns the number of bit has been set. ASSERT_EQ( diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_read_task.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_read_task.cpp index 964482c1702..d72dcf9915f 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_read_task.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_read_task.cpp @@ -90,6 +90,15 @@ class SegmentReadTaskTest : public SegmentTestBasic // hack to change the "immutable" delta-index on delta-snapshot for testing (*const_cast(&first_snap->delta->getSharedDeltaIndex())) = broken_delta_index; + // Always use delta index in this case. + auto & global_settings = db_context->getGlobalContext().getSettingsRef(); + bool enable_version_chain = global_settings.dt_enable_version_chain; + if (enable_version_chain) + global_settings.set("dt_enable_version_chain", "false"); + SCOPE_EXIT({ + if (enable_version_chain) + global_settings.set("dt_enable_version_chain", "true"); + }); auto task = std::make_shared( first, first_snap, From 9a74dedad7dc72cebba3b1566b1c268d3d36e106 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 8 Jan 2025 17:17:20 +0800 Subject: [PATCH 02/48] ci --- .../DeltaMerge/VersionChain/DMFileHandleIndex.h | 13 +++++-------- .../DeltaMerge/VersionChain/HandleColumnView.h | 6 ++++++ 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h index aac51b4073e..688433a54f4 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB::DM { @@ -109,14 +110,10 @@ class DMFileHandleIndex std::optional getBaseVersion(HandleView h, UInt32 clipped_pack_id) { loadHandleIfNotLoaded(); - const auto & handle_col = clipped_handle_packs[clipped_pack_id]; - const auto * handles = toColumnVectorDataPtr(handle_col); - RUNTIME_CHECK_MSG(handles != nullptr, "TODO: support common handle"); - auto itr = std::lower_bound(handles->begin(), handles->end(), h); - if (itr != handles->end() && *itr == h) - { - return itr - handles->begin() + clipped_pack_offsets[clipped_pack_id]; - } + HandleColumnView handle_col(*clipped_handle_packs[clipped_pack_id]); + auto itr = std::lower_bound(handle_col.begin(), handle_col.end(), h); + if (itr != handle_col.end() && *itr == h) + return itr - handle_col.begin() + clipped_pack_offsets[clipped_pack_id]; return {}; } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h b/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h index ccf5ec64e31..14451008bb3 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h @@ -74,6 +74,11 @@ class HandleColumnView return *this; } + size_t operator-(const Iterator & other) const + { + return pos - other.pos; + } + Iterator & operator++() { ++pos; @@ -86,6 +91,7 @@ class HandleColumnView return *this; } + // Postfix increment/decrement is not supported Iterator & operator++(int) = delete; Iterator & operator--(int) = delete; From d83d6c24c370fad0f70262df8dcbba7bc0c6ad00 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 11:58:00 +0800 Subject: [PATCH 03/48] ci --- .../DeltaMerge/BitmapFilter/BitmapFilter.h | 2 +- .../DeltaMerge/Delta/ColumnFileFlushTask.cpp | 2 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 8 ++- .../VersionChain/BuildBitmapFilter.cpp | 40 ++----------- .../VersionChain/BuildBitmapFilter.h | 2 +- .../DeltaMerge/VersionChain/Common.cpp | 31 +++------- .../Storages/DeltaMerge/VersionChain/Common.h | 11 +--- .../VersionChain/DMFileHandleIndex.h | 57 +++++++------------ .../DeltaMerge/VersionChain/DeletedFilter.cpp | 16 ++---- .../DeltaMerge/VersionChain/DeletedFilter.h | 5 +- .../VersionChain/HandleColumnView.h | 10 ++-- .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 34 ++++------- .../DeltaMerge/VersionChain/RowKeyFilter.h | 4 +- .../DeltaMerge/VersionChain/VersionChain.cpp | 12 ++-- .../DeltaMerge/VersionChain/VersionChain.h | 7 ++- .../DeltaMerge/VersionChain/VersionFilter.cpp | 23 ++++---- .../DeltaMerge/VersionChain/VersionFilter.h | 4 +- .../tests/bench_version_chain.cpp | 8 +-- .../tests/gtest_handle_pack_index.cpp | 10 +++- .../DeltaMerge/tests/gtest_dm_segment.cpp | 17 +++++- .../DeltaMerge/tests/gtest_segment_bitmap.cpp | 28 ++++----- 21 files changed, 134 insertions(+), 197 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h index 06e580aefda..d3e1f3bb531 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h @@ -45,7 +45,7 @@ class BitmapFilter size_t count() const; inline size_t size() const { return filter.size(); } - std::vector & getFilter() { return filter; } + IColumn::Filter & getFilter() { return filter; } friend class BitmapFilterView; diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp b/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp index a4fa8f7e104..155528d5b1f 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp @@ -43,7 +43,7 @@ DeltaIndex::Updates ColumnFileFlushTask::prepare(WriteBatches & wbs) { if (!task.block_data) continue; -/* + /* IColumn::Permutation perm; task.sorted = sortBlockByPk(getExtraHandleColumnDefine(context.is_common_handle), task.block_data, perm); if (task.sorted) diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index e568fbe0d7c..e6d8746be02 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -3087,7 +3087,13 @@ BitmapFilterPtr Segment::buildBitmapFilter( if (use_version_chain) { - return buildBitmapFilter(dm_context, *segment_snap, read_ranges, filter, start_ts, version_chain); + return buildBitmapFilter( + dm_context, + *segment_snap, + read_ranges, + pack_filter_results, + start_ts, + version_chain); } if (dm_context.read_stable_only || (segment_snap->delta->getRows() == 0 && segment_snap->delta->getDeletes() == 0)) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 1ac3a64d698..23669d0b5ed 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -23,45 +23,12 @@ namespace DB::DM { -RSResults getDMFileRSFilterResults( - const DMContext & dm_context, - const DMFilePtr & dmfile, - const RSOperatorPtr & rs_operator) -{ - if (!rs_operator) - return RSResults(dmfile->getPacks(), RSResult::Some); - - auto pack_filter = DMFilePackFilter::loadFrom( - dmfile, - dm_context.global_context.getMinMaxIndexCache(), - true, - {}, // read_ranges - rs_operator, - {}, - dm_context.global_context.getFileProvider(), - dm_context.getReadLimiter(), - dm_context.scan_context, - dm_context.tracing_id, - ReadTag::MVCC); - return pack_filter.getPackResConst(); -} - -RSResults getStableRSFilterResults( - const DMContext & dm_context, - const StableValueSpace::Snapshot & stable, - const RSOperatorPtr & rs_operator) -{ - const auto & dmfiles = stable.getDMFiles(); - RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); - return getDMFileRSFilterResults(dm_context, dmfiles[0], rs_operator); -} - template BitmapFilterPtr buildBitmapFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, - const RSOperatorPtr & rs_operator, + const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, VersionChain & version_chain) { @@ -76,7 +43,8 @@ BitmapFilterPtr buildBitmapFilter( // TODO: make these functions return filter out rows. // TODO: send the pack res to buildVersionFilter and buildDeletedFilter to skip some packs. - auto stable_pack_res = getStableRSFilterResults(dm_context, stable, rs_operator); + RUNTIME_CHECK(pack_filter_results.size() == 1, pack_filter_results.size()); + const auto stable_pack_res = pack_filter_results.front()->getPackRes(); buildRowKeyFilter(dm_context, snapshot, read_ranges, stable_pack_res, filter); buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); buildDeletedFilter(dm_context, snapshot, filter); @@ -89,7 +57,7 @@ template BitmapFilterPtr buildBitmapFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, - const RSOperatorPtr & rs_operator, + const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, VersionChain & version_chain); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h index c652b7ce2c1..f4b0b2d1486 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h @@ -32,7 +32,7 @@ BitmapFilterPtr buildBitmapFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, - const RSOperatorPtr & rs_operator, + const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, VersionChain & version_chain); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp index 86c822a1ef4..bd121d92d14 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp @@ -19,30 +19,20 @@ namespace DB::DM { -RSResults getRSResultsByRanges( - const Context & global_context, - const ScanContextPtr & scan_context, - const String & tracing_id, - const DMFilePtr & dmfile, - const RowKeyRanges & ranges) +RSResults getRSResultsByRanges(const DMContext & dm_context, const DMFilePtr & dmfile, const RowKeyRanges & ranges) { if (ranges.empty()) return RSResults(dmfile->getPacks(), RSResult::All); auto pack_filter = DMFilePackFilter::loadFrom( + dm_context, dmfile, - global_context.getMinMaxIndexCache(), - true, + /*set cache*/ true, ranges, EMPTY_RS_OPERATOR, - {}, - global_context.getFileProvider(), - global_context.getReadLimiter(), - scan_context, - tracing_id, - ReadTag::MVCC); + /*read_packs*/ {}); - return pack_filter.getHandleRes(); + return pack_filter->getHandleRes(); } namespace @@ -72,21 +62,14 @@ std::pair clipRSResults(const RSResults & rs_results) } // namespace std::pair getClippedRSResultsByRanges( - const Context & global_context, - const ScanContextPtr & scan_context, - const String & tracing_id, + const DMContext & dm_context, const DMFilePtr & dmfile, const std::optional & segment_range) { if (!segment_range) return std::make_pair(RSResults(dmfile->getPacks(), RSResult::All), 0); - const auto handle_res = getRSResultsByRanges( - global_context, - scan_context, - tracing_id, - dmfile, - {*segment_range}); + const auto handle_res = getRSResultsByRanges(dm_context, dmfile, {*segment_range}); return clipRSResults(handle_res); } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h index 6026c848626..3b0f03029f5 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h @@ -85,17 +85,10 @@ std::pair convertRowKeyRange(const RowKeyRange & range) static_assert(false, "TODO: support common handle"); } -RSResults getRSResultsByRanges( - const Context & global_context, - const ScanContextPtr & scan_context, - const String & tracing_id, - const DMFilePtr & dmfile, - const RowKeyRanges & ranges); +RSResults getRSResultsByRanges(const DMContext & dm_context, const DMFilePtr & dmfile, const RowKeyRanges & ranges); std::pair getClippedRSResultsByRanges( - const Context & global_context, - const ScanContextPtr & scan_context, - const String & tracing_id, + const DMContext & dm_context, const DMFilePtr & dmfile, const std::optional & segment_range); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h index 688433a54f4..b4a1b041be1 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h @@ -21,7 +21,7 @@ #include #include #include - +#include namespace DB::DM { @@ -30,16 +30,15 @@ class DMFileHandleIndex { public: DMFileHandleIndex( - const Context & global_context_, + const DMContext & dm_context_, const DMFilePtr & dmfile_, const RowID start_row_id_, std::optional rowkey_range_) - : global_context(global_context_) - , dmfile(dmfile_) + : dmfile(dmfile_) , start_row_id(start_row_id_) , rowkey_range(std::move(rowkey_range_)) - , clipped_pack_range(getPackRange()) - , clipped_pack_index(loadPackIndex()) + , clipped_pack_range(getPackRange(dm_context_)) + , clipped_pack_index(loadPackIndex(dm_context_)) , clipped_pack_offsets(loadPackOffsets()) , clipped_handle_packs(clipped_pack_range.count()) , clipped_need_read_packs(std::vector(clipped_pack_range.count(), 1)) // read all packs by default @@ -55,12 +54,12 @@ class DMFileHandleIndex } template - std::optional getBaseVersion(HandleView h) + std::optional getBaseVersion(const DMContext & dm_context, HandleView h) { auto clipped_pack_id = getClippedPackId(h); if (!clipped_pack_id) return {}; - auto row_id = getBaseVersion(h, *clipped_pack_id); + auto row_id = getBaseVersion(dm_context, h, *clipped_pack_id); if (!row_id) return {}; return start_row_id + *row_id; @@ -107,9 +106,9 @@ class DMFileHandleIndex } template - std::optional getBaseVersion(HandleView h, UInt32 clipped_pack_id) + std::optional getBaseVersion(const DMContext & dm_context, HandleView h, UInt32 clipped_pack_id) { - loadHandleIfNotLoaded(); + loadHandleIfNotLoaded(dm_context); HandleColumnView handle_col(*clipped_handle_packs[clipped_pack_id]); auto itr = std::lower_bound(handle_col.begin(), handle_col.end(), h); if (itr != handle_col.end() && *itr == h) @@ -117,9 +116,9 @@ class DMFileHandleIndex return {}; } - std::vector loadPackIndex() + std::vector loadPackIndex(const DMContext & dm_context) { - auto max_values = loadPackMaxValue(global_context, *dmfile, EXTRA_HANDLE_COLUMN_ID); + auto max_values = loadPackMaxValue(dm_context.global_context, *dmfile, MutSup::extra_handle_id); return std::vector( max_values.begin() + clipped_pack_range.start_pack_id, max_values.begin() + clipped_pack_range.end_pack_id); @@ -139,7 +138,7 @@ class DMFileHandleIndex static bool isCommonHandle() { return std::is_same_v; } - void loadHandleIfNotLoaded() + void loadHandleIfNotLoaded(const DMContext & dm_context) { if (likely(!clipped_need_read_packs)) return; @@ -152,19 +151,13 @@ class DMFileHandleIndex read_pack_ids->insert(i + clipped_pack_range.start_pack_id); } - auto scan_context = std::make_shared(); auto pack_filter = DMFilePackFilter::loadFrom( + dm_context, dmfile, - global_context.getMinMaxIndexCache(), true, //set_cache_if_miss {}, // rowkey_ranges, empty means all nullptr, // RSOperatorPtr - read_pack_ids, - global_context.getFileProvider(), - global_context.getReadLimiter(), - scan_context, - __FILE__, - ReadTag::MVCC); + read_pack_ids); DMFileReader reader( dmfile, @@ -175,17 +168,17 @@ class DMFileHandleIndex /*is_fast_scan*/ false, /*max_data_version*/ std::numeric_limits::max(), std::move(pack_filter), - global_context.getMarkCache(), + dm_context.global_context.getMarkCache(), /*enable_column_cache*/ false, /*column_cache*/ nullptr, - global_context.getSettingsRef().max_read_buffer_size, - global_context.getFileProvider(), - global_context.getReadLimiter(), + dm_context.global_context.getSettingsRef().max_read_buffer_size, + dm_context.global_context.getFileProvider(), + dm_context.global_context.getReadLimiter(), DEFAULT_MERGE_BLOCK_SIZE, /*read_one_pack_every_time*/ true, - "DMFileHandleIndex", + dm_context.tracing_id, /*max_sharing_column_bytes_for_all*/ false, - scan_context, + dm_context.scan_context, ReadTag::MVCC); @@ -206,23 +199,17 @@ class DMFileHandleIndex UInt32 count() const { return end_pack_id - start_pack_id; } }; - PackRange getPackRange() + PackRange getPackRange(const DMContext & dm_context) { if (!rowkey_range) return PackRange{.start_pack_id = 0, .end_pack_id = static_cast(dmfile->getPacks())}; - const auto [handle_res, start_pack_id] = getClippedRSResultsByRanges( - global_context, - std::make_shared(), - "DMFileHandleIndex", - dmfile, - rowkey_range); + const auto [handle_res, start_pack_id] = getClippedRSResultsByRanges(dm_context, dmfile, rowkey_range); return PackRange{ .start_pack_id = start_pack_id, .end_pack_id = start_pack_id + static_cast(handle_res.size())}; } - const Context & global_context; const DMFilePtr dmfile; const RowID start_row_id; const std::optional rowkey_range; // Range of ColumnFileBig or nullopt for Stable DMFile diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp index f000cbc7680..4f0520945ac 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp @@ -28,7 +28,7 @@ UInt32 buildDeletedFilterBlock( const IColumnFileDataProviderPtr & data_provider, const ColumnFile & cf, const UInt32 start_row_id, - std::vector & filter) + IColumn::Filter & filter) { assert(cf.isInMemoryFile() || cf.isTinyFile()); auto cf_reader = cf.getReader(dm_context, data_provider, getTagColumnDefinesPtr(), ReadTag::MVCC); @@ -48,13 +48,9 @@ UInt32 buildDeletedFilterDMFile( const DMFilePtr & dmfile, const std::optional & segment_range, const ssize_t start_row_id, - std::vector & filter) + IColumn::Filter & filter) { - auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges( - dm_context.global_context, - dm_context.scan_context, - dm_context.tracing_id, - dmfile, segment_range); + auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges(dm_context, dmfile, segment_range); if (valid_handle_res.empty()) return 0; @@ -110,7 +106,7 @@ UInt32 buildDeletedFilterColumnFileBig( const DMContext & dm_context, const ColumnFileBig & cf_big, const ssize_t start_row_id, - std::vector & filter) + IColumn::Filter & filter) { return buildDeletedFilterDMFile(dm_context, cf_big.getFile(), cf_big.getRange(), start_row_id, filter); } @@ -118,14 +114,14 @@ UInt32 buildDeletedFilterColumnFileBig( UInt32 buildDeletedFilterStable( const DMContext & dm_context, const StableValueSpace::Snapshot & stable, - std::vector & filter) + IColumn::Filter & filter) { const auto & dmfiles = stable.getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); return buildDeletedFilterDMFile(dm_context, dmfiles[0], std::nullopt, 0, filter); } -void buildDeletedFilter(const DMContext & dm_context, const SegmentSnapshot & snapshot, std::vector & filter) +void buildDeletedFilter(const DMContext & dm_context, const SegmentSnapshot & snapshot, IColumn::Filter & filter) { const auto & delta = *(snapshot.delta); const auto & stable = *(snapshot.stable); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h index 84f38d980f1..b02ad50e557 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h @@ -14,13 +14,12 @@ #pragma once +#include #include -#include - namespace DB::DM { struct DMContext; struct SegmentSnapshot; -void buildDeletedFilter(const DMContext & dm_context, const SegmentSnapshot & snapshot, std::vector & filter); +void buildDeletedFilter(const DMContext & dm_context, const SegmentSnapshot & snapshot, IColumn::Filter & filter); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h b/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h index 14451008bb3..9ad47fba9d2 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h @@ -18,7 +18,8 @@ namespace DB::DM { template class HandleColumnView -{}; +{ +}; template <> class HandleColumnView @@ -59,7 +60,7 @@ class HandleColumnView assert(offsets[-1] == 0); const auto off = offsets[pos]; const auto size = offsets[pos] - offsets[pos - 1] - 1; - return std::string_view(static_cast(&chars[off]), size); + return std::string_view(reinterpret_cast(&chars[off]), size); } Iterator & operator+(size_t n) @@ -74,10 +75,7 @@ class HandleColumnView return *this; } - size_t operator-(const Iterator & other) const - { - return pos - other.pos; - } + size_t operator-(const Iterator & other) const { return pos - other.pos; } Iterator & operator++() { diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index f4c1a29e206..6695a226761 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -32,7 +32,7 @@ UInt32 buildRowKeyFilterVector( const RowKeyRanges & delete_ranges, const RowKeyRanges & read_ranges, const UInt32 start_row_id, - std::vector & filter) + IColumn::Filter & filter) { for (UInt32 i = 0; i < handles.size(); ++i) { @@ -56,7 +56,7 @@ UInt32 buildRowKeyFilterBlock( const RowKeyRanges & delete_ranges, const RowKeyRanges & read_ranges, const UInt32 start_row_id, - std::vector & filter) + IColumn::Filter & filter) { assert(cf.isInMemoryFile() || cf.isTinyFile()); @@ -81,13 +81,9 @@ UInt32 buildRowKeyFilterDMFile( const RowKeyRanges & read_ranges, const RSResults * stable_pack_res, const UInt32 start_row_id, - std::vector & filter) + IColumn::Filter & filter) { - auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges( - dm_context.global_context, - dm_context.scan_context, - dm_context.tracing_id, - dmfile, segment_range); + auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges(dm_context, dmfile, segment_range); if (valid_handle_res.empty()) return 0; @@ -103,23 +99,13 @@ UInt32 buildRowKeyFilterDMFile( valid_handle_res[i] = RSResult::None; } - const auto read_ranges_handle_res = getRSResultsByRanges( - dm_context.global_context, - dm_context.scan_context, - dm_context.tracing_id, - dmfile, - read_ranges); + const auto read_ranges_handle_res = getRSResultsByRanges(dm_context, dmfile, read_ranges); for (UInt32 i = 0; i < valid_handle_res.size(); ++i) valid_handle_res[i] = valid_handle_res[i] && read_ranges_handle_res[valid_start_pack_id + i]; if (!delete_ranges.empty()) { - const auto delete_ranges_handle_res = getRSResultsByRanges( - dm_context.global_context, - dm_context.scan_context, - dm_context.tracing_id, - dmfile, - delete_ranges); + const auto delete_ranges_handle_res = getRSResultsByRanges(dm_context, dmfile, delete_ranges); for (UInt32 i = 0; i < valid_handle_res.size(); ++i) valid_handle_res[i] = valid_handle_res[i] && !delete_ranges_handle_res[valid_start_pack_id + i]; } @@ -180,7 +166,7 @@ UInt32 buildRowKeyFilterColumnFileBig( const RowKeyRanges & delete_ranges, const RowKeyRanges & read_ranges, const UInt32 start_row_id, - std::vector & filter) + IColumn::Filter & filter) { if (cf_big.getRows() == 0) return 0; @@ -202,7 +188,7 @@ UInt32 buildRowKeyFilterStable( const RowKeyRanges & delete_ranges, const RowKeyRanges & read_ranges, const RSResults & stable_pack_res, - std::vector & filter) + IColumn::Filter & filter) { const auto & dmfiles = stable.getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); @@ -232,7 +218,7 @@ void buildRowKeyFilter( const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, const RSResults & stable_pack_res, - std::vector & filter) + IColumn::Filter & filter) { const auto & delta = *(snapshot.delta); const auto & stable = *(snapshot.stable); @@ -300,7 +286,7 @@ template void buildRowKeyFilter( const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, const RSResults & stable_pack_res, - std::vector & filter); + IColumn::Filter & filter); // TODO: String } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h index 05eaac1a679..2e440758fe3 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h @@ -14,8 +14,8 @@ #pragma once +#include #include - namespace DB::DM { struct DMContext; @@ -29,5 +29,5 @@ void buildRowKeyFilter( const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, const RSResults & stable_pack_res, - std::vector & filter); + IColumn::Filter & filter); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index 83a302b5c03..9ad09b2a111 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -31,7 +31,7 @@ std::shared_ptr> VersionChain::replaySnapshot( const auto & dmfiles = snapshot.stable->getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); dmfile_or_delete_range_list->push_back( - DMFileHandleIndex{dm_context.global_context, dmfiles[0], /*start_row_id*/ 0, std::nullopt}); + DMFileHandleIndex{dm_context, dmfiles[0], /*start_row_id*/ 0, std::nullopt}); } const auto & stable = *(snapshot.stable); @@ -143,7 +143,7 @@ UInt32 VersionChain::replayBlock( base_versions->push_back(itr->second); continue; } - if (auto row_id = findBaseVersionFromDMFileOrDeleteRangeList(h); row_id) + if (auto row_id = findBaseVersionFromDMFileOrDeleteRangeList(dm_context, h); row_id) { base_versions->push_back(*row_id); continue; @@ -166,7 +166,7 @@ UInt32 VersionChain::replayColumnFileBig( base_versions->insert(base_versions->end(), rows, NotExistRowID); dmfile_or_delete_range_list->push_back( - DMFileHandleIndex{dm_context.global_context, cf_big.getFile(), start_row_id, cf_big.getRange()}); + DMFileHandleIndex{dm_context, cf_big.getFile(), start_row_id, cf_big.getRange()}); return rows; } @@ -187,14 +187,16 @@ UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_ } template -std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRangeList(Handle h) +std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRangeList( + const DMContext & dm_context, + Handle h) { for (auto itr = dmfile_or_delete_range_list->rbegin(); itr != dmfile_or_delete_range_list->rend(); ++itr) { auto & dmfile_or_delete_range = *itr; if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) { - if (auto row_id = dmfile_index->getBaseVersion(h); row_id) + if (auto row_id = dmfile_index->getBaseVersion(dm_context, h); row_id) return row_id; } else if (auto * delete_range = std::get_if(&dmfile_or_delete_range); delete_range) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index 2779b5386b7..ea2fc3960e6 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -14,11 +14,12 @@ #pragma once -#include #include #include #include +#include + namespace DB::DM { @@ -69,7 +70,9 @@ class VersionChain const UInt32 stable_rows); [[nodiscard]] UInt32 replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range); - [[nodiscard]] std::optional findBaseVersionFromDMFileOrDeleteRangeList(Handle h); + [[nodiscard]] std::optional findBaseVersionFromDMFileOrDeleteRangeList( + const DMContext & dm_context, + Handle h); void calculateReadPacks(const std::span handles); void cleanHandleColumn(); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp index f95847e2bc4..f6048f86b76 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp @@ -30,7 +30,7 @@ namespace DB::DM const std::vector & base_ver_snap, const UInt32 stable_rows, const UInt32 start_row_id, - std::vector & filter) + IColumn::Filter & filter) { assert(cf.isInMemoryFile() || cf.isTinyFile()); auto cf_reader = cf.getReader(dm_context, data_provider, getVersionColumnDefinesPtr(), ReadTag::MVCC); @@ -78,17 +78,13 @@ namespace DB::DM const std::optional & segment_range, const UInt64 read_ts, const ssize_t start_row_id, - std::vector & filter) + IColumn::Filter & filter) { - auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges( - dm_context.global_context, - dm_context.scan_context, - dm_context.tracing_id, - dmfile, segment_range); + auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges(dm_context, dmfile, segment_range); if (valid_handle_res.empty()) return 0; - const auto max_versions = loadPackMaxValue(dm_context.global_context, *dmfile, VERSION_COLUMN_ID); + const auto max_versions = loadPackMaxValue(dm_context.global_context, *dmfile, MutSup::version_col_id); auto read_packs = std::make_shared(); UInt32 need_read_rows = 0; @@ -130,11 +126,12 @@ namespace DB::DM auto block = stream->read(); RUNTIME_CHECK(block.rows() == pack_stats[pack_id].rows, block.rows(), pack_stats[pack_id].rows); read_rows += block.rows(); - const auto * handles_ptr = toColumnVectorDataPtr(block.getByName(EXTRA_HANDLE_COLUMN_NAME).column); + const auto * handles_ptr + = toColumnVectorDataPtr(block.getByName(MutSup::extra_handle_column_name).column); RUNTIME_CHECK_MSG(handles_ptr != nullptr, "TODO: support common handle"); const auto & handles = *handles_ptr; const auto & versions - = *toColumnVectorDataPtr(block.getByName(VERSION_COLUMN_NAME).column); // Must success. + = *toColumnVectorDataPtr(block.getByName(MutSup::version_column_name).column); // Must success. const auto itr = read_pack_to_start_row_ids.find(pack_id); RUNTIME_CHECK(itr != read_pack_to_start_row_ids.end(), read_pack_to_start_row_ids, pack_id); @@ -197,7 +194,7 @@ namespace DB::DM const ColumnFileBig & cf_big, const UInt64 read_ts, const ssize_t start_row_id, - std::vector & filter) + IColumn::Filter & filter) { return buildVersionFilterDMFile(dm_context, cf_big.getFile(), cf_big.getRange(), read_ts, start_row_id, filter); } @@ -206,7 +203,7 @@ namespace DB::DM const DMContext & dm_context, const StableValueSpace::Snapshot & stable, const UInt64 read_ts, - std::vector & filter) + IColumn::Filter & filter) { const auto & dmfiles = stable.getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); @@ -218,7 +215,7 @@ void buildVersionFilter( const SegmentSnapshot & snapshot, const std::vector & base_ver_snap, const UInt64 read_ts, - std::vector & filter) + IColumn::Filter & filter) { const auto & delta = *(snapshot.delta); const auto & stable = *(snapshot.stable); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h index ff4230ec3d3..53c074f963a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h @@ -14,8 +14,8 @@ #pragma once +#include #include - namespace DB::DM { struct DMContext; @@ -25,5 +25,5 @@ void buildVersionFilter( const SegmentSnapshot & snapshot, const std::vector & base_ver_snap, const UInt64 read_ts, - std::vector & filter); + IColumn::Filter & filter); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 334f9beff9d..471ca1ee403 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -89,7 +89,7 @@ void initContext(bool is_common_handle, BenchType type) /*min_version_*/ 0, NullspaceID, /*physical_table_id*/ 100, - /*pk_col_id*/ EXTRA_HANDLE_COLUMN_ID, + /*pk_col_id*/ MutableSupport::extra_handle_id, is_common_handle, 1, // rowkey_column_size context->getSettingsRef()); @@ -158,9 +158,9 @@ void writeDelta(Segment & seg, UInt32 delta_rows) Block block; const auto n = std::min(delta_rows - i, 2048U); const auto v = random_sequences.get(n); - block.insert(createColumn(v, EXTRA_HANDLE_COLUMN_NAME, EXTRA_HANDLE_COLUMN_ID)); - block.insert(createColumn(std::vector(n, version++), VERSION_COLUMN_NAME, VERSION_COLUMN_ID)); - block.insert(createColumn(std::vector(n, /*deleted*/ 0), TAG_COLUMN_NAME, TAG_COLUMN_ID)); + block.insert(createColumn(v, MutSup::extra_handle_column_name, MutSup::extra_handle_id)); + block.insert(createColumn(std::vector(n, version++), VERSION_COLUMN_NAME, MutSup::version_col_id)); + block.insert(createColumn(std::vector(n, /*deleted*/ 0), TAG_COLUMN_NAME, MutSup::delmark_col_id)); seg.write(*dm_context, block, false); } } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp index af141441993..f244e6b2589 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp @@ -15,6 +15,9 @@ #include #include #include +#include + +using namespace DB::tests; namespace DB::DM::tests { @@ -25,7 +28,10 @@ TEST(HandleIndexTest, Basic) TEST(HandleColumnView, Basic) { - [[maybe_unused]] HandleColumnView handle_column_view_int; - [[maybe_unused]] HandleColumnView handle_column_view_string; + auto int_col = makeColumn(MutSup::getExtraHandleColumnIntType(), {1, 2, 3, 4, 5, 6, 7, 8}); + [[maybe_unused]] HandleColumnView handle_column_view_int(*int_col); + + auto str_col = makeColumn(MutSup::getExtraHandleColumnStringType(), {"1", "2", "3", "4", "5", "6", "7", "8"}); + [[maybe_unused]] HandleColumnView handle_column_view_string(*str_col); } } // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp index 154559315c0..204e19456d5 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp @@ -133,6 +133,18 @@ class SegmentTest : public DB::base::TiFlashStorageTestBasic DMContext & dmContext() { return *dm_context; } + auto loadPackFilterResults(const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) + { + DMFilePackFilterResults results; + results.reserve(snap->stable->getDMFiles().size()); + for (const auto & file : snap->stable->getDMFiles()) + { + auto pack_filter = DMFilePackFilter::loadFrom(*dm_context, file, true, ranges, EMPTY_RS_OPERATOR, {}); + results.push_back(pack_filter); + } + return results; + } + protected: /// all these var lives as ref in dm_context GlobalPageIdAllocatorPtr page_id_allocator; @@ -1151,11 +1163,12 @@ try auto segment_snap = segment->createSnapshot(dmContext(), false, CurrentMetrics::DT_SnapshotOfRead); auto read_ranges = {RowKeyRange::newAll(false, 1)}; auto real_ranges = segment->shrinkRowKeyRanges(read_ranges); + auto bitmap_filter1 = segment->buildBitmapFilter( // dmContext(), segment_snap, real_ranges, - EMPTY_RS_OPERATOR, + loadPackFilterResults(segment_snap, real_ranges), std::numeric_limits::max(), DEFAULT_BLOCK_SIZE, !use_version_chain); @@ -1163,7 +1176,7 @@ try dmContext(), segment_snap, real_ranges, - EMPTY_RS_OPERATOR, + loadPackFilterResults(segment_snap, real_ranges), std::numeric_limits::max(), DEFAULT_BLOCK_SIZE, use_version_chain); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp index edc58b04c13..84c53602d79 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp @@ -183,6 +183,18 @@ class SegmentBitmapFilterTest : public SegmentTestBasic verifyBitmapFilter(SEG_ID); } + + auto loadPackFilterResults(const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) + { + DMFilePackFilterResults results; + results.reserve(snap->stable->getDMFiles().size()); + for (const auto & file : snap->stable->getDMFiles()) + { + auto pack_filter = DMFilePackFilter::loadFrom(*dm_context, file, true, ranges, EMPTY_RS_OPERATOR, {}); + results.push_back(pack_filter); + } + return results; + } inline static constexpr bool use_version_chain = true; void verifyBitmapFilter(const PageIdU64 seg_id, const UInt64 read_ts = std::numeric_limits::max()) @@ -193,7 +205,7 @@ class SegmentBitmapFilterTest : public SegmentTestBasic *dm_context, snap, {seg->getRowKeyRange()}, - nullptr, + loadPackFilterResults(snap, {seg->getRowKeyRange()}), read_ts, DEFAULT_BLOCK_SIZE, !use_version_chain); @@ -202,25 +214,13 @@ class SegmentBitmapFilterTest : public SegmentTestBasic *dm_context, snap, {seg->getRowKeyRange()}, - nullptr, + loadPackFilterResults(snap, {seg->getRowKeyRange()}), read_ts, DEFAULT_BLOCK_SIZE, use_version_chain); ASSERT_EQ(bitmap_filter1->toDebugString(), bitmap_filter2->toDebugString()); } - - auto loadPackFilterResults(const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) - { - DMFilePackFilterResults results; - results.reserve(snap->stable->getDMFiles().size()); - for (const auto & file : snap->stable->getDMFiles()) - { - auto pack_filter = DMFilePackFilter::loadFrom(*dm_context, file, true, ranges, EMPTY_RS_OPERATOR, {}); - results.push_back(pack_filter); - } - return results; - } }; TEST_F(SegmentBitmapFilterTest, InMemory1) From 1064e29bcb1c09439f5b46820837e6e955aa0c20 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 12:53:12 +0800 Subject: [PATCH 04/48] ci --- .../tests/bench_version_chain.cpp | 36 ++++++++++++++----- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 471ca1ee403..8596a72a5cc 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -56,6 +56,19 @@ enum class BenchType VersionChain = 2, }; + +auto loadPackFilterResults(const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) +{ + DMFilePackFilterResults results; + results.reserve(snap->stable->getDMFiles().size()); + for (const auto & file : snap->stable->getDMFiles()) + { + auto pack_filter = DMFilePackFilter::loadFrom(*dm_context, file, true, ranges, EMPTY_RS_OPERATOR, {}); + results.push_back(pack_filter); + } + return results; +} + void initContext(bool is_common_handle, BenchType type) { if (context) @@ -89,7 +102,7 @@ void initContext(bool is_common_handle, BenchType type) /*min_version_*/ 0, NullspaceID, /*physical_table_id*/ 100, - /*pk_col_id*/ MutableSupport::extra_handle_id, + /*pk_col_id*/ MutSup::extra_handle_id, is_common_handle, 1, // rowkey_column_size context->getSettingsRef()); @@ -159,8 +172,14 @@ void writeDelta(Segment & seg, UInt32 delta_rows) const auto n = std::min(delta_rows - i, 2048U); const auto v = random_sequences.get(n); block.insert(createColumn(v, MutSup::extra_handle_column_name, MutSup::extra_handle_id)); - block.insert(createColumn(std::vector(n, version++), VERSION_COLUMN_NAME, MutSup::version_col_id)); - block.insert(createColumn(std::vector(n, /*deleted*/ 0), TAG_COLUMN_NAME, MutSup::delmark_col_id)); + block.insert(createColumn( + std::vector(n, version++), + MutSup::version_column_name, + MutSup::version_col_id)); + block.insert(createColumn( + std::vector(n, /*deleted*/ 0), + MutSup::delmark_column_name, + MutSup::delmark_col_id)); seg.write(*dm_context, block, false); } } @@ -322,6 +341,7 @@ try { const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); initialize(type, is_common_handle, delta_rows); + auto rs_results = loadPackFilterResults(segment_snapshot, {segment->getRowKeyRange()}); if (type == BenchType::DeltaIndex) { @@ -337,7 +357,7 @@ try *dm_context, segment_snapshot, {segment->getRowKeyRange()}, - nullptr, + rs_results, std::numeric_limits::max(), DEFAULT_BLOCK_SIZE, false); @@ -355,7 +375,7 @@ try *dm_context, *segment_snapshot, {segment->getRowKeyRange()}, - nullptr, + rs_results, std::numeric_limits::max(), version_chain); benchmark::DoNotOptimize(bitmap_filter); @@ -383,12 +403,12 @@ try buildVersionChain(*segment_snapshot, version_chain); RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); - + auto rs_results = loadPackFilterResults(segment_snapshot, {segment->getRowKeyRange()}); auto bitmap_filter1 = segment->buildBitmapFilter( *dm_context, segment_snapshot, {segment->getRowKeyRange()}, - nullptr, + rs_results, std::numeric_limits::max(), DEFAULT_BLOCK_SIZE, false); @@ -396,7 +416,7 @@ try *dm_context, *segment_snapshot, {segment->getRowKeyRange()}, - nullptr, + rs_results, std::numeric_limits::max(), version_chain); From 0e0133484c0e27706dc80c5611ac7821ce1d078f Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 16:14:18 +0800 Subject: [PATCH 05/48] ci --- .../{HandleColumnView.h => ColumnView.h} | 19 +++++++++++-------- .../VersionChain/DMFileHandleIndex.h | 5 +++-- .../DeltaMerge/VersionChain/VersionChain.cpp | 1 + .../tests/gtest_handle_pack_index.cpp | 14 +++++++------- 4 files changed, 22 insertions(+), 17 deletions(-) rename dbms/src/Storages/DeltaMerge/VersionChain/{HandleColumnView.h => ColumnView.h} (86%) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h b/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h similarity index 86% rename from dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h rename to dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h index 9ad47fba9d2..e249eadbe96 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/HandleColumnView.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h @@ -14,18 +14,21 @@ #pragma once +#include + namespace DB::DM { template -class HandleColumnView +class ColumnView { + static_assert(false, "Only support Int64 and String"); }; template <> -class HandleColumnView +class ColumnView { public: - HandleColumnView(const IColumn & col) + ColumnView(const IColumn & col) : data(toColumnVectorData(col)) {} @@ -38,10 +41,10 @@ class HandleColumnView }; template <> -class HandleColumnView +class ColumnView { public: - HandleColumnView(const IColumn & col) + ColumnView(const IColumn & col) : offsets(typeid_cast(col).getOffsets()) , chars(typeid_cast(col).getChars()) {} @@ -55,12 +58,12 @@ class HandleColumnView , pos(pos) {} - std::string_view operator*() const + StringRef operator*() const { assert(offsets[-1] == 0); - const auto off = offsets[pos]; + const auto off = offsets[pos - 1]; const auto size = offsets[pos] - offsets[pos - 1] - 1; - return std::string_view(reinterpret_cast(&chars[off]), size); + return StringRef(reinterpret_cast(&chars[off]), size); } Iterator & operator+(size_t n) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h index b4a1b041be1..86e09a108b6 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h @@ -20,8 +20,9 @@ #include #include #include -#include +#include #include + namespace DB::DM { @@ -109,7 +110,7 @@ class DMFileHandleIndex std::optional getBaseVersion(const DMContext & dm_context, HandleView h, UInt32 clipped_pack_id) { loadHandleIfNotLoaded(dm_context); - HandleColumnView handle_col(*clipped_handle_packs[clipped_pack_id]); + ColumnView handle_col(*clipped_handle_packs[clipped_pack_id]); auto itr = std::lower_bound(handle_col.begin(), handle_col.end(), h); if (itr != handle_col.end() && *itr == h) return itr - handle_col.begin() + clipped_pack_offsets[clipped_pack_id]; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index 9ad09b2a111..ce49e8d6691 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -191,6 +191,7 @@ std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRang const DMContext & dm_context, Handle h) { + // From from new to old for (auto itr = dmfile_or_delete_range_list->rbegin(); itr != dmfile_or_delete_range_list->rend(); ++itr) { auto & dmfile_or_delete_range = *itr; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp index f244e6b2589..62518dac8ce 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp @@ -13,7 +13,8 @@ // limitations under the License. #include -#include +#include +#include #include #include @@ -26,12 +27,11 @@ TEST(HandleIndexTest, Basic) [[maybe_unused]] VersionChain version_chain_int; } -TEST(HandleColumnView, Basic) +TEST(ColumnView, Basic) { - auto int_col = makeColumn(MutSup::getExtraHandleColumnIntType(), {1, 2, 3, 4, 5, 6, 7, 8}); - [[maybe_unused]] HandleColumnView handle_column_view_int(*int_col); - - auto str_col = makeColumn(MutSup::getExtraHandleColumnStringType(), {"1", "2", "3", "4", "5", "6", "7", "8"}); - [[maybe_unused]] HandleColumnView handle_column_view_string(*str_col); + auto str_col = ColumnGenerator::instance().generate({1024, "String", RANDOM}).column; + ColumnView str_cv(*str_col); + for (auto s_itr = str_cv.begin(); s_itr != str_cv.end(); ++s_itr) + ASSERT_EQ(*s_itr, str_col->getDataAt(s_itr - str_cv.begin())); } } // namespace DB::DM::tests From 659c93aba5a493b9da1ecfa38b629110a775702a Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 16:52:19 +0800 Subject: [PATCH 06/48] ci --- .../DeltaMerge/VersionChain/VersionChain.cpp | 28 +++++++-------- .../DeltaMerge/VersionChain/VersionChain.h | 35 +++++++++---------- .../tests/bench_version_chain.cpp | 8 ++--- 3 files changed, 34 insertions(+), 37 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index ce49e8d6691..55d36e00316 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -26,11 +26,11 @@ std::shared_ptr> VersionChain::replaySnapshot( const SegmentSnapshot & snapshot) { // Check Stable - if (dmfile_or_delete_range_list->empty()) + if (dmfile_or_delete_range_list.empty()) { const auto & dmfiles = snapshot.stable->getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); - dmfile_or_delete_range_list->push_back( + dmfile_or_delete_range_list.push_back( DMFileHandleIndex{dm_context, dmfiles[0], /*start_row_id*/ 0, std::nullopt}); } @@ -66,7 +66,7 @@ std::shared_ptr> VersionChain::replaySnapshot( RUNTIME_CHECK(offset == 0 || (*pos)->isInMemoryFile() || (*pos)->isTinyFile(), offset, (*pos)->toString()); const bool calculate_read_packs = (cfs.end() - pos == 1) && ((*pos)->isInMemoryFile() || (*pos)->isTinyFile()) - && dmfile_or_delete_range_list->size() == 1; + && dmfile_or_delete_range_list.size() == 1; const auto initial_replayed_rows_and_deletes = replayed_rows_and_deletes; SCOPE_EXIT({ cleanHandleColumn(); }); for (; pos != cfs.end(); ++pos) @@ -138,7 +138,7 @@ UInt32 VersionChain::replayBlock( for (auto h : handles) { const RowID curr_row_id = base_versions->size() + stable_rows; - if (auto itr = new_handle_to_row_ids->find(h); itr != new_handle_to_row_ids->end()) + if (auto itr = new_handle_to_row_ids.find(h); itr != new_handle_to_row_ids.end()) { base_versions->push_back(itr->second); continue; @@ -149,7 +149,7 @@ UInt32 VersionChain::replayBlock( continue; } - new_handle_to_row_ids->insert(std::make_pair(h, curr_row_id)); + new_handle_to_row_ids.insert(std::make_pair(h, curr_row_id)); base_versions->push_back(NotExistRowID); } return handles.size(); @@ -165,7 +165,7 @@ UInt32 VersionChain::replayColumnFileBig( const UInt32 start_row_id = base_versions->size() + stable_rows; base_versions->insert(base_versions->end(), rows, NotExistRowID); - dmfile_or_delete_range_list->push_back( + dmfile_or_delete_range_list.push_back( DMFileHandleIndex{dm_context, cf_big.getFile(), start_row_id, cf_big.getRange()}); return rows; } @@ -174,15 +174,15 @@ template UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range) { auto [start, end] = convertRowKeyRange(cf_delete_range.getDeleteRange()); - auto itr = new_handle_to_row_ids->lower_bound(start); - const auto end_itr = new_handle_to_row_ids->lower_bound(end); + auto itr = new_handle_to_row_ids.lower_bound(start); + const auto end_itr = new_handle_to_row_ids.lower_bound(end); std::vector erased_handles; while (itr != end_itr) { erased_handles.push_back(itr->first); - itr = new_handle_to_row_ids->erase(itr); + itr = new_handle_to_row_ids.erase(itr); } - dmfile_or_delete_range_list->push_back(cf_delete_range.getDeleteRange()); + dmfile_or_delete_range_list.push_back(cf_delete_range.getDeleteRange()); return cf_delete_range.getDeletes(); } @@ -192,7 +192,7 @@ std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRang Handle h) { // From from new to old - for (auto itr = dmfile_or_delete_range_list->rbegin(); itr != dmfile_or_delete_range_list->rend(); ++itr) + for (auto itr = dmfile_or_delete_range_list.rbegin(); itr != dmfile_or_delete_range_list.rend(); ++itr) { auto & dmfile_or_delete_range = *itr; if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) @@ -212,15 +212,15 @@ std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRang template void VersionChain::calculateReadPacks(const std::span handles) { - assert(dmfile_or_delete_range_list->size() == 1); - auto & dmfile_index = std::get>(dmfile_or_delete_range_list->front()); + assert(dmfile_or_delete_range_list.size() == 1); + auto & dmfile_index = std::get>(dmfile_or_delete_range_list.front()); dmfile_index.calculateReadPacks(handles); } template void VersionChain::cleanHandleColumn() { - for (auto & dmfile_or_delete_range : *dmfile_or_delete_range_list) + for (auto & dmfile_or_delete_range : dmfile_or_delete_range_list) { if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) dmfile_index->cleanHandleColumn(); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index ea2fc3960e6..701d59e1a0b 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -14,7 +14,7 @@ #pragma once -#include +#include #include #include @@ -35,26 +35,25 @@ class VersionChain public: VersionChain() : base_versions(std::make_shared>()) - , new_handle_to_row_ids(std::make_shared>()) - , dmfile_or_delete_range_list(std::make_shared>()) {} + // Deep copy, a benchmark helper. + VersionChain(const VersionChain & other) + : replayed_rows_and_deletes(other.replayed_rows_and_deletes) + , base_versions(std::make_shared>(*(other.base_versions))) + , new_handle_to_row_ids(other.new_handle_to_row_ids) + , dmfile_or_delete_range_list(other.dmfile_or_delete_range_list) + {} + + VersionChain & operator=(const VersionChain &) = delete; + VersionChain(VersionChain &&) = delete; + VersionChain & operator=(VersionChain &&) = delete; + [[nodiscard]] std::shared_ptr> replaySnapshot( const DMContext & dm_context, const SegmentSnapshot & snapshot); - std::unique_ptr> deepCopy() - { - auto new_version_chain = std::make_unique>(); - new_version_chain->replayed_rows_and_deletes = replayed_rows_and_deletes; - new_version_chain->base_versions = std::make_shared>(*base_versions); - new_version_chain->new_handle_to_row_ids = std::make_shared>(*new_handle_to_row_ids); - new_version_chain->dmfile_or_delete_range_list - = std::make_shared>(*dmfile_or_delete_range_list); - return new_version_chain; - } - - UInt32 getReplayedRows() const { return base_versions->size(); } + [[nodiscard]] UInt32 getReplayedRows() const { return base_versions->size(); } private: [[nodiscard]] UInt32 replayBlock( @@ -76,13 +75,11 @@ class VersionChain void calculateReadPacks(const std::span handles); void cleanHandleColumn(); - DISALLOW_COPY_AND_MOVE(VersionChain); - std::mutex mtx; UInt32 replayed_rows_and_deletes = 0; // delta.getRows() + delta.getDeletes() std::shared_ptr> base_versions; // base_versions->size() == delta.getRows() - std::shared_ptr> new_handle_to_row_ids; // TODO: shared_ptr is unneccessary + std::map new_handle_to_row_ids; using DMFileOrDeleteRange = std::variant>; - std::shared_ptr> dmfile_or_delete_range_list; // TODO: shared_ptr is unneccessary + std::vector dmfile_or_delete_range_list; }; } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 8596a72a5cc..415ae79bd05 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -324,10 +324,10 @@ try RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); for (auto _ : state) { - auto version_chain = base_version_chain.deepCopy(); - RUNTIME_ASSERT(version_chain->getReplayedRows() == prepared_delta_rows); - buildVersionChain(*segment_snapshot, *version_chain); - RUNTIME_ASSERT(version_chain->getReplayedRows() == prepared_delta_rows + incremental_delta_rows); + auto version_chain = base_version_chain; + RUNTIME_ASSERT(version_chain.getReplayedRows() == prepared_delta_rows); + buildVersionChain(*segment_snapshot, version_chain); + RUNTIME_ASSERT(version_chain.getReplayedRows() == prepared_delta_rows + incremental_delta_rows); } } From 297624580051c715e4453c753c617432f67102c7 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 17:17:49 +0800 Subject: [PATCH 07/48] ci --- .../Storages/DeltaMerge/VersionChain/Common.h | 6 +++--- .../DeltaMerge/VersionChain/DMFileHandleIndex.h | 17 ++++++++--------- .../DeltaMerge/VersionChain/VersionChain.cpp | 2 +- 3 files changed, 12 insertions(+), 13 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h index 3b0f03029f5..e033c340810 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h @@ -34,7 +34,7 @@ template concept Int64OrString = std::same_as || std::same_as; template -concept Int64OrStringView = std::same_as || std::same_as; +concept Int64OrStringRef = std::same_as || std::same_as; template ColumnDefine getHandleColumnDefine() @@ -67,8 +67,8 @@ inline ColumnDefinesPtr getTagColumnDefinesPtr() return cds_ptr; } -template -bool inRowKeyRange(const RowKeyRange & range, HandleView handle) +template +bool inRowKeyRange(const RowKeyRange & range, HandleRef handle) { if constexpr (std::is_same_v) return range.start.int_value <= handle && handle < range.end.int_value; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h index 86e09a108b6..23eb636e8a2 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h @@ -18,9 +18,8 @@ #include #include #include -#include -#include #include +#include #include namespace DB::DM @@ -54,8 +53,8 @@ class DMFileHandleIndex clipped_pack_range.count()); } - template - std::optional getBaseVersion(const DMContext & dm_context, HandleView h) + template + std::optional getBaseVersion(const DMContext & dm_context, HandleRef h) { auto clipped_pack_id = getClippedPackId(h); if (!clipped_pack_id) @@ -81,7 +80,7 @@ class DMFileHandleIndex // Read too many packs, read all by default if (calc_read_count * 4 >= clipped_pack_range.count()) - return; + return; // return, instead of break, because `clipped_need_read_packs` is read all by default. } clipped_need_read_packs->swap(calc_read_packs); } @@ -94,8 +93,8 @@ class DMFileHandleIndex } private: - template - std::optional getClippedPackId(HandleView h) + template + std::optional getClippedPackId(HandleRef h) { if (unlikely(rowkey_range && !inRowKeyRange(*rowkey_range, h))) return {}; @@ -106,8 +105,8 @@ class DMFileHandleIndex return itr - clipped_pack_index.begin(); } - template - std::optional getBaseVersion(const DMContext & dm_context, HandleView h, UInt32 clipped_pack_id) + template + std::optional getBaseVersion(const DMContext & dm_context, HandleRef h, UInt32 clipped_pack_id) { loadHandleIfNotLoaded(dm_context); ColumnView handle_col(*clipped_handle_packs[clipped_pack_id]); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index 55d36e00316..52c7dcedf54 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -25,7 +25,7 @@ std::shared_ptr> VersionChain::replaySnapshot( const DMContext & dm_context, const SegmentSnapshot & snapshot) { - // Check Stable + // Check Stable: stable always has one DMFile. if (dmfile_or_delete_range_list.empty()) { const auto & dmfiles = snapshot.stable->getDMFiles(); From 1ead298405f8258d83c7c4ba5b77f7a3f92823f0 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 18:02:10 +0800 Subject: [PATCH 08/48] ci --- .../DeltaMerge/VersionChain/VersionChain.cpp | 24 ++++++++++++------- .../DeltaMerge/VersionChain/VersionChain.h | 2 +- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index 52c7dcedf54..f32d63b20ae 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -31,7 +31,7 @@ std::shared_ptr> VersionChain::replaySnapshot( const auto & dmfiles = snapshot.stable->getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); dmfile_or_delete_range_list.push_back( - DMFileHandleIndex{dm_context, dmfiles[0], /*start_row_id*/ 0, std::nullopt}); + DMFileHandleIndex{dm_context, dmfiles[0], /*start_row_id*/ 0, /*rowkey_range*/ std::nullopt}); } const auto & stable = *(snapshot.stable); @@ -46,6 +46,7 @@ std::shared_ptr> VersionChain::replaySnapshot( return base_versions; } + // Copy for write base_versions = std::make_shared>(*base_versions); const auto cfs = delta.getColumnFiles(); const auto & data_provider = delta.getDataProvider(); @@ -59,10 +60,10 @@ std::shared_ptr> VersionChain::replaySnapshot( break; skipped_rows_and_deletes += skip_n; } - - // `pos` points to the first ColumnFile that has data not been replayed + // `pos` points to the first ColumnFile that has records not been replayed. + // `offset` points to the first records that has not been replayed in `pos`. auto offset = replayed_rows_and_deletes - skipped_rows_and_deletes; - // Only ColumnFileInMemory or ColumnFileTiny can be half replayed + // Only ColumnFileInMemory or ColumnFileTiny can be half replayed. RUNTIME_CHECK(offset == 0 || (*pos)->isInMemoryFile() || (*pos)->isTinyFile(), offset, (*pos)->toString()); const bool calculate_read_packs = (cfs.end() - pos == 1) && ((*pos)->isInMemoryFile() || (*pos)->isTinyFile()) @@ -126,10 +127,17 @@ UInt32 VersionChain::replayBlock( auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); auto block = cf_reader->readNextBlock(); - RUNTIME_CHECK_MSG(!cf_reader->readNextBlock(), "{}: read all rows in one block is required!", cf.toString()); - const auto * handle_col = toColumnVectorDataPtr(block.begin()->column); - RUNTIME_CHECK_MSG(handle_col != nullptr, "TODO: support common handle"); - RUNTIME_CHECK(handle_col->size() > offset, handle_col->size(), offset); + RUNTIME_CHECK_MSG( + cf.getRows() == block.rows(), + "ColumnFile<{}> returns {} rows. Read all rows in one block is required!", + cf.toString(), + block.rows()); + + const auto & column = *(block.begin()->column); + RUNTIME_CHECK(column.size() > offset, column.size(), offset); + + const auto handles_col = ColumnView(*(block.begin()->column)); + const std::span handles{handle_col->data() + offset, handle_col->size() - offset}; if (calculate_read_packs) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index 701d59e1a0b..19a7271bd1a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -37,7 +37,7 @@ class VersionChain : base_versions(std::make_shared>()) {} - // Deep copy, a benchmark helper. + // Deep copy, only use for micro-benchmark or unit-tests. VersionChain(const VersionChain & other) : replayed_rows_and_deletes(other.replayed_rows_and_deletes) , base_versions(std::make_shared>(*(other.base_versions))) From d4673fce636e1be80e2e6d458d965834fb74f8e8 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 21:28:25 +0800 Subject: [PATCH 09/48] ci --- .../Storages/DeltaMerge/VersionChain/Common.h | 9 ------ .../VersionChain/DMFileHandleIndex.h | 7 +++-- .../DeltaMerge/VersionChain/VersionChain.cpp | 30 ++++++++++++------- .../DeltaMerge/VersionChain/VersionChain.h | 5 +++- 4 files changed, 27 insertions(+), 24 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h index e033c340810..bafc7233a88 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h @@ -76,15 +76,6 @@ bool inRowKeyRange(const RowKeyRange & range, HandleRef handle) static_assert(false, "TODO: support common handle"); } -template -std::pair convertRowKeyRange(const RowKeyRange & range) -{ - if constexpr (std::is_same_v) - return {range.start.int_value, range.end.int_value}; - else - static_assert(false, "TODO: support common handle"); -} - RSResults getRSResultsByRanges(const DMContext & dm_context, const DMFilePtr & dmfile, const RowKeyRanges & ranges); std::pair getClippedRSResultsByRanges( diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h index 23eb636e8a2..cabadd0a171 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h @@ -65,13 +65,14 @@ class DMFileHandleIndex return start_row_id + *row_id; } - void calculateReadPacks(const std::span handles) + template + void calculateReadPacks(Iterator begin, Iterator end) { std::vector calc_read_packs(clipped_pack_range.count(), 0); UInt32 calc_read_count = 0; - for (const Handle & h : handles) + for (auto itr = begin; itr != end; ++itr) { - auto clipped_pack_id = getClippedPackId(h); + auto clipped_pack_id = getClippedPackId(*itr); if (!clipped_pack_id || calc_read_packs[*clipped_pack_id]) continue; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index f32d63b20ae..8ffc07b6087 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -136,15 +136,15 @@ UInt32 VersionChain::replayBlock( const auto & column = *(block.begin()->column); RUNTIME_CHECK(column.size() > offset, column.size(), offset); - const auto handles_col = ColumnView(*(block.begin()->column)); - - const std::span handles{handle_col->data() + offset, handle_col->size() - offset}; + const auto handle_col = ColumnView(*(block.begin()->column)); + auto itr = handle_col.begin() + offset; if (calculate_read_packs) - calculateReadPacks(handles); + calculateReadPacks(itr, handle_col.end()); - for (auto h : handles) + for (; itr != handle_col.end(); ++itr) { + const auto h = *itr; const RowID curr_row_id = base_versions->size() + stable_rows; if (auto itr = new_handle_to_row_ids.find(h); itr != new_handle_to_row_ids.end()) { @@ -160,7 +160,7 @@ UInt32 VersionChain::replayBlock( new_handle_to_row_ids.insert(std::make_pair(h, curr_row_id)); base_versions->push_back(NotExistRowID); } - return handles.size(); + return column.size() - offset; } template @@ -181,13 +181,11 @@ UInt32 VersionChain::replayColumnFileBig( template UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range) { - auto [start, end] = convertRowKeyRange(cf_delete_range.getDeleteRange()); + auto [start, end] = convertRowKeyRange(cf_delete_range.getDeleteRange()); auto itr = new_handle_to_row_ids.lower_bound(start); const auto end_itr = new_handle_to_row_ids.lower_bound(end); - std::vector erased_handles; while (itr != end_itr) { - erased_handles.push_back(itr->first); itr = new_handle_to_row_ids.erase(itr); } dmfile_or_delete_range_list.push_back(cf_delete_range.getDeleteRange()); @@ -218,11 +216,12 @@ std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRang } template -void VersionChain::calculateReadPacks(const std::span handles) +template +void VersionChain::calculateReadPacks(Iterator begin, Iterator end) { assert(dmfile_or_delete_range_list.size() == 1); auto & dmfile_index = std::get>(dmfile_or_delete_range_list.front()); - dmfile_index.calculateReadPacks(handles); + dmfile_index.calculateReadPacks(begin, end); } template @@ -235,6 +234,15 @@ void VersionChain::cleanHandleColumn() } } +template +std::pair VersionChain::convertRowKeyRange(const RowKeyRange & range) +{ + if constexpr (std::is_same_v) + return {range.start.int_value, range.end.int_value}; + else + return {*(range.start.value), *(range.end.value)}; +} + template class VersionChain; //template class VersionChain; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index 19a7271bd1a..b76df64c15a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -72,9 +72,12 @@ class VersionChain [[nodiscard]] std::optional findBaseVersionFromDMFileOrDeleteRangeList( const DMContext & dm_context, Handle h); - void calculateReadPacks(const std::span handles); + template + void calculateReadPacks(Iterator begin, Iterator end); void cleanHandleColumn(); + static std::pair convertRowKeyRange(const RowKeyRange & range); + std::mutex mtx; UInt32 replayed_rows_and_deletes = 0; // delta.getRows() + delta.getDeletes() std::shared_ptr> base_versions; // base_versions->size() == delta.getRows() From a2df5c9e3a176ec044f480a9f0be29a14e85fffa Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 21:33:34 +0800 Subject: [PATCH 10/48] ci --- dbms/gtest.log | 1 - 1 file changed, 1 deletion(-) delete mode 100644 dbms/gtest.log diff --git a/dbms/gtest.log b/dbms/gtest.log deleted file mode 100644 index c24670afad1..00000000000 --- a/dbms/gtest.log +++ /dev/null @@ -1 +0,0 @@ -zsh: no such file or directory: ./dbms/gtests_dbms From 67652e8350503532f03acb394c6f95d9ce0ee3e3 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 21:35:20 +0800 Subject: [PATCH 11/48] ci --- .../DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp | 4 ++-- .../Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp | 2 +- dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp index 62518dac8ce..8299a0a4431 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp @@ -12,11 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include +#include #include -#include #include +#include using namespace DB::tests; 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 67f7fbcd803..ac60e35ff77 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 @@ -1030,7 +1030,7 @@ try // Test write multi blocks with overlap { - UInt64 tso1 = 3; // ts of the same key should incre... + UInt64 tso1 = 3; // ts of the same key should incre... UInt64 tso2 = 100; Block block1 = DMTestEnv::prepareSimpleWriteBlock(0, 1 * num_write_rows, false, tso1); Block block2 = DMTestEnv::prepareSimpleWriteBlock(1 * num_write_rows, 2 * num_write_rows, false, tso1); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp index 84c53602d79..69da6c789ff 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_bitmap.cpp @@ -183,7 +183,7 @@ class SegmentBitmapFilterTest : public SegmentTestBasic verifyBitmapFilter(SEG_ID); } - + auto loadPackFilterResults(const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) { DMFilePackFilterResults results; From 2d6f7d8b4bb1458c749732d1d0ec2ce9006a464c Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 21:51:49 +0800 Subject: [PATCH 12/48] ci --- dbms/CMakeLists.txt | 4 + .../DeltaMerge/tests/bench_vector_index.cpp | 98 ++++++++++ .../tests/bench_vector_index_utils.h | 177 ++++++++++++++++++ 3 files changed, 279 insertions(+) create mode 100644 dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp create mode 100644 dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 8116e56400e..fdc5a70fbc5 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -408,6 +408,10 @@ if (ENABLE_TESTS) tiflash_aggregate_functions kvstore) + if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") + target_link_libraries(bench_dbms tiflash_contrib::highfive) + endif() + add_check(bench_dbms) endif () diff --git a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp b/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp new file mode 100644 index 00000000000..7792664168a --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp @@ -0,0 +1,98 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include + +namespace DB::DM::bench +{ + +static void VectorIndexBuild(::benchmark::State & state) +try +{ + const auto & dataset = DatasetMnist::get(); + + auto train_data = dataset.buildDataTrainColumn(/* max_rows= */ 10000); + auto index_def = dataset.createIndexDef(tipb::VectorIndexKind::HNSW); + for (auto _ : state) + { + auto builder = std::make_unique(0, index_def); + builder->addBlock(*train_data, nullptr, []() { return true; }); + } +} +CATCH + +static void VectorIndexSearchTop10(::benchmark::State & state) +try +{ + const auto & dataset = DatasetMnist::get(); + + auto index_path = DB::tests::TiFlashTestEnv::getTemporaryPath("vector_search_top_10/vector_index.idx"); + VectorIndexBenchUtils::saveVectorIndex( // + index_path, + dataset, + /* max_rows= */ 10000); + + auto viewer = VectorIndexBenchUtils::viewVectorIndex(index_path, dataset); + + std::random_device rd; + std::mt19937 rng(rd()); + std::uniform_int_distribution dist(0, dataset.dataTestSize() - 1); + + for (auto _ : state) + { + auto test_index = dist(rng); + const auto & query_vector = DatasetMnist::get().dataTestAt(test_index); + auto keys = VectorIndexBenchUtils::queryTopK(viewer, query_vector, 10, state); + RUNTIME_CHECK(keys.size() == 10); + } +} +CATCH + +static void VectorIndexSearchTop100(::benchmark::State & state) +try +{ + const auto & dataset = DatasetMnist::get(); + + auto index_path = DB::tests::TiFlashTestEnv::getTemporaryPath("vector_search_top_10/vector_index.idx"); + VectorIndexBenchUtils::saveVectorIndex( // + index_path, + dataset, + /* max_rows= */ 10000); + + auto viewer = VectorIndexBenchUtils::viewVectorIndex(index_path, dataset); + + std::random_device rd; + std::mt19937 rng(rd()); + std::uniform_int_distribution dist(0, dataset.dataTestSize() - 1); + + for (auto _ : state) + { + auto test_index = dist(rng); + const auto & query_vector = DatasetMnist::get().dataTestAt(test_index); + auto keys = VectorIndexBenchUtils::queryTopK(viewer, query_vector, 100, state); + RUNTIME_CHECK(keys.size() == 100); + } +} +CATCH + +BENCHMARK(VectorIndexBuild); + +BENCHMARK(VectorIndexSearchTop10); + +BENCHMARK(VectorIndexSearchTop100); + +} // namespace DB::DM::bench diff --git a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h b/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h new file mode 100644 index 00000000000..27e6c3cfa7e --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h @@ -0,0 +1,177 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB::DM::bench +{ + +/** + * @brief Compatible with datasets on ANN-Benchmark: + * https://github.com/erikbern/ann-benchmarks + */ +class Dataset +{ +public: + explicit Dataset(std::string_view file_name) + { + auto dataset_directory = std::filesystem::path(__FILE__).parent_path().string() + "/bench_dataset"; + auto dataset_path = fmt::format("{}/{}", dataset_directory, file_name); + + if (!std::filesystem::exists(dataset_path)) + { + throw Exception(fmt::format( + "Benchmark cannot run because dataset file {} not found. See {}/README.md for setup instructions.", + dataset_path, + dataset_directory)); + } + + auto file = HighFive::File(dataset_path, HighFive::File::ReadOnly); + + auto dataset_train = file.getDataSet("train"); + dataset_train.read(data_train); + + auto dataset_test = file.getDataSet("test"); + dataset_test.read(data_test); + } + + virtual ~Dataset() = default; + + virtual UInt32 dimension() const = 0; + + virtual tipb::VectorDistanceMetric distanceMetric() const = 0; + +public: + MutableColumnPtr buildDataTrainColumn(std::optional max_rows = std::nullopt) const + { + auto vec_column = ColumnArray::create(ColumnFloat32::create()); + size_t rows = data_train.size(); + if (max_rows.has_value()) + rows = std::min(rows, *max_rows); + for (size_t i = 0; i < rows; ++i) + { + const auto & row = data_train[i]; + vec_column->insertData(reinterpret_cast(row.data()), row.size() * sizeof(Float32)); + } + return vec_column; + } + + size_t dataTestSize() const { return data_test.size(); } + + const std::vector & dataTestAt(size_t index) const { return data_test.at(index); } + + TiDB::VectorIndexDefinitionPtr createIndexDef(tipb::VectorIndexKind kind) const + { + return std::make_shared(TiDB::VectorIndexDefinition{ + .kind = kind, + .dimension = dimension(), + .distance_metric = distanceMetric(), + }); + } + +protected: + std::vector> data_train; + std::vector> data_test; +}; + +class DatasetMnist : public Dataset +{ +public: + DatasetMnist() + : Dataset("fashion-mnist-784-euclidean.hdf5") + { + RUNTIME_CHECK(data_train[0].size() == dimension()); + RUNTIME_CHECK(data_test[0].size() == dimension()); + } + + UInt32 dimension() const override { return 784; } + + tipb::VectorDistanceMetric distanceMetric() const override { return tipb::VectorDistanceMetric::L2; } + + static const DatasetMnist & get() + { + static DatasetMnist dataset; + return dataset; + } +}; + +class VectorIndexBenchUtils +{ +public: + template + static void saveVectorIndex( + std::string_view index_path, + const Dataset & dataset, + std::optional max_rows = std::nullopt) + { + Poco::File(index_path.data()).createDirectories(); + + auto train_data = dataset.buildDataTrainColumn(max_rows); + auto index_def = dataset.createIndexDef(Builder::kind()); + auto builder = std::make_unique(0, index_def); + builder->addBlock(*train_data, nullptr, []() { return true; }); + builder->saveToFile(index_path); + } + + template + static auto viewVectorIndex(std::string_view index_path, const Dataset & dataset) + { + auto index_view_props = dtpb::VectorIndexFileProps(); + index_view_props.set_index_kind(tipb::VectorIndexKind_Name(Viewer::kind())); + index_view_props.set_dimensions(dataset.dimension()); + index_view_props.set_distance_metric(tipb::VectorDistanceMetric_Name(dataset.distanceMetric())); + return Viewer::view(index_view_props, index_path); + } + + static auto queryTopK( + VectorIndexViewerPtr viewer, + const std::vector & ref, + UInt32 top_k, + std::optional> state = std::nullopt) + { + if (state.has_value()) + state->get().PauseTiming(); + + auto ann_query_info = std::make_shared(); + auto distance_metric = tipb::VectorDistanceMetric::INVALID_DISTANCE_METRIC; + tipb::VectorDistanceMetric_Parse(viewer->file_props.distance_metric(), &distance_metric); + ann_query_info->set_distance_metric(distance_metric); + ann_query_info->set_top_k(top_k); + ann_query_info->set_ref_vec_f32(DB::DM::tests::VectorIndexTestUtils::encodeVectorFloat32(ref)); + + auto filter = BitmapFilterView::createWithFilter(viewer->size(), true); + + if (state.has_value()) + state->get().ResumeTiming(); + + return viewer->search(ann_query_info, filter); + } +}; + + +} // namespace DB::DM::bench From f03e18ada01601b2c02a68150590d5df06c80100 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 10 Jan 2025 22:58:35 +0800 Subject: [PATCH 13/48] ci --- .../VersionChain/BuildBitmapFilter.cpp | 2 +- .../DeltaMerge/VersionChain/BuildBitmapFilter.h | 2 +- .../DeltaMerge/VersionChain/ColumnView.h | 10 ++++++---- .../Storages/DeltaMerge/VersionChain/Common.h | 10 +++++----- .../DeltaMerge/VersionChain/DMFileHandleIndex.h | 8 ++++---- .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 12 ++++++------ .../DeltaMerge/VersionChain/RowKeyFilter.h | 2 +- .../DeltaMerge/VersionChain/VersionChain.cpp | 16 ++++++++-------- .../DeltaMerge/VersionChain/VersionChain.h | 2 +- .../tests/gtest_handle_pack_index.cpp | 1 + 10 files changed, 34 insertions(+), 31 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 23669d0b5ed..edf7a0e57a4 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -23,7 +23,7 @@ namespace DB::DM { -template +template BitmapFilterPtr buildBitmapFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h index f4b0b2d1486..1a3c803b35a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h @@ -27,7 +27,7 @@ using RowKeyRanges = std::vector; class BitmapFilter; using BitmapFilterPtr = std::shared_ptr; -template +template BitmapFilterPtr buildBitmapFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h b/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h index e249eadbe96..bc4004ecd44 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h @@ -14,8 +14,10 @@ #pragma once -#include - +//#include +#include +#include +#include namespace DB::DM { template @@ -58,12 +60,12 @@ class ColumnView , pos(pos) {} - StringRef operator*() const + std::string_view operator*() const { assert(offsets[-1] == 0); const auto off = offsets[pos - 1]; const auto size = offsets[pos] - offsets[pos - 1] - 1; - return StringRef(reinterpret_cast(&chars[off]), size); + return std::string_view(reinterpret_cast(&chars[off]), size); } Iterator & operator+(size_t n) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h index bafc7233a88..20e6b13fc5e 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h @@ -31,12 +31,12 @@ static constexpr RowID NotExistRowID = std::numeric_limits::max(); static constexpr RowID UnknownRowID = NotExistRowID - 1; template -concept Int64OrString = std::same_as || std::same_as; +concept HandleType = std::same_as || std::same_as; template -concept Int64OrStringRef = std::same_as || std::same_as; +concept HandleRefType = std::same_as || std::same_as; -template +template ColumnDefine getHandleColumnDefine() { if constexpr (std::is_same_v) @@ -48,7 +48,7 @@ ColumnDefine getHandleColumnDefine() } // For ColumnFileReader -template +template ColumnDefinesPtr getHandleColumnDefinesPtr() { static auto cds_ptr = std::make_shared(1, getHandleColumnDefine()); @@ -67,7 +67,7 @@ inline ColumnDefinesPtr getTagColumnDefinesPtr() return cds_ptr; } -template +template bool inRowKeyRange(const RowKeyRange & range, HandleRef handle) { if constexpr (std::is_same_v) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h index cabadd0a171..11d870b7cfb 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h @@ -25,7 +25,7 @@ namespace DB::DM { -template +template class DMFileHandleIndex { public: @@ -53,7 +53,7 @@ class DMFileHandleIndex clipped_pack_range.count()); } - template + template std::optional getBaseVersion(const DMContext & dm_context, HandleRef h) { auto clipped_pack_id = getClippedPackId(h); @@ -94,7 +94,7 @@ class DMFileHandleIndex } private: - template + template std::optional getClippedPackId(HandleRef h) { if (unlikely(rowkey_range && !inRowKeyRange(*rowkey_range, h))) @@ -106,7 +106,7 @@ class DMFileHandleIndex return itr - clipped_pack_index.begin(); } - template + template std::optional getBaseVersion(const DMContext & dm_context, HandleRef h, UInt32 clipped_pack_id) { loadHandleIfNotLoaded(dm_context); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index 6695a226761..a2448e04852 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -26,7 +26,7 @@ namespace { // TODO: shrinking read_range by segment_range -template +template UInt32 buildRowKeyFilterVector( const PaddedPODArray & handles, const RowKeyRanges & delete_ranges, @@ -48,7 +48,7 @@ UInt32 buildRowKeyFilterVector( return handles.size(); } -template +template UInt32 buildRowKeyFilterBlock( const DMContext & dm_context, const IColumnFileDataProviderPtr & data_provider, @@ -72,7 +72,7 @@ UInt32 buildRowKeyFilterBlock( return buildRowKeyFilterVector(handles, delete_ranges, read_ranges, start_row_id, filter); } -template +template UInt32 buildRowKeyFilterDMFile( const DMContext & dm_context, const DMFilePtr & dmfile, @@ -159,7 +159,7 @@ UInt32 buildRowKeyFilterDMFile( return rows; } -template +template UInt32 buildRowKeyFilterColumnFileBig( const DMContext & dm_context, const ColumnFileBig & cf_big, @@ -181,7 +181,7 @@ UInt32 buildRowKeyFilterColumnFileBig( filter); } -template +template UInt32 buildRowKeyFilterStable( const DMContext & dm_context, const StableValueSpace::Snapshot & stable, @@ -212,7 +212,7 @@ void buildRowKeyFilterDeleteRange(const ColumnFileDeleteRange & cf_delete_range, } } // namespace -template +template void buildRowKeyFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h index 2e440758fe3..f5d6cba5493 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h @@ -23,7 +23,7 @@ struct SegmentSnapshot; struct RowKeyRange; using RowKeyRanges = std::vector; -template +template void buildRowKeyFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index 8ffc07b6087..bf762577a89 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -20,7 +20,7 @@ namespace DB::DM { -template +template std::shared_ptr> VersionChain::replaySnapshot( const DMContext & dm_context, const SegmentSnapshot & snapshot) @@ -114,7 +114,7 @@ std::shared_ptr> VersionChain::replaySnapshot( return base_versions; } -template +template UInt32 VersionChain::replayBlock( const DMContext & dm_context, const IColumnFileDataProviderPtr & data_provider, @@ -163,7 +163,7 @@ UInt32 VersionChain::replayBlock( return column.size() - offset; } -template +template UInt32 VersionChain::replayColumnFileBig( const DMContext & dm_context, const ColumnFileBig & cf_big, @@ -178,7 +178,7 @@ UInt32 VersionChain::replayColumnFileBig( return rows; } -template +template UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range) { auto [start, end] = convertRowKeyRange(cf_delete_range.getDeleteRange()); @@ -192,7 +192,7 @@ UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_ return cf_delete_range.getDeletes(); } -template +template std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRangeList( const DMContext & dm_context, Handle h) @@ -215,7 +215,7 @@ std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRang return {}; } -template +template template void VersionChain::calculateReadPacks(Iterator begin, Iterator end) { @@ -224,7 +224,7 @@ void VersionChain::calculateReadPacks(Iterator begin, Iterator end) dmfile_index.calculateReadPacks(begin, end); } -template +template void VersionChain::cleanHandleColumn() { for (auto & dmfile_or_delete_range : dmfile_or_delete_range_list) @@ -234,7 +234,7 @@ void VersionChain::cleanHandleColumn() } } -template +template std::pair VersionChain::convertRowKeyRange(const RowKeyRange & range) { if constexpr (std::is_same_v) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index b76df64c15a..e517869cf30 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -29,7 +29,7 @@ class ColumnFile; class ColumnFileBig; class ColumnFileDeleteRange; -template +template class VersionChain { public: diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp index 8299a0a4431..71a3b3c73af 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp @@ -25,6 +25,7 @@ namespace DB::DM::tests TEST(HandleIndexTest, Basic) { [[maybe_unused]] VersionChain version_chain_int; + //[[maybe_unused]] VersionChain version_chain_str; } TEST(ColumnView, Basic) From cf10dec491ba5d9d9a16c0b8b9455cc5e2f2b7fa Mon Sep 17 00:00:00 2001 From: jinhelin Date: Sat, 11 Jan 2025 00:38:03 +0800 Subject: [PATCH 14/48] ci --- .../DeltaMerge/VersionChain/ColumnView.h | 72 +++++++++++++------ .../DeltaMerge/VersionChain/Common.cpp | 7 +- .../Storages/DeltaMerge/VersionChain/Common.h | 4 +- .../DeltaMerge/VersionChain/VersionChain.cpp | 5 +- .../DeltaMerge/VersionChain/VersionChain.h | 5 +- .../tests/gtest_handle_pack_index.cpp | 11 ++- 6 files changed, 73 insertions(+), 31 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h b/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h index bc4004ecd44..8fffc3ff155 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h @@ -15,8 +15,8 @@ #pragma once //#include -#include #include +#include #include namespace DB::DM { @@ -54,55 +54,85 @@ class ColumnView class Iterator { public: + using iterator_category = std::random_access_iterator_tag; + using value_type = std::string_view; + using difference_type = std::ptrdiff_t; + //using pointer = std::string_view*; + //using reference = std::string_view&; + Iterator(const IColumn::Offsets & offsets, const ColumnString::Chars_t & chars, size_t pos) - : offsets(offsets) - , chars(chars) + : offsets(&offsets) + , chars(&chars) , pos(pos) {} - std::string_view operator*() const + value_type operator*() const { - assert(offsets[-1] == 0); - const auto off = offsets[pos - 1]; - const auto size = offsets[pos] - offsets[pos - 1] - 1; - return std::string_view(reinterpret_cast(&chars[off]), size); + assert((*offsets)[-1] == 0); + const auto off = (*offsets)[pos - 1]; + const auto size = (*offsets)[pos] - (*offsets)[pos - 1] - 1; + return std::string_view(reinterpret_cast(chars->data() + off), size); } - Iterator & operator+(size_t n) + Iterator operator+(difference_type n) { return Iterator{*offsets, *chars, pos + n}; } + + Iterator operator-(difference_type n) { return Iterator{*offsets, *chars, pos - n}; } + + difference_type operator-(const Iterator & other) const { return pos - other.pos; } + + Iterator & operator++() { - pos += n; + ++pos; return *this; } - Iterator & operator-(size_t n) + Iterator & operator--() { - pos -= n; + --pos; return *this; } - size_t operator-(const Iterator & other) const { return pos - other.pos; } + // Postfix increment/decrement is not supported + //Iterator & operator++(int) = delete; + //Iterator & operator--(int) = delete; - Iterator & operator++() + // Postfix increment (optional) + Iterator operator++(int) { + Iterator tmp = *this; ++pos; - return *this; + return tmp; } - Iterator & operator--() + // Postfix decrement (optional) + Iterator operator--(int) { + Iterator tmp = *this; --pos; + return tmp; + } + + // Compound addition + Iterator & operator+=(difference_type n) + { + pos += n; return *this; } - // Postfix increment/decrement is not supported - Iterator & operator++(int) = delete; - Iterator & operator--(int) = delete; + // Compound subtraction + Iterator & operator-=(difference_type n) + { + pos -= n; + return *this; + } bool operator!=(const Iterator & other) const { return pos != other.pos; } + bool operator==(const Iterator & other) const { return pos == other.pos; } + bool operator<(const Iterator & other) const { return pos < other.pos; } private: - const IColumn::Offsets & offsets; - const ColumnString::Chars_t & chars; + const IColumn::Offsets * offsets; + const ColumnString::Chars_t * chars; size_t pos = 0; }; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp index bd121d92d14..a17f464fcfc 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.cpp @@ -43,7 +43,7 @@ T getMaxValue(const MinMaxIndex & minmax_index, size_t i) if constexpr (std::is_same_v) return minmax_index.getIntMinMax(i).second; else if constexpr (std::is_same_v) - return minmax_index.getStringMinMax(i).second; + return minmax_index.getStringMinMax(i).second.toString(); else if constexpr (std::is_same_v) return minmax_index.getUInt64MinMax(i).second; else @@ -107,6 +107,9 @@ template std::vector loadPackMaxValue( const Context & global_context, const DMFile & dmfile, const ColId col_id); -//template std::vector loadPackMaxValue(const Context & global_context, const DMFile & dmfile, const ColId col_id); +template std::vector loadPackMaxValue( + const Context & global_context, + const DMFile & dmfile, + const ColId col_id); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h index 20e6b13fc5e..fec9a87facf 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h @@ -70,10 +70,10 @@ inline ColumnDefinesPtr getTagColumnDefinesPtr() template bool inRowKeyRange(const RowKeyRange & range, HandleRef handle) { - if constexpr (std::is_same_v) + if constexpr (std::is_same_v) return range.start.int_value <= handle && handle < range.end.int_value; else - static_assert(false, "TODO: support common handle"); + return *(range.start.value) <= handle && handle < *(range.end.value); } RSResults getRSResultsByRanges(const DMContext & dm_context, const DMFilePtr & dmfile, const RowKeyRanges & ranges); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index bf762577a89..8a792baafdd 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -193,9 +193,10 @@ UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_ } template +template std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRangeList( const DMContext & dm_context, - Handle h) + HandleRef h) { // From from new to old for (auto itr = dmfile_or_delete_range_list.rbegin(); itr != dmfile_or_delete_range_list.rend(); ++itr) @@ -244,6 +245,6 @@ std::pair VersionChain::convertRowKeyRange(const RowKeyR } template class VersionChain; -//template class VersionChain; +template class VersionChain; } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index e517869cf30..d0ba3e3439a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -69,9 +69,10 @@ class VersionChain const UInt32 stable_rows); [[nodiscard]] UInt32 replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range); + template [[nodiscard]] std::optional findBaseVersionFromDMFileOrDeleteRangeList( const DMContext & dm_context, - Handle h); + HandleRef h); template void calculateReadPacks(Iterator begin, Iterator end); void cleanHandleColumn(); @@ -81,7 +82,7 @@ class VersionChain std::mutex mtx; UInt32 replayed_rows_and_deletes = 0; // delta.getRows() + delta.getDeletes() std::shared_ptr> base_versions; // base_versions->size() == delta.getRows() - std::map new_handle_to_row_ids; + std::map> new_handle_to_row_ids; using DMFileOrDeleteRange = std::variant>; std::vector dmfile_or_delete_range_list; }; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp index 71a3b3c73af..cb28b606da4 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_handle_pack_index.cpp @@ -25,14 +25,21 @@ namespace DB::DM::tests TEST(HandleIndexTest, Basic) { [[maybe_unused]] VersionChain version_chain_int; - //[[maybe_unused]] VersionChain version_chain_str; + [[maybe_unused]] VersionChain version_chain_str; } TEST(ColumnView, Basic) { auto str_col = ColumnGenerator::instance().generate({1024, "String", RANDOM}).column; + + //auto str_col = makeColumn(MutSup::getExtraHandleColumnStringType(), {"0000", "11", "222", "33333"}); ColumnView str_cv(*str_col); for (auto s_itr = str_cv.begin(); s_itr != str_cv.end(); ++s_itr) - ASSERT_EQ(*s_itr, str_col->getDataAt(s_itr - str_cv.begin())); + { + ASSERT_EQ(*s_itr, str_col->getDataAt(s_itr - str_cv.begin()).toStringView()); + } + //fmt::println("======================="); + //auto itr = std::lower_bound(str_cv.begin(), str_cv.end(), "hello"); + //std::ignore = itr; } } // namespace DB::DM::tests From 6be97b181009dc9e8b3f23e243a69b8b80ad1f4b Mon Sep 17 00:00:00 2001 From: jinhelin Date: Sat, 11 Jan 2025 00:40:30 +0800 Subject: [PATCH 15/48] ci --- .../Storages/DeltaMerge/VersionChain/ColumnView.h | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h b/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h index 8fffc3ff155..f265ed1f6ad 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/ColumnView.h @@ -57,8 +57,6 @@ class ColumnView using iterator_category = std::random_access_iterator_tag; using value_type = std::string_view; using difference_type = std::ptrdiff_t; - //using pointer = std::string_view*; - //using reference = std::string_view&; Iterator(const IColumn::Offsets & offsets, const ColumnString::Chars_t & chars, size_t pos) : offsets(&offsets) @@ -92,11 +90,6 @@ class ColumnView return *this; } - // Postfix increment/decrement is not supported - //Iterator & operator++(int) = delete; - //Iterator & operator--(int) = delete; - - // Postfix increment (optional) Iterator operator++(int) { Iterator tmp = *this; @@ -104,7 +97,6 @@ class ColumnView return tmp; } - // Postfix decrement (optional) Iterator operator--(int) { Iterator tmp = *this; @@ -112,14 +104,12 @@ class ColumnView return tmp; } - // Compound addition Iterator & operator+=(difference_type n) { pos += n; return *this; } - // Compound subtraction Iterator & operator-=(difference_type n) { pos -= n; @@ -131,7 +121,7 @@ class ColumnView bool operator<(const Iterator & other) const { return pos < other.pos; } private: - const IColumn::Offsets * offsets; + const IColumn::Offsets * offsets; // Using pointer for operator= const ColumnString::Chars_t * chars; size_t pos = 0; }; From c6f982f900acf39020266cb7b743fb86014de0d6 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Mon, 13 Jan 2025 15:17:27 +0800 Subject: [PATCH 16/48] ci --- .../VersionChain/BuildBitmapFilter.cpp | 3 +- .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 79 +++++++++---------- .../DeltaMerge/VersionChain/RowKeyFilter.h | 1 + 3 files changed, 41 insertions(+), 42 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index edf7a0e57a4..3068dc20024 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -41,10 +41,9 @@ BitmapFilterPtr buildBitmapFilter( auto bitmap_filter = std::make_shared(total_rows, true); auto & filter = bitmap_filter->getFilter(); - // TODO: make these functions return filter out rows. - // TODO: send the pack res to buildVersionFilter and buildDeletedFilter to skip some packs. RUNTIME_CHECK(pack_filter_results.size() == 1, pack_filter_results.size()); const auto stable_pack_res = pack_filter_results.front()->getPackRes(); + buildRowKeyFilter(dm_context, snapshot, read_ranges, stable_pack_res, filter); buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); buildDeletedFilter(dm_context, snapshot, filter); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index a2448e04852..b2a05c13367 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -28,24 +28,25 @@ namespace // TODO: shrinking read_range by segment_range template UInt32 buildRowKeyFilterVector( - const PaddedPODArray & handles, + const ColumnView & handles, const RowKeyRanges & delete_ranges, const RowKeyRanges & read_ranges, const UInt32 start_row_id, IColumn::Filter & filter) { - for (UInt32 i = 0; i < handles.size(); ++i) + for (auto itr = handles.begin(); itr != handles.end(); ++itr) { - auto in_range = [h = handles[i]](const RowKeyRange & range) { + auto in_range = [h = *itr](const RowKeyRange & range) { return inRowKeyRange(range, h); }; + // IN delete_ranges or NOT IN read_ranges if (std::any_of(delete_ranges.begin(), delete_ranges.end(), in_range) || std::none_of(read_ranges.begin(), read_ranges.end(), in_range)) { - filter[start_row_id + i] = 0; + filter[itr - handles.begin() + start_row_id] = 0; } } - return handles.size(); + return handles.end() - handles.begin(); } template @@ -60,15 +61,19 @@ UInt32 buildRowKeyFilterBlock( { assert(cf.isInMemoryFile() || cf.isTinyFile()); - if (cf.getRows() == 0) + const auto rows = cf.getRows(); + if (unlikely(rows == 0)) return 0; auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); auto block = cf_reader->readNextBlock(); - RUNTIME_CHECK_MSG(!cf_reader->readNextBlock(), "{}: MUST read all rows in one block!", cf.toString()); - const auto * handles_ptr = toColumnVectorDataPtr(block.begin()->column); - RUNTIME_CHECK_MSG(handles_ptr != nullptr, "TODO: support common handle"); - const auto & handles = *handles_ptr; + RUNTIME_CHECK_MSG( + rows == block.rows(), + "ColumnFile<{}> returns {} rows. Read all rows in one block is required!", + cf.toString(), + block.rows()); + + const auto handles = ColumnView(*(block.begin()->column)); return buildRowKeyFilterVector(handles, delete_ranges, read_ranges, start_row_id, filter); } @@ -84,21 +89,20 @@ UInt32 buildRowKeyFilterDMFile( IColumn::Filter & filter) { auto [valid_handle_res, valid_start_pack_id] = getClippedRSResultsByRanges(dm_context, dmfile, segment_range); - if (valid_handle_res.empty()) + if (unlikely(valid_handle_res.empty())) return 0; if (stable_pack_res) { + // Only use the None result of stable_pack_res const auto & s_pack_res = *stable_pack_res; - RUNTIME_CHECK( - stable_pack_res->size() == valid_handle_res.size(), - stable_pack_res->size(), - valid_handle_res.size()); + RUNTIME_CHECK(s_pack_res.size() == valid_handle_res.size(), s_pack_res.size(), valid_handle_res.size()); for (UInt32 i = 0; i < valid_handle_res.size(); ++i) if (!s_pack_res[i].isUse()) valid_handle_res[i] = RSResult::None; } + // Seems stable_pack_res is the result of read_ranges and ... const auto read_ranges_handle_res = getRSResultsByRanges(dm_context, dmfile, read_ranges); for (UInt32 i = 0; i < valid_handle_res.size(); ++i) valid_handle_res[i] = valid_handle_res[i] && read_ranges_handle_res[valid_start_pack_id + i]; @@ -110,53 +114,52 @@ UInt32 buildRowKeyFilterDMFile( valid_handle_res[i] = valid_handle_res[i] && !delete_ranges_handle_res[valid_start_pack_id + i]; } - auto read_packs = std::make_shared(); + auto need_read_packs = std::make_shared(); UInt32 need_read_rows = 0; - std::unordered_map read_pack_to_start_row_ids; + std::unordered_map need_read_pack_to_start_row_ids; const auto & pack_stats = dmfile->getPackStats(); - UInt32 rows = 0; + UInt32 processed_rows = 0; for (UInt32 i = 0; i < valid_handle_res.size(); ++i) { const auto pack_id = valid_start_pack_id + i; if (!valid_handle_res[i].isUse()) { - std::fill_n(filter.begin() + start_row_id + rows, pack_stats[pack_id].rows, false); + std::fill_n(filter.begin() + start_row_id + processed_rows, pack_stats[pack_id].rows, false); } else if (!valid_handle_res[i].allMatch()) { - read_packs->insert(pack_id); - read_pack_to_start_row_ids.emplace(pack_id, start_row_id + rows); + need_read_packs->insert(pack_id); + need_read_pack_to_start_row_ids.emplace(pack_id, start_row_id + processed_rows); need_read_rows += pack_stats[pack_id].rows; } - rows += pack_stats[pack_id].rows; + processed_rows += pack_stats[pack_id].rows; } if (need_read_rows == 0) - return rows; + return processed_rows; DMFileBlockInputStreamBuilder builder(dm_context.global_context); - builder.onlyReadOnePackEveryTime().setReadPacks(read_packs).setReadTag(ReadTag::MVCC); - auto stream = builder.build(dmfile, {getHandleColumnDefine()}, {}, dm_context.scan_context); + builder.onlyReadOnePackEveryTime().setReadPacks(need_read_packs).setReadTag(ReadTag::MVCC); + auto stream + = builder.build(dmfile, {getHandleColumnDefine()}, /*rowkey_ranges*/ {}, dm_context.scan_context); UInt32 read_rows = 0; - for (auto pack_id : *read_packs) + for (auto pack_id : *need_read_packs) { auto block = stream->read(); - const auto * handles_ptr = toColumnVectorDataPtr(block.begin()->column); - RUNTIME_CHECK_MSG(handles_ptr != nullptr, "TODO: support common handle"); - const auto & handles = *handles_ptr; - - const auto itr = read_pack_to_start_row_ids.find(pack_id); - RUNTIME_CHECK(itr != read_pack_to_start_row_ids.end(), read_pack_to_start_row_ids, pack_id); + RUNTIME_CHECK(block.rows() == pack_stats[pack_id].rows, block.rows(), pack_stats[pack_id].rows); + const auto handles = ColumnView(*(block.begin()->column)); + const auto itr = need_read_pack_to_start_row_ids.find(pack_id); + RUNTIME_CHECK(itr != need_read_pack_to_start_row_ids.end(), need_read_pack_to_start_row_ids, pack_id); read_rows += buildRowKeyFilterVector( handles, delete_ranges, read_ranges, - itr->second, // start_row_id + /*start_row_id*/ itr->second, filter); } RUNTIME_CHECK(read_rows == need_read_rows, read_rows, need_read_rows); - return rows; + return processed_rows; } template @@ -206,10 +209,6 @@ UInt32 buildRowKeyFilterStable( filter); } -void buildRowKeyFilterDeleteRange(const ColumnFileDeleteRange & cf_delete_range, RowKeyRanges & delete_ranges) -{ - delete_ranges.push_back(cf_delete_range.getDeleteRange()); -} } // namespace template @@ -231,13 +230,13 @@ void buildRowKeyFilter( const auto & data_provider = delta.getDataProvider(); RowKeyRanges delete_ranges; UInt32 read_rows = 0; - // Read from new cfs to old cfs to handle delete range + // Read ColumnFiles from new to old for handling delete ranges for (auto itr = cfs.rbegin(); itr != cfs.rend(); ++itr) { const auto & cf = *itr; if (const auto * cf_delete_range = cf->tryToDeleteRange(); cf_delete_range) { - buildRowKeyFilterDeleteRange(*cf_delete_range, delete_ranges); + delete_ranges.push_back(cf_delete_range->getDeleteRange()); continue; } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h index f5d6cba5493..3cf7f55f2aa 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h @@ -16,6 +16,7 @@ #include #include + namespace DB::DM { struct DMContext; From 4a58a501e42bcb1091e0a992bf3e74e997def6bf Mon Sep 17 00:00:00 2001 From: jinhelin Date: Mon, 13 Jan 2025 16:08:17 +0800 Subject: [PATCH 17/48] ci --- .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index b2a05c13367..8ede9123494 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -24,8 +24,6 @@ namespace DB::DM { namespace { - -// TODO: shrinking read_range by segment_range template UInt32 buildRowKeyFilterVector( const ColumnView & handles, @@ -94,19 +92,24 @@ UInt32 buildRowKeyFilterDMFile( if (stable_pack_res) { - // Only use the None result of stable_pack_res const auto & s_pack_res = *stable_pack_res; RUNTIME_CHECK(s_pack_res.size() == valid_handle_res.size(), s_pack_res.size(), valid_handle_res.size()); + // stable_pack_res is the result of `read_ranges` && `rs_filter`. + // If the result of a pack filtered by read_ranges is All, but filtered by `filter` is Some, its result + // is Some in stable_pack_res. + // Result of `rs_filter` is no help here, because we just want to filter out rowkey here. + // So only use the None results of stable_pack_res for (UInt32 i = 0; i < valid_handle_res.size(); ++i) if (!s_pack_res[i].isUse()) valid_handle_res[i] = RSResult::None; } - // Seems stable_pack_res is the result of read_ranges and ... + // RSResult of read_ranges. const auto read_ranges_handle_res = getRSResultsByRanges(dm_context, dmfile, read_ranges); for (UInt32 i = 0; i < valid_handle_res.size(); ++i) valid_handle_res[i] = valid_handle_res[i] && read_ranges_handle_res[valid_start_pack_id + i]; + // RSResult of delete_ranges. if (!delete_ranges.empty()) { const auto delete_ranges_handle_res = getRSResultsByRanges(dm_context, dmfile, delete_ranges); @@ -179,7 +182,7 @@ UInt32 buildRowKeyFilterColumnFileBig( cf_big.getRange(), delete_ranges, Segment::shrinkRowKeyRanges(cf_big.getRange(), read_ranges), - nullptr, // stable_pack_res + /*stable_pack_res*/ nullptr, start_row_id, filter); } @@ -196,16 +199,16 @@ UInt32 buildRowKeyFilterStable( const auto & dmfiles = stable.getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); const auto & dmfile = dmfiles[0]; - if (dmfile->getPacks() == 0) + if (unlikely(dmfile->getPacks() == 0)) return 0; return buildRowKeyFilterDMFile( dm_context, dmfile, - std::nullopt, // segment_range + /*segment_range*/ std::nullopt, delete_ranges, read_ranges, &stable_pack_res, - 0, // start_row_id + /*start_row_id*/ 0, filter); } From 2bd5e64a61c3a0c1d53cbd3abd4f4b12e018ba07 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Mon, 13 Jan 2025 16:12:27 +0800 Subject: [PATCH 18/48] ci --- dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index 8ede9123494..9e3decd69a1 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -290,5 +290,10 @@ template void buildRowKeyFilter( const RSResults & stable_pack_res, IColumn::Filter & filter); -// TODO: String +template void buildRowKeyFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const RSResults & stable_pack_res, + IColumn::Filter & filter); } // namespace DB::DM From 138e328a2c064e0b8a7de37f95ee760f84f0aa1a Mon Sep 17 00:00:00 2001 From: jinhelin Date: Mon, 13 Jan 2025 17:53:05 +0800 Subject: [PATCH 19/48] ci --- .../VersionChain/BuildBitmapFilter.cpp | 2 +- .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 6 +- .../DeltaMerge/VersionChain/VersionFilter.cpp | 115 ++++++++++-------- .../DeltaMerge/VersionChain/VersionFilter.h | 3 + 4 files changed, 73 insertions(+), 53 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 3068dc20024..74b67feff4f 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -45,7 +45,7 @@ BitmapFilterPtr buildBitmapFilter( const auto stable_pack_res = pack_filter_results.front()->getPackRes(); buildRowKeyFilter(dm_context, snapshot, read_ranges, stable_pack_res, filter); - buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); + buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); buildDeletedFilter(dm_context, snapshot, filter); bitmap_filter->runOptimize(); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index 9e3decd69a1..69ded74e757 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -58,10 +58,8 @@ UInt32 buildRowKeyFilterBlock( IColumn::Filter & filter) { assert(cf.isInMemoryFile() || cf.isTinyFile()); - - const auto rows = cf.getRows(); - if (unlikely(rows == 0)) - return 0; + const UInt32 rows = cf.getRows(); + assert(rows > 0); auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); auto block = cf_reader->readNextBlock(); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp index f6048f86b76..4feea636018 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp @@ -35,43 +35,46 @@ namespace DB::DM assert(cf.isInMemoryFile() || cf.isTinyFile()); auto cf_reader = cf.getReader(dm_context, data_provider, getVersionColumnDefinesPtr(), ReadTag::MVCC); auto block = cf_reader->readNextBlock(); + RUNTIME_CHECK_MSG( + cf.getRows() == block.rows(), + "ColumnFile<{}> returns {} rows. Read all rows in one block is required!", + cf.toString(), + block.rows()); if (!block) return 0; - RUNTIME_CHECK_MSG(!cf_reader->readNextBlock(), "{}: read all rows in one block is required!", cf.toString()); - const auto & versions = *toColumnVectorDataPtr(block.begin()->column); // Must success. + const auto & versions = *toColumnVectorDataPtr(block.begin()->column); // Must success // Traverse data from new to old for (ssize_t i = versions.size() - 1; i >= 0; --i) { const UInt32 row_id = start_row_id + i; - + // Already filtered ou, maybe by RowKeyFilter if (!filter[row_id]) continue; - // invisible + // Invisible if (versions[i] > read_ts) { filter[row_id] = 0; continue; } - // visible - + // Visible const auto base_row_id = base_ver_snap[row_id - stable_rows]; - // Newer version has been chosen. + // Newer version has been chosen if (base_row_id != NotExistRowID && !filter[base_row_id]) { filter[row_id] = 0; continue; } - // Choose this version + // Choose this version. If has based version, filter it out. if (base_row_id != NotExistRowID) filter[base_row_id] = 0; } return versions.size(); } - +template [[nodiscard]] UInt32 buildVersionFilterDMFile( const DMContext & dm_context, const DMFilePtr & dmfile, @@ -86,34 +89,34 @@ namespace DB::DM const auto max_versions = loadPackMaxValue(dm_context.global_context, *dmfile, MutSup::version_col_id); - auto read_packs = std::make_shared(); + auto need_read_packs = std::make_shared(); UInt32 need_read_rows = 0; - std::unordered_map read_pack_to_start_row_ids; + std::unordered_map need_read_pack_to_start_row_ids; const auto & pack_stats = dmfile->getPackStats(); - UInt32 rows = 0; + UInt32 processed_rows = 0; for (UInt32 i = 0; i < valid_handle_res.size(); ++i) { const UInt32 pack_id = valid_start_pack_id + i; - const UInt32 pack_start_row_id = start_row_id + rows; + const UInt32 pack_start_row_id = start_row_id + processed_rows; const auto & stat = pack_stats[pack_id]; if (stat.not_clean || max_versions[pack_id] > read_ts) { - read_packs->insert(pack_id); - read_pack_to_start_row_ids.emplace(pack_id, pack_start_row_id); + need_read_packs->insert(pack_id); + need_read_pack_to_start_row_ids.emplace(pack_id, pack_start_row_id); need_read_rows += stat.rows; } - rows += stat.rows; + processed_rows += stat.rows; } if (need_read_rows == 0) - return rows; + return processed_rows; // If all packs need to read is clean, we can just read version column. // However, the benefits in general scenarios may not be significant. - + // For simplicity, read handle column and version column directly. DMFileBlockInputStreamBuilder builder(dm_context.global_context); - builder.onlyReadOnePackEveryTime().setReadPacks(read_packs).setReadTag(ReadTag::MVCC); + builder.onlyReadOnePackEveryTime().setReadPacks(need_read_packs).setReadTag(ReadTag::MVCC); auto stream = builder.build( dmfile, {getHandleColumnDefine(), getVersionColumnDefine()}, @@ -121,52 +124,43 @@ namespace DB::DM dm_context.scan_context); UInt32 read_rows = 0; - for (auto pack_id : *read_packs) + for (auto pack_id : *need_read_packs) { auto block = stream->read(); RUNTIME_CHECK(block.rows() == pack_stats[pack_id].rows, block.rows(), pack_stats[pack_id].rows); read_rows += block.rows(); - const auto * handles_ptr - = toColumnVectorDataPtr(block.getByName(MutSup::extra_handle_column_name).column); - RUNTIME_CHECK_MSG(handles_ptr != nullptr, "TODO: support common handle"); - const auto & handles = *handles_ptr; - const auto & versions - = *toColumnVectorDataPtr(block.getByName(MutSup::version_column_name).column); // Must success. - - const auto itr = read_pack_to_start_row_ids.find(pack_id); - RUNTIME_CHECK(itr != read_pack_to_start_row_ids.end(), read_pack_to_start_row_ids, pack_id); + const auto handles = ColumnView(*(block.getByPosition(0).column)); + const auto & versions = *toColumnVectorDataPtr(block.getByPosition(1).column); // Must success. + const auto itr = need_read_pack_to_start_row_ids.find(pack_id); + RUNTIME_CHECK(itr != need_read_pack_to_start_row_ids.end(), need_read_pack_to_start_row_ids, pack_id); const UInt32 pack_start_row_id = itr->second; // Filter invisible versions if (max_versions[pack_id] > read_ts) { for (UInt32 i = 0; i < block.rows(); ++i) - { - // TODO: benchmark - // filter[pack_start_row_id + i] = versions[i] <= read_ts; - if unlikely (versions[i] > read_ts) - filter[pack_start_row_id + i] = 0; - } + filter[pack_start_row_id + i] = versions[i] <= read_ts; } // Filter multiple versions if (pack_stats[pack_id].not_clean) { - // [handle_itr, handle_end) is a pack. auto handle_itr = handles.begin(); - const auto handle_end = handle_itr + pack_stats[pack_id].rows; + auto handle_end = handles.end(); for (;;) { + // Search for the first two consecutive equal elements auto itr = std::adjacent_find(handle_itr, handle_end); if (itr == handle_end) break; // Let `handle_itr` point to next different handle. - handle_itr = std::find_if(itr, handle_end, [h = *itr](Int64 a) { return h != a; }); + handle_itr = std::find_if(itr, handle_end, [h = *itr](auto a) { return h != a; }); // [itr, handle_itr) are the same handle of different verions. - auto count = std::distance(itr, handle_itr); + auto count = handle_itr - itr; + RUNTIME_CHECK(count >= 2, count); - const UInt32 base_row_id = pack_start_row_id + std::distance(handles.begin(), itr); + const UInt32 base_row_id = itr - handles.begin() + pack_start_row_id; if (!filter[base_row_id]) { std::fill_n(filter.begin() + base_row_id + 1, count - 1, 0); @@ -174,6 +168,9 @@ namespace DB::DM } else { + // Find the newest but not filtered out version. + // If it is invisiable to `read_ts`, it already filtered out before. + // So we just get the last not filtered out version here. for (UInt32 i = 1; i < count; ++i) { if (filter[base_row_id + i]) @@ -186,9 +183,10 @@ namespace DB::DM } } RUNTIME_CHECK(read_rows == need_read_rows, read_rows, need_read_rows); - return rows; + return processed_rows; } +template [[nodiscard]] UInt32 buildVersionFilterColumnFileBig( const DMContext & dm_context, const ColumnFileBig & cf_big, @@ -196,9 +194,16 @@ namespace DB::DM const ssize_t start_row_id, IColumn::Filter & filter) { - return buildVersionFilterDMFile(dm_context, cf_big.getFile(), cf_big.getRange(), read_ts, start_row_id, filter); + return buildVersionFilterDMFile( + dm_context, + cf_big.getFile(), + cf_big.getRange(), + read_ts, + start_row_id, + filter); } +template [[nodiscard]] UInt32 buildVersionFilterStable( const DMContext & dm_context, const StableValueSpace::Snapshot & stable, @@ -207,9 +212,10 @@ namespace DB::DM { const auto & dmfiles = stable.getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); - return buildVersionFilterDMFile(dm_context, dmfiles[0], std::nullopt, read_ts, 0, filter); + return buildVersionFilterDMFile(dm_context, dmfiles[0], std::nullopt, read_ts, 0, filter); } +template void buildVersionFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, @@ -229,12 +235,11 @@ void buildVersionFilter( const auto & data_provider = delta.getDataProvider(); UInt32 read_rows = 0; - - // Traverse data from new to old + // Read versions from new to old for (auto itr = cfs.rbegin(); itr != cfs.rend(); ++itr) { const auto & cf = *itr; - if (cf->isDeleteRange()) + if (cf->isDeleteRange()) // Delete range is handled by RowKeyFilter. continue; const UInt32 cf_rows = cf->getRows(); @@ -260,14 +265,28 @@ void buildVersionFilter( if (const auto * cf_big = cf->tryToBigFile(); cf_big) { - const auto n = buildVersionFilterColumnFileBig(dm_context, *cf_big, read_ts, start_row_id, filter); + const auto n = buildVersionFilterColumnFileBig(dm_context, *cf_big, read_ts, start_row_id, filter); RUNTIME_CHECK(cf_rows == n, cf_rows, n); continue; } RUNTIME_CHECK_MSG(false, "{}: unknow ColumnFile type", cf->toString()); } RUNTIME_CHECK(read_rows == delta_rows, read_rows, delta_rows); - const auto n = buildVersionFilterStable(dm_context, stable, read_ts, filter); + const auto n = buildVersionFilterStable(dm_context, stable, read_ts, filter); RUNTIME_CHECK(n == stable_rows, n, stable_rows); } + +template void buildVersionFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const std::vector & base_ver_snap, + const UInt64 read_ts, + IColumn::Filter & filter); + +template void buildVersionFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const std::vector & base_ver_snap, + const UInt64 read_ts, + IColumn::Filter & filter); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h index 53c074f963a..f79576a0819 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h @@ -16,10 +16,13 @@ #include #include + namespace DB::DM { struct DMContext; struct SegmentSnapshot; + +template void buildVersionFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, From 7e353c1eaad7fdaa5e55fb37f068be435c7af519 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Mon, 13 Jan 2025 18:01:47 +0800 Subject: [PATCH 20/48] ci --- .../VersionChain/BuildBitmapFilter.cpp | 10 ++-- .../VersionChain/BuildBitmapFilter.h | 4 +- .../Storages/DeltaMerge/VersionChain/Common.h | 12 ++--- .../VersionChain/DMFileHandleIndex.h | 16 +++---- .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 34 ++++++------- .../DeltaMerge/VersionChain/RowKeyFilter.h | 2 +- .../DeltaMerge/VersionChain/VersionChain.cpp | 48 +++++++++---------- .../DeltaMerge/VersionChain/VersionChain.h | 8 ++-- .../DeltaMerge/VersionChain/VersionFilter.cpp | 21 ++++---- .../DeltaMerge/VersionChain/VersionFilter.h | 2 +- 10 files changed, 79 insertions(+), 78 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 74b67feff4f..41fe8359606 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -23,14 +23,14 @@ namespace DB::DM { -template +template BitmapFilterPtr buildBitmapFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, - VersionChain & version_chain) + VersionChain & version_chain) { const auto base_ver_snap = version_chain.replaySnapshot(dm_context, snapshot); const auto & delta = *(snapshot.delta); @@ -44,8 +44,8 @@ BitmapFilterPtr buildBitmapFilter( RUNTIME_CHECK(pack_filter_results.size() == 1, pack_filter_results.size()); const auto stable_pack_res = pack_filter_results.front()->getPackRes(); - buildRowKeyFilter(dm_context, snapshot, read_ranges, stable_pack_res, filter); - buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); + buildRowKeyFilter(dm_context, snapshot, read_ranges, stable_pack_res, filter); + buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); buildDeletedFilter(dm_context, snapshot, filter); bitmap_filter->runOptimize(); @@ -58,5 +58,5 @@ template BitmapFilterPtr buildBitmapFilter( const RowKeyRanges & read_ranges, const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, - VersionChain & version_chain); + VersionChain & version_chain); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h index 1a3c803b35a..07e3e799a44 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h @@ -27,12 +27,12 @@ using RowKeyRanges = std::vector; class BitmapFilter; using BitmapFilterPtr = std::shared_ptr; -template +template BitmapFilterPtr buildBitmapFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, - VersionChain & version_chain); + VersionChain & version_chain); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h index fec9a87facf..b8468389786 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/Common.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/Common.h @@ -31,27 +31,27 @@ static constexpr RowID NotExistRowID = std::numeric_limits::max(); static constexpr RowID UnknownRowID = NotExistRowID - 1; template -concept HandleType = std::same_as || std::same_as; +concept ExtraHandleType = std::same_as || std::same_as; template concept HandleRefType = std::same_as || std::same_as; -template +template ColumnDefine getHandleColumnDefine() { - if constexpr (std::is_same_v) + if constexpr (std::is_same_v) return getExtraIntHandleColumnDefine(); - else if constexpr (std::is_same_v) + else if constexpr (std::is_same_v) return getExtraStringHandleColumnDefine(); else static_assert(false, "Not support type"); } // For ColumnFileReader -template +template ColumnDefinesPtr getHandleColumnDefinesPtr() { - static auto cds_ptr = std::make_shared(1, getHandleColumnDefine()); + static auto cds_ptr = std::make_shared(1, getHandleColumnDefine()); return cds_ptr; } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h index 11d870b7cfb..fe43a1c4d26 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DMFileHandleIndex.h @@ -25,7 +25,7 @@ namespace DB::DM { -template +template class DMFileHandleIndex { public: @@ -110,17 +110,17 @@ class DMFileHandleIndex std::optional getBaseVersion(const DMContext & dm_context, HandleRef h, UInt32 clipped_pack_id) { loadHandleIfNotLoaded(dm_context); - ColumnView handle_col(*clipped_handle_packs[clipped_pack_id]); + ColumnView handle_col(*clipped_handle_packs[clipped_pack_id]); auto itr = std::lower_bound(handle_col.begin(), handle_col.end(), h); if (itr != handle_col.end() && *itr == h) return itr - handle_col.begin() + clipped_pack_offsets[clipped_pack_id]; return {}; } - std::vector loadPackIndex(const DMContext & dm_context) + std::vector loadPackIndex(const DMContext & dm_context) { - auto max_values = loadPackMaxValue(dm_context.global_context, *dmfile, MutSup::extra_handle_id); - return std::vector( + auto max_values = loadPackMaxValue(dm_context.global_context, *dmfile, MutSup::extra_handle_id); + return std::vector( max_values.begin() + clipped_pack_range.start_pack_id, max_values.begin() + clipped_pack_range.end_pack_id); } @@ -137,7 +137,7 @@ class DMFileHandleIndex return pack_offsets; } - static bool isCommonHandle() { return std::is_same_v; } + static bool isCommonHandle() { return std::is_same_v; } void loadHandleIfNotLoaded(const DMContext & dm_context) { @@ -162,7 +162,7 @@ class DMFileHandleIndex DMFileReader reader( dmfile, - {getHandleColumnDefine()}, + {getHandleColumnDefine()}, isCommonHandle(), /*enable_handle_clean_read*/ false, /*enable_del_clean_read*/ false, @@ -217,7 +217,7 @@ class DMFileHandleIndex // Clipped by rowkey_range const PackRange clipped_pack_range; - const std::vector clipped_pack_index; // max value of each pack + const std::vector clipped_pack_index; // max value of each pack const std::vector clipped_pack_offsets; // offset of each pack std::vector clipped_handle_packs; // handle column of each pack std::optional> clipped_need_read_packs; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index 69ded74e757..0aa449f870a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -24,9 +24,9 @@ namespace DB::DM { namespace { -template +template UInt32 buildRowKeyFilterVector( - const ColumnView & handles, + const ColumnView & handles, const RowKeyRanges & delete_ranges, const RowKeyRanges & read_ranges, const UInt32 start_row_id, @@ -47,7 +47,7 @@ UInt32 buildRowKeyFilterVector( return handles.end() - handles.begin(); } -template +template UInt32 buildRowKeyFilterBlock( const DMContext & dm_context, const IColumnFileDataProviderPtr & data_provider, @@ -61,7 +61,7 @@ UInt32 buildRowKeyFilterBlock( const UInt32 rows = cf.getRows(); assert(rows > 0); - auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); + auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); auto block = cf_reader->readNextBlock(); RUNTIME_CHECK_MSG( rows == block.rows(), @@ -69,11 +69,11 @@ UInt32 buildRowKeyFilterBlock( cf.toString(), block.rows()); - const auto handles = ColumnView(*(block.begin()->column)); - return buildRowKeyFilterVector(handles, delete_ranges, read_ranges, start_row_id, filter); + const auto handles = ColumnView(*(block.begin()->column)); + return buildRowKeyFilterVector(handles, delete_ranges, read_ranges, start_row_id, filter); } -template +template UInt32 buildRowKeyFilterDMFile( const DMContext & dm_context, const DMFilePtr & dmfile, @@ -143,13 +143,13 @@ UInt32 buildRowKeyFilterDMFile( DMFileBlockInputStreamBuilder builder(dm_context.global_context); builder.onlyReadOnePackEveryTime().setReadPacks(need_read_packs).setReadTag(ReadTag::MVCC); auto stream - = builder.build(dmfile, {getHandleColumnDefine()}, /*rowkey_ranges*/ {}, dm_context.scan_context); + = builder.build(dmfile, {getHandleColumnDefine()}, /*rowkey_ranges*/ {}, dm_context.scan_context); UInt32 read_rows = 0; for (auto pack_id : *need_read_packs) { auto block = stream->read(); RUNTIME_CHECK(block.rows() == pack_stats[pack_id].rows, block.rows(), pack_stats[pack_id].rows); - const auto handles = ColumnView(*(block.begin()->column)); + const auto handles = ColumnView(*(block.begin()->column)); const auto itr = need_read_pack_to_start_row_ids.find(pack_id); RUNTIME_CHECK(itr != need_read_pack_to_start_row_ids.end(), need_read_pack_to_start_row_ids, pack_id); read_rows += buildRowKeyFilterVector( @@ -163,7 +163,7 @@ UInt32 buildRowKeyFilterDMFile( return processed_rows; } -template +template UInt32 buildRowKeyFilterColumnFileBig( const DMContext & dm_context, const ColumnFileBig & cf_big, @@ -174,7 +174,7 @@ UInt32 buildRowKeyFilterColumnFileBig( { if (cf_big.getRows() == 0) return 0; - return buildRowKeyFilterDMFile( + return buildRowKeyFilterDMFile( dm_context, cf_big.getFile(), cf_big.getRange(), @@ -185,7 +185,7 @@ UInt32 buildRowKeyFilterColumnFileBig( filter); } -template +template UInt32 buildRowKeyFilterStable( const DMContext & dm_context, const StableValueSpace::Snapshot & stable, @@ -199,7 +199,7 @@ UInt32 buildRowKeyFilterStable( const auto & dmfile = dmfiles[0]; if (unlikely(dmfile->getPacks() == 0)) return 0; - return buildRowKeyFilterDMFile( + return buildRowKeyFilterDMFile( dm_context, dmfile, /*segment_range*/ std::nullopt, @@ -212,7 +212,7 @@ UInt32 buildRowKeyFilterStable( } // namespace -template +template void buildRowKeyFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, @@ -249,7 +249,7 @@ void buildRowKeyFilter( // TODO: add min-max value in tiny file to optimize rowkey filter. if (cf->isInMemoryFile() || cf->isTinyFile()) { - const auto n = buildRowKeyFilterBlock( + const auto n = buildRowKeyFilterBlock( dm_context, data_provider, *cf, @@ -262,7 +262,7 @@ void buildRowKeyFilter( } else if (const auto * cf_big = cf->tryToBigFile(); cf_big) { - const auto n = buildRowKeyFilterColumnFileBig( + const auto n = buildRowKeyFilterColumnFileBig( dm_context, *cf_big, delete_ranges, @@ -277,7 +277,7 @@ void buildRowKeyFilter( RUNTIME_CHECK(read_rows == delta_rows, read_rows, delta_rows); const auto n - = buildRowKeyFilterStable(dm_context, stable, delete_ranges, read_ranges, stable_pack_res, filter); + = buildRowKeyFilterStable(dm_context, stable, delete_ranges, read_ranges, stable_pack_res, filter); RUNTIME_CHECK(n == stable_rows, n, stable_rows); } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h index 3cf7f55f2aa..a66037989e7 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h @@ -24,7 +24,7 @@ struct SegmentSnapshot; struct RowKeyRange; using RowKeyRanges = std::vector; -template +template void buildRowKeyFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index 8a792baafdd..3021ea27476 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -20,8 +20,8 @@ namespace DB::DM { -template -std::shared_ptr> VersionChain::replaySnapshot( +template +std::shared_ptr> VersionChain::replaySnapshot( const DMContext & dm_context, const SegmentSnapshot & snapshot) { @@ -31,7 +31,7 @@ std::shared_ptr> VersionChain::replaySnapshot( const auto & dmfiles = snapshot.stable->getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); dmfile_or_delete_range_list.push_back( - DMFileHandleIndex{dm_context, dmfiles[0], /*start_row_id*/ 0, /*rowkey_range*/ std::nullopt}); + DMFileHandleIndex{dm_context, dmfiles[0], /*start_row_id*/ 0, /*rowkey_range*/ std::nullopt}); } const auto & stable = *(snapshot.stable); @@ -114,8 +114,8 @@ std::shared_ptr> VersionChain::replaySnapshot( return base_versions; } -template -UInt32 VersionChain::replayBlock( +template +UInt32 VersionChain::replayBlock( const DMContext & dm_context, const IColumnFileDataProviderPtr & data_provider, const ColumnFile & cf, @@ -125,7 +125,7 @@ UInt32 VersionChain::replayBlock( { assert(cf.isInMemoryFile() || cf.isTinyFile()); - auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); + auto cf_reader = cf.getReader(dm_context, data_provider, getHandleColumnDefinesPtr(), ReadTag::MVCC); auto block = cf_reader->readNextBlock(); RUNTIME_CHECK_MSG( cf.getRows() == block.rows(), @@ -136,7 +136,7 @@ UInt32 VersionChain::replayBlock( const auto & column = *(block.begin()->column); RUNTIME_CHECK(column.size() > offset, column.size(), offset); - const auto handle_col = ColumnView(*(block.begin()->column)); + const auto handle_col = ColumnView(*(block.begin()->column)); auto itr = handle_col.begin() + offset; if (calculate_read_packs) @@ -163,8 +163,8 @@ UInt32 VersionChain::replayBlock( return column.size() - offset; } -template -UInt32 VersionChain::replayColumnFileBig( +template +UInt32 VersionChain::replayColumnFileBig( const DMContext & dm_context, const ColumnFileBig & cf_big, const UInt32 stable_rows) @@ -174,12 +174,12 @@ UInt32 VersionChain::replayColumnFileBig( base_versions->insert(base_versions->end(), rows, NotExistRowID); dmfile_or_delete_range_list.push_back( - DMFileHandleIndex{dm_context, cf_big.getFile(), start_row_id, cf_big.getRange()}); + DMFileHandleIndex{dm_context, cf_big.getFile(), start_row_id, cf_big.getRange()}); return rows; } -template -UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range) +template +UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_delete_range) { auto [start, end] = convertRowKeyRange(cf_delete_range.getDeleteRange()); auto itr = new_handle_to_row_ids.lower_bound(start); @@ -192,9 +192,9 @@ UInt32 VersionChain::replayDeleteRange(const ColumnFileDeleteRange & cf_ return cf_delete_range.getDeletes(); } -template +template template -std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRangeList( +std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRangeList( const DMContext & dm_context, HandleRef h) { @@ -202,7 +202,7 @@ std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRang for (auto itr = dmfile_or_delete_range_list.rbegin(); itr != dmfile_or_delete_range_list.rend(); ++itr) { auto & dmfile_or_delete_range = *itr; - if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) + if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) { if (auto row_id = dmfile_index->getBaseVersion(dm_context, h); row_id) return row_id; @@ -216,29 +216,29 @@ std::optional VersionChain::findBaseVersionFromDMFileOrDeleteRang return {}; } -template +template template -void VersionChain::calculateReadPacks(Iterator begin, Iterator end) +void VersionChain::calculateReadPacks(Iterator begin, Iterator end) { assert(dmfile_or_delete_range_list.size() == 1); - auto & dmfile_index = std::get>(dmfile_or_delete_range_list.front()); + auto & dmfile_index = std::get>(dmfile_or_delete_range_list.front()); dmfile_index.calculateReadPacks(begin, end); } -template -void VersionChain::cleanHandleColumn() +template +void VersionChain::cleanHandleColumn() { for (auto & dmfile_or_delete_range : dmfile_or_delete_range_list) { - if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) + if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) dmfile_index->cleanHandleColumn(); } } -template -std::pair VersionChain::convertRowKeyRange(const RowKeyRange & range) +template +std::pair VersionChain::convertRowKeyRange(const RowKeyRange & range) { - if constexpr (std::is_same_v) + if constexpr (std::is_same_v) return {range.start.int_value, range.end.int_value}; else return {*(range.start.value), *(range.end.value)}; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index d0ba3e3439a..d8e7a501bca 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -29,7 +29,7 @@ class ColumnFile; class ColumnFileBig; class ColumnFileDeleteRange; -template +template class VersionChain { public: @@ -77,13 +77,13 @@ class VersionChain void calculateReadPacks(Iterator begin, Iterator end); void cleanHandleColumn(); - static std::pair convertRowKeyRange(const RowKeyRange & range); + static std::pair convertRowKeyRange(const RowKeyRange & range); std::mutex mtx; UInt32 replayed_rows_and_deletes = 0; // delta.getRows() + delta.getDeletes() std::shared_ptr> base_versions; // base_versions->size() == delta.getRows() - std::map> new_handle_to_row_ids; - using DMFileOrDeleteRange = std::variant>; + std::map> new_handle_to_row_ids; + using DMFileOrDeleteRange = std::variant>; std::vector dmfile_or_delete_range_list; }; } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp index 4feea636018..fbd6a69f5a7 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp @@ -74,7 +74,7 @@ namespace DB::DM return versions.size(); } -template +template [[nodiscard]] UInt32 buildVersionFilterDMFile( const DMContext & dm_context, const DMFilePtr & dmfile, @@ -119,7 +119,7 @@ template builder.onlyReadOnePackEveryTime().setReadPacks(need_read_packs).setReadTag(ReadTag::MVCC); auto stream = builder.build( dmfile, - {getHandleColumnDefine(), getVersionColumnDefine()}, + {getHandleColumnDefine(), getVersionColumnDefine()}, {}, dm_context.scan_context); @@ -129,7 +129,7 @@ template auto block = stream->read(); RUNTIME_CHECK(block.rows() == pack_stats[pack_id].rows, block.rows(), pack_stats[pack_id].rows); read_rows += block.rows(); - const auto handles = ColumnView(*(block.getByPosition(0).column)); + const auto handles = ColumnView(*(block.getByPosition(0).column)); const auto & versions = *toColumnVectorDataPtr(block.getByPosition(1).column); // Must success. const auto itr = need_read_pack_to_start_row_ids.find(pack_id); RUNTIME_CHECK(itr != need_read_pack_to_start_row_ids.end(), need_read_pack_to_start_row_ids, pack_id); @@ -186,7 +186,7 @@ template return processed_rows; } -template +template [[nodiscard]] UInt32 buildVersionFilterColumnFileBig( const DMContext & dm_context, const ColumnFileBig & cf_big, @@ -194,7 +194,7 @@ template const ssize_t start_row_id, IColumn::Filter & filter) { - return buildVersionFilterDMFile( + return buildVersionFilterDMFile( dm_context, cf_big.getFile(), cf_big.getRange(), @@ -203,7 +203,7 @@ template filter); } -template +template [[nodiscard]] UInt32 buildVersionFilterStable( const DMContext & dm_context, const StableValueSpace::Snapshot & stable, @@ -212,10 +212,10 @@ template { const auto & dmfiles = stable.getDMFiles(); RUNTIME_CHECK(dmfiles.size() == 1, dmfiles.size()); - return buildVersionFilterDMFile(dm_context, dmfiles[0], std::nullopt, read_ts, 0, filter); + return buildVersionFilterDMFile(dm_context, dmfiles[0], std::nullopt, read_ts, 0, filter); } -template +template void buildVersionFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, @@ -265,14 +265,15 @@ void buildVersionFilter( if (const auto * cf_big = cf->tryToBigFile(); cf_big) { - const auto n = buildVersionFilterColumnFileBig(dm_context, *cf_big, read_ts, start_row_id, filter); + const auto n + = buildVersionFilterColumnFileBig(dm_context, *cf_big, read_ts, start_row_id, filter); RUNTIME_CHECK(cf_rows == n, cf_rows, n); continue; } RUNTIME_CHECK_MSG(false, "{}: unknow ColumnFile type", cf->toString()); } RUNTIME_CHECK(read_rows == delta_rows, read_rows, delta_rows); - const auto n = buildVersionFilterStable(dm_context, stable, read_ts, filter); + const auto n = buildVersionFilterStable(dm_context, stable, read_ts, filter); RUNTIME_CHECK(n == stable_rows, n, stable_rows); } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h index f79576a0819..7d962d9367f 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.h @@ -22,7 +22,7 @@ namespace DB::DM struct DMContext; struct SegmentSnapshot; -template +template void buildVersionFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, From 8e16688c78c027bff97426340d941718708532ef Mon Sep 17 00:00:00 2001 From: jinhelin Date: Mon, 13 Jan 2025 18:04:46 +0800 Subject: [PATCH 21/48] ci --- dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 41fe8359606..e571a69b44f 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -58,5 +58,5 @@ template BitmapFilterPtr buildBitmapFilter( const RowKeyRanges & read_ranges, const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, - VersionChain & version_chain); + VersionChain & version_chain); } // namespace DB::DM From 8d5df1fe069cfb58cb98a9c8f80f870e066f896a Mon Sep 17 00:00:00 2001 From: jinhelin Date: Tue, 14 Jan 2025 11:52:50 +0800 Subject: [PATCH 22/48] ci --- .../VersionChain/BuildBitmapFilter.h | 1 + .../DeltaMerge/VersionChain/DeletedFilter.cpp | 52 ++++++++++--------- .../DeltaMerge/VersionChain/DeletedFilter.h | 1 - .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 2 +- .../DeltaMerge/VersionChain/VersionFilter.cpp | 3 +- 5 files changed, 31 insertions(+), 28 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h index 07e3e799a44..231ccb6a964 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h @@ -35,4 +35,5 @@ BitmapFilterPtr buildBitmapFilter( const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, VersionChain & version_chain); + } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp index 4f0520945ac..f786d3c3297 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp @@ -31,14 +31,21 @@ UInt32 buildDeletedFilterBlock( IColumn::Filter & filter) { assert(cf.isInMemoryFile() || cf.isTinyFile()); + const auto rows = cf.getRows(); + if (rows == 0) + return 0; + auto cf_reader = cf.getReader(dm_context, data_provider, getTagColumnDefinesPtr(), ReadTag::MVCC); auto block = cf_reader->readNextBlock(); - RUNTIME_CHECK_MSG(!cf_reader->readNextBlock(), "{}: read all rows in one block is required!", cf.toString()); + RUNTIME_CHECK_MSG( + rows == block.rows(), + "ColumnFile<{}> returns {} rows. Read all rows in one block is required!", + cf.toString(), + block.rows()); const auto & deleteds = *toColumnVectorDataPtr(block.begin()->column); // Must success. for (UInt32 i = 0; i < deleteds.size(); ++i) { - if (deleteds[i]) - filter[start_row_id + i] = 0; + filter[start_row_id + i] = !deleteds[i]; } return deleteds.size(); } @@ -54,52 +61,50 @@ UInt32 buildDeletedFilterDMFile( if (valid_handle_res.empty()) return 0; - auto read_packs = std::make_shared(); + auto need_read_packs = std::make_shared(); UInt32 need_read_rows = 0; - std::unordered_map read_pack_to_start_row_ids; + std::unordered_map need_read_pack_to_start_row_ids; const auto & pack_stats = dmfile->getPackStats(); const auto & pack_properties = dmfile->getPackProperties(); - UInt32 rows = 0; + UInt32 processed_rows = 0; for (UInt32 i = 0; i < valid_handle_res.size(); ++i) { const UInt32 pack_id = valid_start_pack_id + i; - const UInt32 pack_start_row_id = start_row_id + rows; + const UInt32 pack_start_row_id = start_row_id + processed_rows; if (pack_properties.property(pack_id).deleted_rows() > 0) { - read_packs->insert(pack_id); - read_pack_to_start_row_ids.emplace(pack_id, pack_start_row_id); + need_read_packs->insert(pack_id); + need_read_pack_to_start_row_ids.emplace(pack_id, pack_start_row_id); need_read_rows += pack_stats[pack_id].rows; } - rows += pack_stats[pack_id].rows; + processed_rows += pack_stats[pack_id].rows; } if (need_read_rows == 0) - return rows; + return processed_rows; DMFileBlockInputStreamBuilder builder(dm_context.global_context); - builder.onlyReadOnePackEveryTime().setReadPacks(read_packs).setReadTag(ReadTag::MVCC); + builder.onlyReadOnePackEveryTime().setReadPacks(need_read_packs).setReadTag(ReadTag::MVCC); auto stream = builder.build(dmfile, {getTagColumnDefine()}, {}, dm_context.scan_context); UInt32 read_rows = 0; - for (auto pack_id : *read_packs) + for (auto pack_id : *need_read_packs) { auto block = stream->read(); RUNTIME_CHECK(block.rows() == pack_stats[pack_id].rows, block.rows(), pack_stats[pack_id].rows); const auto & deleteds = *toColumnVectorDataPtr(block.begin()->column); // Must success - const auto itr = read_pack_to_start_row_ids.find(pack_id); - RUNTIME_CHECK(itr != read_pack_to_start_row_ids.end(), read_pack_to_start_row_ids, pack_id); + const auto itr = need_read_pack_to_start_row_ids.find(pack_id); + RUNTIME_CHECK(itr != need_read_pack_to_start_row_ids.end(), need_read_pack_to_start_row_ids, pack_id); const UInt32 pack_start_row_id = itr->second; - for (UInt32 i = 0; i < deleteds.size(); ++i) { - if (deleteds[i]) - filter[pack_start_row_id + i] = 0; + filter[pack_start_row_id + i] = !deleteds[i]; } read_rows += pack_stats[pack_id].rows; } RUNTIME_CHECK(read_rows == need_read_rows, read_rows, need_read_rows); - return rows; + return processed_rows; } UInt32 buildDeletedFilterColumnFileBig( @@ -128,16 +133,15 @@ void buildDeletedFilter(const DMContext & dm_context, const SegmentSnapshot & sn const UInt32 delta_rows = delta.getRows(); const UInt32 stable_rows = stable.getDMFilesRows(); const UInt32 total_rows = delta_rows + stable_rows; - RUNTIME_CHECK(filter.size() == total_rows, filter.size(), total_rows); + const auto cfs = delta.getColumnFiles(); + const auto & data_provider = delta.getDataProvider(); + assert(filter.size() == total_rows); auto read_rows = buildDeletedFilterStable(dm_context, stable, filter); RUNTIME_CHECK(stable_rows == read_rows, stable_rows, read_rows); - const auto cfs = delta.getColumnFiles(); - const auto & data_provider = delta.getDataProvider(); - for (auto itr = cfs.begin(); itr != cfs.end(); ++itr) + for (const auto & cf : cfs) { - const auto & cf = *itr; if (cf->isDeleteRange()) continue; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h index b02ad50e557..8418b63222a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.h @@ -15,7 +15,6 @@ #pragma once #include -#include namespace DB::DM { diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index 0aa449f870a..f081e080c94 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -226,9 +226,9 @@ void buildRowKeyFilter( const UInt32 stable_rows = stable.getDMFilesRows(); const UInt32 total_rows = delta_rows + stable_rows; RUNTIME_CHECK(filter.size() == total_rows, filter.size(), total_rows); - const auto cfs = delta.getColumnFiles(); const auto & data_provider = delta.getDataProvider(); + RowKeyRanges delete_ranges; UInt32 read_rows = 0; // Read ColumnFiles from new to old for handling delete ranges diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp index fbd6a69f5a7..16cc457ed07 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp @@ -229,11 +229,10 @@ void buildVersionFilter( const UInt32 stable_rows = stable.getDMFilesRows(); const UInt32 total_rows = delta_rows + stable_rows; RUNTIME_CHECK(filter.size() == total_rows, filter.size(), total_rows); - - // Delta MVCC const auto cfs = delta.getColumnFiles(); const auto & data_provider = delta.getDataProvider(); + // Delta MVCC UInt32 read_rows = 0; // Read versions from new to old for (auto itr = cfs.rbegin(); itr != cfs.rend(); ++itr) From 29f7bb44b516027dc0a821cf8db8a268708e4e45 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Tue, 14 Jan 2025 13:19:14 +0800 Subject: [PATCH 23/48] ci --- .../DeltaMerge/VersionChain/BuildBitmapFilter.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index e571a69b44f..a9faae1840b 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -59,4 +59,12 @@ template BitmapFilterPtr buildBitmapFilter( const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, VersionChain & version_chain); + +template BitmapFilterPtr buildBitmapFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const DMFilePackFilterResults & pack_filter_results, + const UInt64 read_ts, + VersionChain & version_chain); } // namespace DB::DM From 89f364537bf54d07e663414c9f57dd6de231948c Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 15 Jan 2025 17:09:49 +0800 Subject: [PATCH 24/48] ci --- .../tests/bench_version_chain.cpp | 154 +++++++++--------- dbms/src/TestUtils/bench_dbms_main.cpp | 6 +- 2 files changed, 80 insertions(+), 80 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 415ae79bd05..9e3937e024e 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -24,6 +24,7 @@ #include #include + using namespace DB; using namespace DB::tests; using namespace DB::DM; @@ -36,15 +37,6 @@ extern const Metric DT_SnapshotOfRead; namespace { -ContextPtr context; -DMContextPtr dm_context; -ColumnDefinesPtr cols; - -SegmentPtr segment; -SegmentSnapshotPtr segment_snapshot; - -constexpr const char * log_level = "error"; - const String db_name = "test"; UInt64 version = 1; @@ -56,47 +48,44 @@ enum class BenchType VersionChain = 2, }; - -auto loadPackFilterResults(const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) +[[maybe_unused]] auto loadPackFilterResults( + const DMContext & dm_context, + const SegmentSnapshotPtr & snap, + const RowKeyRanges & ranges) { DMFilePackFilterResults results; results.reserve(snap->stable->getDMFiles().size()); for (const auto & file : snap->stable->getDMFiles()) { - auto pack_filter = DMFilePackFilter::loadFrom(*dm_context, file, true, ranges, EMPTY_RS_OPERATOR, {}); + auto pack_filter = DMFilePackFilter::loadFrom(dm_context, file, true, ranges, EMPTY_RS_OPERATOR, {}); results.push_back(pack_filter); } return results; } -void initContext(bool is_common_handle, BenchType type) +auto getContext() { - if (context) - return; - - bool enable_colors = isatty(STDERR_FILENO) && isatty(STDOUT_FILENO); - DB::tests::TiFlashTestEnv::setupLogger(log_level, std::cerr, enable_colors); - - auto table_name = String(magic_enum::enum_name(type)); - UInt64 curr_ns = clock_gettime_ns(); - String testdata_path = fmt::format("/tmp/{}.{}", curr_ns, table_name); + const auto table_name = std::to_string(clock_gettime_ns()); + const auto testdata_path = fmt::format("/tmp/{}", table_name); constexpr auto run_mode = DB::PageStorageRunMode::ONLY_V3; TiFlashTestEnv::initializeGlobalContext({testdata_path}, run_mode); - context = TiFlashTestEnv::getContext(); + return std::pair{TiFlashTestEnv::getContext(), std::move(table_name)}; +} +auto getDMContext(Context & context, const String & table_name, bool is_common_handle) +{ auto storage_path_pool - = std::make_shared(context->getPathPool().withTable(db_name, table_name, false)); + = std::make_shared(context.getPathPool().withTable(db_name, table_name, false)); auto storage_pool = std::make_shared( - *context, + context, NullspaceID, /*NAMESPACE_ID*/ 100, *storage_path_pool, fmt::format("{}.{}", db_name, table_name)); storage_pool->restore(); - cols = DMTestEnv::getDefaultColumns( - is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID); - dm_context = DMContext::createUnique( - *context, + + auto dm_context = DMContext::createUnique( + context, storage_path_pool, storage_pool, /*min_version_*/ 0, @@ -105,14 +94,19 @@ void initContext(bool is_common_handle, BenchType type) /*pk_col_id*/ MutSup::extra_handle_id, is_common_handle, 1, // rowkey_column_size - context->getSettingsRef()); + context.getSettingsRef()); + + auto cols = DMTestEnv::getDefaultColumns( + is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID); + + return std::pair{std::move(dm_context), std::move(cols)}; } -SegmentPtr createSegment(bool is_common_handle) +SegmentPtr createSegment(DMContext & dm_context, const ColumnDefinesPtr & cols, bool is_common_handle) { return Segment::newSegment( Logger::get(), - *dm_context, + dm_context, cols, RowKeyRange::newAll(is_common_handle, 1), DELTA_MERGE_FIRST_SEGMENT_ID, @@ -162,9 +156,7 @@ class RandomSequence std::vector::iterator pos; }; -RandomSequence random_sequences{10 * MaxHandle}; - -void writeDelta(Segment & seg, UInt32 delta_rows) +void writeDelta(DMContext & dm_context, Segment & seg, UInt32 delta_rows, RandomSequence & random_sequences) { for (UInt32 i = 0; i < delta_rows; i += 2048) { @@ -180,38 +172,42 @@ void writeDelta(Segment & seg, UInt32 delta_rows) std::vector(n, /*deleted*/ 0), MutSup::delmark_column_name, MutSup::delmark_col_id)); - seg.write(*dm_context, block, false); + seg.write(dm_context, block, false); } } -SegmentPtr createSegment(bool is_common_handle, UInt32 delta_rows) +SegmentPtr createSegmentWithData( + DMContext & dm_context, + const ColumnDefinesPtr & cols, + bool is_common_handle, + UInt32 delta_rows, + RandomSequence & random_sequences) { - auto seg = createSegment(is_common_handle); - { - auto block = DMTestEnv::prepareSimpleWriteBlock(0, 1000000, false, version++); - seg->write(*dm_context, block, false); - seg = seg->mergeDelta(*dm_context, cols); - } - - writeDelta(*seg, delta_rows); - + auto seg = createSegment(dm_context, cols, is_common_handle); + auto block = DMTestEnv::prepareSimpleWriteBlock(0, 1000000, false, version++); + seg->write(dm_context, block, false); + seg = seg->mergeDelta(dm_context, cols); + writeDelta(dm_context, *seg, delta_rows, random_sequences); return seg; } -DeltaIndexPtr buildDeltaIndex(const SegmentSnapshotPtr & snapshot, Segment & segment) +DeltaIndexPtr buildDeltaIndex( + const DMContext & dm_context, + const ColumnDefines & cols, + const SegmentSnapshotPtr & snapshot, + Segment & segment) { - auto pk_ver_col_defs = std::make_shared( - ColumnDefines{getExtraHandleColumnDefine(dm_context->is_common_handle), getVersionColumnDefine()}); + auto pk_ver_col_defs = std::make_shared(ColumnDefines{cols[0], cols[1]}); auto delta_reader = std::make_shared( - *dm_context, + dm_context, snapshot->delta, pk_ver_col_defs, segment.getRowKeyRange(), ReadTag::MVCC); auto [delta_index, fully_indexed] = segment.ensurePlace( - *dm_context, + dm_context, snapshot, delta_reader, {segment.getRowKeyRange()}, @@ -223,28 +219,26 @@ DeltaIndexPtr buildDeltaIndex(const SegmentSnapshotPtr & snapshot, Segment & seg return delta_index; } -void buildVersionChain(const SegmentSnapshot & snapshot, VersionChain & version_chain) +template +void buildVersionChain(const DMContext & dm_context, const SegmentSnapshot & snapshot, VersionChain & version_chain) { - const auto base_ver_snap = version_chain.replaySnapshot(*dm_context, snapshot); + const auto base_ver_snap = version_chain.replaySnapshot(dm_context, snapshot); benchmark::DoNotOptimize(base_ver_snap); } -void initialize(BenchType type, bool is_common_handle, UInt32 delta_rows) +auto initialize(bool is_common_handle, UInt32 delta_rows) { - random_sequences.reset(); - initContext(is_common_handle, type); - segment = createSegment(is_common_handle, delta_rows); - segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); -} - -void shutdown() -{ - segment_snapshot = nullptr; - segment = nullptr; - cols = nullptr; - dm_context = nullptr; - context->shutdown(); - context = nullptr; + auto [context, table_name] = getContext(); + auto [dm_context, cols] = getDMContext(*context, table_name, is_common_handle); + RandomSequence random_sequences{10 * MaxHandle}; + auto segment = createSegmentWithData(*dm_context, cols, is_common_handle, delta_rows, random_sequences); + auto segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); + return std::tuple{ + std::move(context), + std::move(dm_context), + std::move(cols), + std::move(segment), + std::move(segment_snapshot)}; } template @@ -252,17 +246,20 @@ void MVCCFullBuildIndex(benchmark::State & state, Args &&... args) try { const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); - initialize(type, is_common_handle, delta_rows); + auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); + SCOPE_EXIT({ context->shutdown(); }); if (type == BenchType::DeltaIndex) { - RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); - auto delta_index = buildDeltaIndex(segment_snapshot, *segment); // Warming up - RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); + { + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); + auto delta_index = buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); // Warming up + RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); + } for (auto _ : state) { RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); - delta_index = buildDeltaIndex(segment_snapshot, *segment); + auto delta_index = buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); } } @@ -270,20 +267,21 @@ try { { VersionChain version_chain; - buildVersionChain(*segment_snapshot, version_chain); // Warming up + buildVersionChain(*dm_context, *segment_snapshot, version_chain); // Warming up RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); } for (auto _ : state) { VersionChain version_chain; - buildVersionChain(*segment_snapshot, version_chain); + buildVersionChain(*dm_context, *segment_snapshot, version_chain); RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); } } - shutdown(); } CATCH +/* + template void MVCCIncrementalBuildIndex(benchmark::State & state, Args &&... args) try @@ -437,7 +435,7 @@ try shutdown(); } CATCH - +*/ //constexpr bool IsCommonHandle = true; constexpr bool IsNotCommonHandle = false; @@ -464,6 +462,7 @@ BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_10000, BenchType::VersionCha BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_50000, BenchType::VersionChain, IsNotCommonHandle, 50000u); BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_100000, BenchType::VersionChain, IsNotCommonHandle, 100000u); +/* BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_10, BenchType::DeltaIndex, IsNotCommonHandle, 10u); @@ -510,4 +509,5 @@ BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5000, BenchType::None, IsNotComm BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10000, BenchType::None, IsNotCommonHandle, 10000u); BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_50000, BenchType::None, IsNotCommonHandle, 50000u); BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100000, BenchType::None, IsNotCommonHandle, 100000u); +*/ } // namespace diff --git a/dbms/src/TestUtils/bench_dbms_main.cpp b/dbms/src/TestUtils/bench_dbms_main.cpp index df19a4691cc..894abdec9cf 100644 --- a/dbms/src/TestUtils/bench_dbms_main.cpp +++ b/dbms/src/TestUtils/bench_dbms_main.cpp @@ -20,9 +20,9 @@ int main(int argc, char * argv[]) { benchmark::Initialize(&argc, argv); - DB::tests::TiFlashTestEnv::setupLogger(); - // Each time TiFlashTestEnv::getContext() is called, some log will print, it's annoying. - Poco::Logger::root().setLevel("error"); + + const bool enable_colors = isatty(STDERR_FILENO) && isatty(STDOUT_FILENO); + DB::tests::TiFlashTestEnv::setupLogger("error", std::cerr, enable_colors); DB::tests::TiFlashTestEnv::initializeGlobalContext(); if (::benchmark::ReportUnrecognizedArguments(argc, argv)) return 1; From 267dc0067f4b5b93d44abbc6bb8ec26b12c310d5 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 15 Jan 2025 19:09:38 +0800 Subject: [PATCH 25/48] ci --- .../VersionChain/BuildBitmapFilter.cpp | 2 + .../DeltaMerge/VersionChain/DeletedFilter.cpp | 4 +- .../tests/bench_version_chain.cpp | 380 ++++++++++++++++-- dbms/src/TestUtils/bench_dbms_main.cpp | 4 +- 4 files changed, 346 insertions(+), 44 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index a9faae1840b..d8d03a8994a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -33,6 +33,7 @@ BitmapFilterPtr buildBitmapFilter( VersionChain & version_chain) { const auto base_ver_snap = version_chain.replaySnapshot(dm_context, snapshot); + //fmt::println("base_ver_snap={}", *base_ver_snap); const auto & delta = *(snapshot.delta); const auto & stable = *(snapshot.stable); const UInt32 delta_rows = delta.getRows(); @@ -44,6 +45,7 @@ BitmapFilterPtr buildBitmapFilter( RUNTIME_CHECK(pack_filter_results.size() == 1, pack_filter_results.size()); const auto stable_pack_res = pack_filter_results.front()->getPackRes(); + //fmt::println("stable_rows={}, delta_rows={}, stable_pack_res={}", stable_rows, delta_rows, stable_pack_res); buildRowKeyFilter(dm_context, snapshot, read_ranges, stable_pack_res, filter); buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); buildDeletedFilter(dm_context, snapshot, filter); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp index f786d3c3297..6696d4b4a26 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/DeletedFilter.cpp @@ -45,7 +45,7 @@ UInt32 buildDeletedFilterBlock( const auto & deleteds = *toColumnVectorDataPtr(block.begin()->column); // Must success. for (UInt32 i = 0; i < deleteds.size(); ++i) { - filter[start_row_id + i] = !deleteds[i]; + filter[start_row_id + i] = filter[start_row_id + i] && !deleteds[i]; } return deleteds.size(); } @@ -99,7 +99,7 @@ UInt32 buildDeletedFilterDMFile( const UInt32 pack_start_row_id = itr->second; for (UInt32 i = 0; i < deleteds.size(); ++i) { - filter[pack_start_row_id + i] = !deleteds[i]; + filter[pack_start_row_id + i] = filter[pack_start_row_id + i] && !deleteds[i]; } read_rows += pack_stats[pack_id].rows; } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 9e3937e024e..89b869c0791 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -48,7 +48,18 @@ enum class BenchType VersionChain = 2, }; -[[maybe_unused]] auto loadPackFilterResults( +enum class WriteLoad +{ + RandomUpdate = 1, + AppendOnly = 2, + RandomInsert = 3, +}; + +// constexpr bool IsCommonHandle = true; +constexpr bool IsNotCommonHandle = false; + + +auto loadPackFilterResults( const DMContext & dm_context, const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) @@ -156,14 +167,27 @@ class RandomSequence std::vector::iterator pos; }; -void writeDelta(DMContext & dm_context, Segment & seg, UInt32 delta_rows, RandomSequence & random_sequences) +Strings toMockCommonHandles(const std::vector & v) +{ + Strings handles; + for (Int64 i : v) + handles.push_back(genMockCommonHandle(i, 1)); + return handles; +} + +std::vector delta; +void writeDelta(DMContext & dm_context, bool is_common_handle, Segment & seg, UInt32 delta_rows, RandomSequence & random_sequences) { for (UInt32 i = 0; i < delta_rows; i += 2048) { Block block; const auto n = std::min(delta_rows - i, 2048U); const auto v = random_sequences.get(n); - block.insert(createColumn(v, MutSup::extra_handle_column_name, MutSup::extra_handle_id)); + delta = v; + if (is_common_handle) + block.insert(createColumn(toMockCommonHandles(v), MutSup::extra_handle_column_name, MutSup::extra_handle_id)); + else + block.insert(createColumn(v, MutSup::extra_handle_column_name, MutSup::extra_handle_id)); block.insert(createColumn( std::vector(n, version++), MutSup::version_column_name, @@ -184,10 +208,18 @@ SegmentPtr createSegmentWithData( RandomSequence & random_sequences) { auto seg = createSegment(dm_context, cols, is_common_handle); - auto block = DMTestEnv::prepareSimpleWriteBlock(0, 1000000, false, version++); + auto block = DMTestEnv::prepareSimpleWriteBlock( + 0, + 1000000, + false, + version++, + MutSup::extra_handle_column_name, + MutSup::extra_handle_id, + is_common_handle ? MutSup::getExtraHandleColumnStringType() : MutSup::getExtraHandleColumnIntType(), + is_common_handle); seg->write(dm_context, block, false); seg = seg->mergeDelta(dm_context, cols); - writeDelta(dm_context, *seg, delta_rows, random_sequences); + writeDelta(dm_context, is_common_handle, *seg, delta_rows, random_sequences); return seg; } @@ -245,7 +277,7 @@ template void MVCCFullBuildIndex(benchmark::State & state, Args &&... args) try { - const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); + const auto [type, write_load, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); SCOPE_EXIT({ context->shutdown(); }); @@ -382,26 +414,28 @@ try shutdown(); } CATCH - +*/ // TODO: move verify to unit-tests. template void MVCCBuildBitmapVerify(benchmark::State & state, Args &&... args) try { const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); - initialize(type, is_common_handle, delta_rows); - + auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); + SCOPE_EXIT({ context->shutdown(); }); + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); - auto delta_index = buildDeltaIndex(segment_snapshot, *segment); + auto delta_index = + buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*delta_index); RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == delta_rows); VersionChain version_chain; - buildVersionChain(*segment_snapshot, version_chain); + buildVersionChain(*dm_context, *segment_snapshot, version_chain); RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); - auto rs_results = loadPackFilterResults(segment_snapshot, {segment->getRowKeyRange()}); + auto rs_results = loadPackFilterResults(*dm_context, segment_snapshot, {segment->getRowKeyRange()}); auto bitmap_filter1 = segment->buildBitmapFilter( *dm_context, segment_snapshot, @@ -431,37 +465,305 @@ try } for (auto _ : state) {} - - shutdown(); } CATCH -*/ - -//constexpr bool IsCommonHandle = true; -constexpr bool IsNotCommonHandle = false; -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_10, BenchType::DeltaIndex, IsNotCommonHandle, 10u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_100, BenchType::DeltaIndex, IsNotCommonHandle, 100u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_500, BenchType::DeltaIndex, IsNotCommonHandle, 500u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_1000, BenchType::DeltaIndex, IsNotCommonHandle, 1000u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_5000, BenchType::DeltaIndex, IsNotCommonHandle, 5000u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_10000, BenchType::DeltaIndex, IsNotCommonHandle, 10000u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_50000, BenchType::DeltaIndex, IsNotCommonHandle, 50000u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, delta_index_100000, BenchType::DeltaIndex, IsNotCommonHandle, 100000u); - -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_1, BenchType::VersionChain, IsNotCommonHandle, 1u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_5, BenchType::VersionChain, IsNotCommonHandle, 5u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_10, BenchType::VersionChain, IsNotCommonHandle, 10u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_100, BenchType::VersionChain, IsNotCommonHandle, 100u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_500, BenchType::VersionChain, IsNotCommonHandle, 500u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_1000, BenchType::VersionChain, IsNotCommonHandle, 1000u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_5000, BenchType::VersionChain, IsNotCommonHandle, 5000u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_10000, BenchType::VersionChain, IsNotCommonHandle, 10000u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_50000, BenchType::VersionChain, IsNotCommonHandle, 50000u); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, version_chain_100000, BenchType::VersionChain, IsNotCommonHandle, 100000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_1, BenchType::None, IsNotCommonHandle, 1u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5, BenchType::None, IsNotCommonHandle, 5u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10, BenchType::None, IsNotCommonHandle, 10u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100, BenchType::None, IsNotCommonHandle, 100u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_500, BenchType::None, IsNotCommonHandle, 500u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_1k, BenchType::None, IsNotCommonHandle, 1000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5k, BenchType::None, IsNotCommonHandle, 5000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10k, BenchType::None, IsNotCommonHandle, 10000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_50k, BenchType::None, IsNotCommonHandle, 50000u); +BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100k, BenchType::None, IsNotCommonHandle, 100000u); +/* +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_1, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 1u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_5, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 5u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_10, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 10u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_100, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 100u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_500, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 500u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_1k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 1000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_5k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 5000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_10k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 10000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_50k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 50000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_100k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 100000u); + +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_1, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 1u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_5, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 5u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_10, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 10u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_100, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 100u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_500, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 500u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_1k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 1000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_5k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 5000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_10k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 10000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_50k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 50000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_100k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsNotCommonHandle, + 100000u); +*/ +/* +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_1, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 1u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_5, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 5u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_10, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 10u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_100, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 100u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_500, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 500u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_1k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 1000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_5k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 5000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_10k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 10000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_50k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 50000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + delta_idx_update_common_handle_100k, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle, + 100000u); + +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_1, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 1u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_5, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 5u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_10, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 10u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_100, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 100u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_500, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 500u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_1k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 1000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_5k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 5000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_10k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 10000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_50k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 50000u); +BENCHMARK_CAPTURE( + MVCCFullBuildIndex, + ver_chain_update_common_handle_100k, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle, + 100000u);*/ /* BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); diff --git a/dbms/src/TestUtils/bench_dbms_main.cpp b/dbms/src/TestUtils/bench_dbms_main.cpp index 894abdec9cf..8b3cf0ed957 100644 --- a/dbms/src/TestUtils/bench_dbms_main.cpp +++ b/dbms/src/TestUtils/bench_dbms_main.cpp @@ -20,13 +20,11 @@ int main(int argc, char * argv[]) { benchmark::Initialize(&argc, argv); - - const bool enable_colors = isatty(STDERR_FILENO) && isatty(STDOUT_FILENO); - DB::tests::TiFlashTestEnv::setupLogger("error", std::cerr, enable_colors); DB::tests::TiFlashTestEnv::initializeGlobalContext(); if (::benchmark::ReportUnrecognizedArguments(argc, argv)) return 1; DB::LocalAdmissionController::global_instance = std::make_unique(); + DB::tests::TiFlashTestEnv::setupLogger("error"); ::benchmark::RunSpecifiedBenchmarks(); DB::tests::TiFlashTestEnv::shutdown(); ::benchmark::Shutdown(); From 3ad515900ea302dec23665d026d044fa36a0879e Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 16 Jan 2025 14:43:42 +0800 Subject: [PATCH 26/48] ci --- dbms/src/Storages/DeltaMerge/Segment.cpp | 2 - .../tests/bench_version_chain.cpp | 246 +--------------- .../VersionChain/tests/gtest_mvcc_bitmap.cpp | 121 ++++++++ .../VersionChain/tests/mvcc_test_utils.h | 273 ++++++++++++++++++ 4 files changed, 398 insertions(+), 244 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp create mode 100644 dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index e6d8746be02..ed913d5de9d 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -3083,8 +3083,6 @@ BitmapFilterPtr Segment::buildBitmapFilter( { RUNTIME_CHECK_MSG(!dm_context.read_delta_only, "Read delta only is unsupported"); sanitizeCheckReadRanges(__FUNCTION__, read_ranges, rowkey_range, log); - RUNTIME_CHECK(!is_common_handle); - if (use_version_chain) { return buildBitmapFilter( diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 89b869c0791..e1f127c0651 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -17,18 +17,17 @@ #include #include #include +#include #include #include #include #include -#include -#include - using namespace DB; using namespace DB::tests; using namespace DB::DM; using namespace DB::DM::tests; +using namespace DB::DM::tests::MVCC; namespace CurrentMetrics { @@ -37,242 +36,6 @@ extern const Metric DT_SnapshotOfRead; namespace { -const String db_name = "test"; - -UInt64 version = 1; - -enum class BenchType -{ - None = 0, - DeltaIndex = 1, - VersionChain = 2, -}; - -enum class WriteLoad -{ - RandomUpdate = 1, - AppendOnly = 2, - RandomInsert = 3, -}; - -// constexpr bool IsCommonHandle = true; -constexpr bool IsNotCommonHandle = false; - - -auto loadPackFilterResults( - const DMContext & dm_context, - const SegmentSnapshotPtr & snap, - const RowKeyRanges & ranges) -{ - DMFilePackFilterResults results; - results.reserve(snap->stable->getDMFiles().size()); - for (const auto & file : snap->stable->getDMFiles()) - { - auto pack_filter = DMFilePackFilter::loadFrom(dm_context, file, true, ranges, EMPTY_RS_OPERATOR, {}); - results.push_back(pack_filter); - } - return results; -} - -auto getContext() -{ - const auto table_name = std::to_string(clock_gettime_ns()); - const auto testdata_path = fmt::format("/tmp/{}", table_name); - constexpr auto run_mode = DB::PageStorageRunMode::ONLY_V3; - TiFlashTestEnv::initializeGlobalContext({testdata_path}, run_mode); - return std::pair{TiFlashTestEnv::getContext(), std::move(table_name)}; -} - -auto getDMContext(Context & context, const String & table_name, bool is_common_handle) -{ - auto storage_path_pool - = std::make_shared(context.getPathPool().withTable(db_name, table_name, false)); - auto storage_pool = std::make_shared( - context, - NullspaceID, - /*NAMESPACE_ID*/ 100, - *storage_path_pool, - fmt::format("{}.{}", db_name, table_name)); - storage_pool->restore(); - - auto dm_context = DMContext::createUnique( - context, - storage_path_pool, - storage_pool, - /*min_version_*/ 0, - NullspaceID, - /*physical_table_id*/ 100, - /*pk_col_id*/ MutSup::extra_handle_id, - is_common_handle, - 1, // rowkey_column_size - context.getSettingsRef()); - - auto cols = DMTestEnv::getDefaultColumns( - is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID); - - return std::pair{std::move(dm_context), std::move(cols)}; -} - -SegmentPtr createSegment(DMContext & dm_context, const ColumnDefinesPtr & cols, bool is_common_handle) -{ - return Segment::newSegment( - Logger::get(), - dm_context, - cols, - RowKeyRange::newAll(is_common_handle, 1), - DELTA_MERGE_FIRST_SEGMENT_ID, - 0); -} - -constexpr Int64 MaxHandle = 1000000; - -class RandomSequence -{ -public: - RandomSequence(UInt32 n) - : v(randomInt64s(n)) - , pos(v.begin()) - {} - - std::vector get(UInt32 n) - { - std::vector res; - while (res.size() < n) - { - auto copied = std::min(std::distance(pos, v.end()), static_cast(n - res.size())); - res.insert(res.end(), pos, pos + copied); - std::advance(pos, copied); - if (pos == v.end()) - reset(); - } - return res; - } - - void reset() { pos = v.begin(); } - -private: - std::vector randomInt64s(UInt32 n) - { - static constexpr int rnd_seed = 573172; - std::mt19937 g(rnd_seed); - std::vector v(n); - for (UInt32 i = 0; i < n; ++i) - { - v[i] = g() % MaxHandle; - } - return v; - } - - std::vector v; - std::vector::iterator pos; -}; - -Strings toMockCommonHandles(const std::vector & v) -{ - Strings handles; - for (Int64 i : v) - handles.push_back(genMockCommonHandle(i, 1)); - return handles; -} - -std::vector delta; -void writeDelta(DMContext & dm_context, bool is_common_handle, Segment & seg, UInt32 delta_rows, RandomSequence & random_sequences) -{ - for (UInt32 i = 0; i < delta_rows; i += 2048) - { - Block block; - const auto n = std::min(delta_rows - i, 2048U); - const auto v = random_sequences.get(n); - delta = v; - if (is_common_handle) - block.insert(createColumn(toMockCommonHandles(v), MutSup::extra_handle_column_name, MutSup::extra_handle_id)); - else - block.insert(createColumn(v, MutSup::extra_handle_column_name, MutSup::extra_handle_id)); - block.insert(createColumn( - std::vector(n, version++), - MutSup::version_column_name, - MutSup::version_col_id)); - block.insert(createColumn( - std::vector(n, /*deleted*/ 0), - MutSup::delmark_column_name, - MutSup::delmark_col_id)); - seg.write(dm_context, block, false); - } -} - -SegmentPtr createSegmentWithData( - DMContext & dm_context, - const ColumnDefinesPtr & cols, - bool is_common_handle, - UInt32 delta_rows, - RandomSequence & random_sequences) -{ - auto seg = createSegment(dm_context, cols, is_common_handle); - auto block = DMTestEnv::prepareSimpleWriteBlock( - 0, - 1000000, - false, - version++, - MutSup::extra_handle_column_name, - MutSup::extra_handle_id, - is_common_handle ? MutSup::getExtraHandleColumnStringType() : MutSup::getExtraHandleColumnIntType(), - is_common_handle); - seg->write(dm_context, block, false); - seg = seg->mergeDelta(dm_context, cols); - writeDelta(dm_context, is_common_handle, *seg, delta_rows, random_sequences); - return seg; -} - -DeltaIndexPtr buildDeltaIndex( - const DMContext & dm_context, - const ColumnDefines & cols, - const SegmentSnapshotPtr & snapshot, - Segment & segment) -{ - auto pk_ver_col_defs = std::make_shared(ColumnDefines{cols[0], cols[1]}); - - auto delta_reader = std::make_shared( - dm_context, - snapshot->delta, - pk_ver_col_defs, - segment.getRowKeyRange(), - ReadTag::MVCC); - - auto [delta_index, fully_indexed] = segment.ensurePlace( - dm_context, - snapshot, - delta_reader, - {segment.getRowKeyRange()}, - std::numeric_limits::max()); - - if (delta_index == nullptr || !fully_indexed) - std::abort(); - - return delta_index; -} - -template -void buildVersionChain(const DMContext & dm_context, const SegmentSnapshot & snapshot, VersionChain & version_chain) -{ - const auto base_ver_snap = version_chain.replaySnapshot(dm_context, snapshot); - benchmark::DoNotOptimize(base_ver_snap); -} - -auto initialize(bool is_common_handle, UInt32 delta_rows) -{ - auto [context, table_name] = getContext(); - auto [dm_context, cols] = getDMContext(*context, table_name, is_common_handle); - RandomSequence random_sequences{10 * MaxHandle}; - auto segment = createSegmentWithData(*dm_context, cols, is_common_handle, delta_rows, random_sequences); - auto segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); - return std::tuple{ - std::move(context), - std::move(dm_context), - std::move(cols), - std::move(segment), - std::move(segment_snapshot)}; -} - template void MVCCFullBuildIndex(benchmark::State & state, Args &&... args) try @@ -423,10 +186,9 @@ try const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); SCOPE_EXIT({ context->shutdown(); }); - + RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); - auto delta_index = - buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); + auto delta_index = buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*delta_index); RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == delta_rows); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp new file mode 100644 index 00000000000..5a15882d1e9 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp @@ -0,0 +1,121 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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 +#include +#include +#include +#include + +using namespace DB::tests; +using namespace DB::DM::tests::MVCC; + +namespace DB::DM::tests +{ +template +void randomMVCCBitmapVerify(UInt32 delta_rows) +try +{ + constexpr bool is_common_handle = std::is_same_v; + auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); + SCOPE_EXIT({ context->shutdown(); }); + + ASSERT_EQ(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first, 0); + auto delta_index = buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); + ASSERT_EQ(delta_index->getPlacedStatus().first, delta_rows); + segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*delta_index); + ASSERT_EQ(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first, delta_rows); + + + VersionChain version_chain; + buildVersionChain(*dm_context, *segment_snapshot, version_chain); + ASSERT_EQ(version_chain.getReplayedRows(), delta_rows); + + auto rs_results = loadPackFilterResults(*dm_context, segment_snapshot, {segment->getRowKeyRange()}); + auto bitmap_filter1 = segment->buildBitmapFilter( + *dm_context, + segment_snapshot, + {segment->getRowKeyRange()}, + rs_results, + std::numeric_limits::max(), + DEFAULT_BLOCK_SIZE, + false); + auto bitmap_filter2 = buildBitmapFilter( + *dm_context, + *segment_snapshot, + {segment->getRowKeyRange()}, + rs_results, + std::numeric_limits::max(), + version_chain); + + const auto & filter1 = bitmap_filter1->getFilter(); + const auto & filter2 = bitmap_filter2->getFilter(); + RUNTIME_ASSERT(filter1.size() == filter2.size()); + for (UInt32 i = 0; i < filter1.size(); ++i) + { + ASSERT_EQ(filter1[i], filter2[i]) + << fmt::format("i={}, filter1={}, filter2={}, delta_rows={}", i, filter1[i], filter2[i], delta_rows); + } +} +CATCH + +TEST(TestVersionChain, randomMVCCBitmapVerify) +{ + std::vector delta_rows{ + 1, + 10, + 50, + 100, + 500, + 1000, + 5000, + 10000, + 20000, + 30000, + 40000, + 50000, + 60000, + 70000, + 80000, + 90000, + 100000}; + for (auto rows : delta_rows) + randomMVCCBitmapVerify(rows); +} + +TEST(TestVersionChain, randomMVCCBitmapVerify_CommonHandle) +{ + std::vector delta_rows{ + 1, + 10, + 50, + 100, + 500, + 1000, + 5000, + 10000, + 20000, + 30000, + 40000, + 50000, + 60000, + 70000, + 80000, + 90000, + 100000}; + for (auto rows : delta_rows) + randomMVCCBitmapVerify(rows); +} +} // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h new file mode 100644 index 00000000000..abdf638421a --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h @@ -0,0 +1,273 @@ +// Copyright 2024 PingCAP, Inc. +// +// 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. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace CurrentMetrics +{ +extern const Metric DT_SnapshotOfRead; +} // namespace CurrentMetrics + +namespace DB::DM::tests::MVCC +{ +const String db_name = "test"; + +UInt64 version = 1; + +enum class BenchType +{ + None = 0, + DeltaIndex = 1, + VersionChain = 2, +}; + +enum class WriteLoad +{ + RandomUpdate = 1, + AppendOnly = 2, + RandomInsert = 3, +}; + +// constexpr bool IsCommonHandle = true; +constexpr bool IsNotCommonHandle = false; + + +auto loadPackFilterResults(const DMContext & dm_context, const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) +{ + DMFilePackFilterResults results; + results.reserve(snap->stable->getDMFiles().size()); + for (const auto & file : snap->stable->getDMFiles()) + { + auto pack_filter = DMFilePackFilter::loadFrom(dm_context, file, true, ranges, EMPTY_RS_OPERATOR, {}); + results.push_back(pack_filter); + } + return results; +} + +auto getContext() +{ + const auto table_name = std::to_string(clock_gettime_ns()); + const auto testdata_path = fmt::format("/tmp/{}", table_name); + constexpr auto run_mode = DB::PageStorageRunMode::ONLY_V3; + TiFlashTestEnv::initializeGlobalContext({testdata_path}, run_mode); + return std::pair{TiFlashTestEnv::getContext(), std::move(table_name)}; +} + +auto getDMContext(Context & context, const String & table_name, bool is_common_handle) +{ + auto storage_path_pool + = std::make_shared(context.getPathPool().withTable(db_name, table_name, false)); + auto storage_pool = std::make_shared( + context, + NullspaceID, + /*NAMESPACE_ID*/ 100, + *storage_path_pool, + fmt::format("{}.{}", db_name, table_name)); + storage_pool->restore(); + + auto dm_context = DMContext::createUnique( + context, + storage_path_pool, + storage_pool, + /*min_version_*/ 0, + NullspaceID, + /*physical_table_id*/ 100, + /*pk_col_id*/ MutSup::extra_handle_id, + is_common_handle, + 1, // rowkey_column_size + context.getSettingsRef()); + + auto cols = DMTestEnv::getDefaultColumns( + is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID); + + return std::pair{std::move(dm_context), std::move(cols)}; +} + +SegmentPtr createSegment(DMContext & dm_context, const ColumnDefinesPtr & cols, bool is_common_handle) +{ + return Segment::newSegment( + Logger::get(), + dm_context, + cols, + RowKeyRange::newAll(is_common_handle, 1), + DELTA_MERGE_FIRST_SEGMENT_ID, + 0); +} + +constexpr Int64 MaxHandle = 1000000; + +class RandomSequence +{ +public: + RandomSequence(UInt32 n) + : v(randomInt64s(n)) + , pos(v.begin()) + {} + + std::vector get(UInt32 n) + { + std::vector res; + while (res.size() < n) + { + auto copied = std::min(std::distance(pos, v.end()), static_cast(n - res.size())); + res.insert(res.end(), pos, pos + copied); + std::advance(pos, copied); + if (pos == v.end()) + reset(); + } + return res; + } + + void reset() { pos = v.begin(); } + +private: + std::vector randomInt64s(UInt32 n) + { + static constexpr int rnd_seed = 573172; + std::mt19937 g(rnd_seed); + std::vector v(n); + for (UInt32 i = 0; i < n; ++i) + { + v[i] = g() % MaxHandle; + } + return v; + } + + std::vector v; + std::vector::iterator pos; +}; + +Strings toMockCommonHandles(const std::vector & v) +{ + Strings handles; + for (Int64 i : v) + handles.push_back(genMockCommonHandle(i, 1)); + return handles; +} + +void writeDelta( + DMContext & dm_context, + bool is_common_handle, + Segment & seg, + UInt32 delta_rows, + RandomSequence & random_sequences) +{ + for (UInt32 i = 0; i < delta_rows; i += 2048) + { + Block block; + const auto n = std::min(delta_rows - i, 2048U); + const auto v = random_sequences.get(n); + if (is_common_handle) + block.insert(createColumn( + toMockCommonHandles(v), + MutSup::extra_handle_column_name, + MutSup::extra_handle_id)); + else + block.insert(createColumn(v, MutSup::extra_handle_column_name, MutSup::extra_handle_id)); + block.insert(createColumn( + std::vector(n, version++), + MutSup::version_column_name, + MutSup::version_col_id)); + block.insert(createColumn( + std::vector(n, /*deleted*/ 0), + MutSup::delmark_column_name, + MutSup::delmark_col_id)); + seg.write(dm_context, block, false); + } +} + +SegmentPtr createSegmentWithData( + DMContext & dm_context, + const ColumnDefinesPtr & cols, + bool is_common_handle, + UInt32 delta_rows, + RandomSequence & random_sequences) +{ + auto seg = createSegment(dm_context, cols, is_common_handle); + auto block = DMTestEnv::prepareSimpleWriteBlock( + 0, + 1000000, + false, + version++, + MutSup::extra_handle_column_name, + MutSup::extra_handle_id, + is_common_handle ? MutSup::getExtraHandleColumnStringType() : MutSup::getExtraHandleColumnIntType(), + is_common_handle); + seg->write(dm_context, block, false); + seg = seg->mergeDelta(dm_context, cols); + writeDelta(dm_context, is_common_handle, *seg, delta_rows, random_sequences); + return seg; +} + +DeltaIndexPtr buildDeltaIndex( + const DMContext & dm_context, + const ColumnDefines & cols, + const SegmentSnapshotPtr & snapshot, + Segment & segment) +{ + auto pk_ver_col_defs = std::make_shared(ColumnDefines{cols[0], cols[1]}); + + auto delta_reader = std::make_shared( + dm_context, + snapshot->delta, + pk_ver_col_defs, + segment.getRowKeyRange(), + ReadTag::MVCC); + + auto [delta_index, fully_indexed] = segment.ensurePlace( + dm_context, + snapshot, + delta_reader, + {segment.getRowKeyRange()}, + std::numeric_limits::max()); + + if (delta_index == nullptr || !fully_indexed) + std::abort(); + + return delta_index; +} + +template +auto buildVersionChain(const DMContext & dm_context, const SegmentSnapshot & snapshot, VersionChain & version_chain) +{ + return version_chain.replaySnapshot(dm_context, snapshot); +} + +auto initialize(bool is_common_handle, UInt32 delta_rows) +{ + auto [context, table_name] = getContext(); + auto [dm_context, cols] = getDMContext(*context, table_name, is_common_handle); + RandomSequence random_sequences{10 * MaxHandle}; + auto segment = createSegmentWithData(*dm_context, cols, is_common_handle, delta_rows, random_sequences); + auto segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); + return std::tuple{ + std::move(context), + std::move(dm_context), + std::move(cols), + std::move(segment), + std::move(segment_snapshot)}; +} + +} // namespace DB::DM::tests::MVCC From 96a09684ef5d7f832d73703c8719ef8a225e2323 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 16 Jan 2025 15:48:19 +0800 Subject: [PATCH 27/48] ci --- .../tests/bench_version_chain.cpp | 354 +----------------- 1 file changed, 7 insertions(+), 347 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index e1f127c0651..1a0191c275c 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -40,7 +40,8 @@ template void MVCCFullBuildIndex(benchmark::State & state, Args &&... args) try { - const auto [type, write_load, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); + const auto [type, write_load, is_common_handle] = std::make_tuple(std::move(args)...); + const UInt32 delta_rows = state.range(0); auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); SCOPE_EXIT({ context->shutdown(); }); @@ -178,354 +179,13 @@ try } CATCH */ -// TODO: move verify to unit-tests. -template -void MVCCBuildBitmapVerify(benchmark::State & state, Args &&... args) -try -{ - const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); - auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); - SCOPE_EXIT({ context->shutdown(); }); - - RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); - auto delta_index = buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); - RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); - segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*delta_index); - RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == delta_rows); - - VersionChain version_chain; - buildVersionChain(*dm_context, *segment_snapshot, version_chain); - RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); - - auto rs_results = loadPackFilterResults(*dm_context, segment_snapshot, {segment->getRowKeyRange()}); - auto bitmap_filter1 = segment->buildBitmapFilter( - *dm_context, - segment_snapshot, - {segment->getRowKeyRange()}, - rs_results, - std::numeric_limits::max(), - DEFAULT_BLOCK_SIZE, - false); - auto bitmap_filter2 = buildBitmapFilter( - *dm_context, - *segment_snapshot, - {segment->getRowKeyRange()}, - rs_results, - std::numeric_limits::max(), - version_chain); - - const auto & filter1 = bitmap_filter1->getFilter(); - const auto & filter2 = bitmap_filter2->getFilter(); - RUNTIME_ASSERT(filter1.size() == filter2.size()); - for (UInt32 i = 0; i < filter1.size(); ++i) - { - if (filter1[i] != filter2[i]) - { - fmt::println("i={}, filter1={}, filter2={}", i, filter1[i], filter2[i]); - std::abort(); - } - } - for (auto _ : state) {} -} -CATCH - -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_1, BenchType::None, IsNotCommonHandle, 1u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5, BenchType::None, IsNotCommonHandle, 5u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10, BenchType::None, IsNotCommonHandle, 10u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100, BenchType::None, IsNotCommonHandle, 100u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_500, BenchType::None, IsNotCommonHandle, 500u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_1k, BenchType::None, IsNotCommonHandle, 1000u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5k, BenchType::None, IsNotCommonHandle, 5000u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10k, BenchType::None, IsNotCommonHandle, 10000u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_50k, BenchType::None, IsNotCommonHandle, 50000u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100k, BenchType::None, IsNotCommonHandle, 100000u); - -/* -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_1, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 1u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_5, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 5u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_10, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 10u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_100, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 100u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_500, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 500u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_1k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 1000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_5k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 5000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_10k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 10000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_50k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 50000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_100k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 100000u); - -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_1, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 1u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_5, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 5u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_10, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 10u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_100, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 100u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_500, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 500u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_1k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 1000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_5k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 5000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_10k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 10000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_50k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 50000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_100k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsNotCommonHandle, - 100000u); -*/ -/* -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_1, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 1u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_5, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 5u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_10, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 10u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_100, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 100u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_500, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 500u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_1k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 1000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_5k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 5000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_10k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 10000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_50k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 50000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - delta_idx_update_common_handle_100k, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle, - 100000u); +// [ 1, 8, 64, 512, 4k, 8k, 64k ] +BENCHMARK_CAPTURE(MVCCFullBuildIndex, index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) + ->Range(1, 8 << 13); +BENCHMARK_CAPTURE(MVCCFullBuildIndex, chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) + ->Range(1, 8 << 13); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_1, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 1u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_5, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 5u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_10, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 10u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_100, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 100u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_500, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 500u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_1k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 1000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_5k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 5000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_10k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 10000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_50k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 50000u); -BENCHMARK_CAPTURE( - MVCCFullBuildIndex, - ver_chain_update_common_handle_100k, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle, - 100000u);*/ /* BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); From 0656616591a48366ed12cf736b6eba4fde7ca054 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 16 Jan 2025 16:09:06 +0800 Subject: [PATCH 28/48] ci --- .../tests/bench_version_chain.cpp | 65 +++++++------------ .../VersionChain/tests/gtest_mvcc_bitmap.cpp | 3 +- .../VersionChain/tests/mvcc_test_utils.h | 3 +- 3 files changed, 26 insertions(+), 45 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 1a0191c275c..de9e0dca4fe 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -37,12 +37,13 @@ extern const Metric DT_SnapshotOfRead; namespace { template -void MVCCFullBuildIndex(benchmark::State & state, Args &&... args) +void MVCCFullPlace(benchmark::State & state, Args &&... args) try { const auto [type, write_load, is_common_handle] = std::make_tuple(std::move(args)...); const UInt32 delta_rows = state.range(0); - auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); + auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] + = initialize(is_common_handle, delta_rows); SCOPE_EXIT({ context->shutdown(); }); if (type == BenchType::DeltaIndex) @@ -76,26 +77,27 @@ try } CATCH -/* - template -void MVCCIncrementalBuildIndex(benchmark::State & state, Args &&... args) +void MVCCIncrementalPlace(benchmark::State & state, Args &&... args) try { - const auto [type, is_common_handle, incremental_delta_rows] = std::make_tuple(std::move(args)...); + const auto [type, write_load, is_common_handle] = std::make_tuple(std::move(args)...); + const UInt32 incremental_delta_rows = state.range(0); constexpr UInt32 prepared_delta_rows = 10000; - initialize(type, is_common_handle, prepared_delta_rows); + auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] + = initialize(is_common_handle, prepared_delta_rows); + SCOPE_EXIT({ context->shutdown(); }); if (type == BenchType::DeltaIndex) { RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); - auto base_delta_index = buildDeltaIndex(segment_snapshot, *segment); + auto base_delta_index = buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); RUNTIME_ASSERT(base_delta_index->getPlacedStatus().first == prepared_delta_rows); segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*base_delta_index); RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == prepared_delta_rows); - writeDelta(*segment, incremental_delta_rows); + writeDelta(*dm_context, is_common_handle, *segment, incremental_delta_rows, random_sequences); segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == prepared_delta_rows); RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); @@ -104,31 +106,29 @@ try { RUNTIME_ASSERT( segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == prepared_delta_rows); - auto delta_index = buildDeltaIndex(segment_snapshot, *segment); + auto delta_index = buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); RUNTIME_ASSERT(delta_index->getPlacedStatus().first == prepared_delta_rows + incremental_delta_rows); } } else if (type == BenchType::VersionChain) { VersionChain base_version_chain; - buildVersionChain(*segment_snapshot, base_version_chain); + buildVersionChain(*dm_context, *segment_snapshot, base_version_chain); RUNTIME_ASSERT(base_version_chain.getReplayedRows() == prepared_delta_rows); - writeDelta(*segment, incremental_delta_rows); + writeDelta(*dm_context, is_common_handle, *segment, incremental_delta_rows, random_sequences); segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); for (auto _ : state) { auto version_chain = base_version_chain; RUNTIME_ASSERT(version_chain.getReplayedRows() == prepared_delta_rows); - buildVersionChain(*segment_snapshot, version_chain); + buildVersionChain(*dm_context, *segment_snapshot, version_chain); RUNTIME_ASSERT(version_chain.getReplayedRows() == prepared_delta_rows + incremental_delta_rows); } } - - shutdown(); } CATCH - +/* template void MVCCBuildBitmap(benchmark::State & state, Args &&... args) try @@ -181,26 +181,16 @@ CATCH */ // [ 1, 8, 64, 512, 4k, 8k, 64k ] -BENCHMARK_CAPTURE(MVCCFullBuildIndex, index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) +BENCHMARK_CAPTURE(MVCCFullPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCFullBuildIndex, chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) +BENCHMARK_CAPTURE(MVCCFullPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) ->Range(1, 8 << 13); +BENCHMARK_CAPTURE(MVCCIncrementalPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) + ->Range(1, 8 << 13); +BENCHMARK_CAPTURE(MVCCIncrementalPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) + ->Range(1, 8 << 13); /* -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_10, BenchType::DeltaIndex, IsNotCommonHandle, 10u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_100, BenchType::DeltaIndex, IsNotCommonHandle, 100u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_500, BenchType::DeltaIndex, IsNotCommonHandle, 500u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, delta_index_1000, BenchType::DeltaIndex, IsNotCommonHandle, 1000u); - -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_1, BenchType::VersionChain, IsNotCommonHandle, 1u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_5, BenchType::VersionChain, IsNotCommonHandle, 5u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_10, BenchType::VersionChain, IsNotCommonHandle, 10u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_100, BenchType::VersionChain, IsNotCommonHandle, 100u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_500, BenchType::VersionChain, IsNotCommonHandle, 500u); -BENCHMARK_CAPTURE(MVCCIncrementalBuildIndex, version_chain_1000, BenchType::VersionChain, IsNotCommonHandle, 1000u); - BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_10, BenchType::DeltaIndex, IsNotCommonHandle, 10u); @@ -222,16 +212,5 @@ BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_5000, BenchType::VersionChain, BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_10000, BenchType::VersionChain, IsNotCommonHandle, 10000u); BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_50000, BenchType::VersionChain, IsNotCommonHandle, 50000u); BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_100000, BenchType::VersionChain, IsNotCommonHandle, 100000u); - -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_1, BenchType::None, IsNotCommonHandle, 1u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5, BenchType::None, IsNotCommonHandle, 5u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10, BenchType::None, IsNotCommonHandle, 10u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100, BenchType::None, IsNotCommonHandle, 100u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_500, BenchType::None, IsNotCommonHandle, 500u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_1000, BenchType::None, IsNotCommonHandle, 1000u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_5000, BenchType::None, IsNotCommonHandle, 5000u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_10000, BenchType::None, IsNotCommonHandle, 10000u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_50000, BenchType::None, IsNotCommonHandle, 50000u); -BENCHMARK_CAPTURE(MVCCBuildBitmapVerify, verify_100000, BenchType::None, IsNotCommonHandle, 100000u); */ } // namespace diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp index 5a15882d1e9..6672cd83d32 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp @@ -29,7 +29,8 @@ void randomMVCCBitmapVerify(UInt32 delta_rows) try { constexpr bool is_common_handle = std::is_same_v; - auto [context, dm_context, cols, segment, segment_snapshot] = initialize(is_common_handle, delta_rows); + auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] + = initialize(is_common_handle, delta_rows); SCOPE_EXIT({ context->shutdown(); }); ASSERT_EQ(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first, 0); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h index abdf638421a..140584daa1b 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h @@ -267,7 +267,8 @@ auto initialize(bool is_common_handle, UInt32 delta_rows) std::move(dm_context), std::move(cols), std::move(segment), - std::move(segment_snapshot)}; + std::move(segment_snapshot), + std::move(random_sequences)}; } } // namespace DB::DM::tests::MVCC From 9b1fa3aa500ace154153db565e75cff8cef3af69 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 16 Jan 2025 16:28:51 +0800 Subject: [PATCH 29/48] ci --- .../tests/bench_version_chain.cpp | 46 ++++++------------- 1 file changed, 15 insertions(+), 31 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index de9e0dca4fe..faef94607d5 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -128,19 +128,23 @@ try } } CATCH -/* + template void MVCCBuildBitmap(benchmark::State & state, Args &&... args) try { - const auto [type, is_common_handle, delta_rows] = std::make_tuple(std::move(args)...); - initialize(type, is_common_handle, delta_rows); - auto rs_results = loadPackFilterResults(segment_snapshot, {segment->getRowKeyRange()}); + const auto [type, write_load, is_common_handle] = std::make_tuple(std::move(args)...); + const UInt32 delta_rows = state.range(0); + auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] + = initialize(is_common_handle, delta_rows); + SCOPE_EXIT({ context->shutdown(); }); + + auto rs_results = loadPackFilterResults(*dm_context, segment_snapshot, {segment->getRowKeyRange()}); if (type == BenchType::DeltaIndex) { RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == 0); - auto delta_index = buildDeltaIndex(segment_snapshot, *segment); + auto delta_index = buildDeltaIndex(*dm_context, *cols, segment_snapshot, *segment); RUNTIME_ASSERT(delta_index->getPlacedStatus().first == delta_rows); segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*delta_index); RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == delta_rows); @@ -161,7 +165,7 @@ try else if (type == BenchType::VersionChain) { VersionChain version_chain; - buildVersionChain(*segment_snapshot, version_chain); + buildVersionChain(*dm_context, *segment_snapshot, version_chain); RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); for (auto _ : state) { @@ -175,10 +179,8 @@ try benchmark::DoNotOptimize(bitmap_filter); } } - shutdown(); } CATCH -*/ // [ 1, 8, 64, 512, 4k, 8k, 64k ] BENCHMARK_CAPTURE(MVCCFullPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) @@ -190,27 +192,9 @@ BENCHMARK_CAPTURE(MVCCIncrementalPlace, Index, BenchType::DeltaIndex, WriteLoad: ->Range(1, 8 << 13); BENCHMARK_CAPTURE(MVCCIncrementalPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) ->Range(1, 8 << 13); -/* -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_1, BenchType::DeltaIndex, IsNotCommonHandle, 1u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_5, BenchType::DeltaIndex, IsNotCommonHandle, 5u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_10, BenchType::DeltaIndex, IsNotCommonHandle, 10u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_100, BenchType::DeltaIndex, IsNotCommonHandle, 100u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_500, BenchType::DeltaIndex, IsNotCommonHandle, 500u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_1000, BenchType::DeltaIndex, IsNotCommonHandle, 1000u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_5000, BenchType::DeltaIndex, IsNotCommonHandle, 5000u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_10000, BenchType::DeltaIndex, IsNotCommonHandle, 10000u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_50000, BenchType::DeltaIndex, IsNotCommonHandle, 50000u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, delta_index_100000, BenchType::DeltaIndex, IsNotCommonHandle, 100000u); - -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_1, BenchType::VersionChain, IsNotCommonHandle, 1u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_5, BenchType::VersionChain, IsNotCommonHandle, 5u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_10, BenchType::VersionChain, IsNotCommonHandle, 10u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_100, BenchType::VersionChain, IsNotCommonHandle, 100u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_500, BenchType::VersionChain, IsNotCommonHandle, 500u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_1000, BenchType::VersionChain, IsNotCommonHandle, 1000u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_5000, BenchType::VersionChain, IsNotCommonHandle, 5000u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_10000, BenchType::VersionChain, IsNotCommonHandle, 10000u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_50000, BenchType::VersionChain, IsNotCommonHandle, 50000u); -BENCHMARK_CAPTURE(MVCCBuildBitmap, version_chain_100000, BenchType::VersionChain, IsNotCommonHandle, 100000u); -*/ + +BENCHMARK_CAPTURE(MVCCBuildBitmap, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) + ->Range(1, 8 << 13); +BENCHMARK_CAPTURE(MVCCBuildBitmap, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) + ->Range(1, 8 << 13); } // namespace From db6824c65a1f6ff82eaaf4f7ae57ce3cd55eca6c Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 16 Jan 2025 16:45:11 +0800 Subject: [PATCH 30/48] ci --- .../tests/bench_version_chain.cpp | 4 +- .../VersionChain/tests/gtest_mvcc_bitmap.cpp | 60 ++++++------------- 2 files changed, 21 insertions(+), 43 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index faef94607d5..e6e14a70e11 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -64,13 +64,13 @@ try { { VersionChain version_chain; - buildVersionChain(*dm_context, *segment_snapshot, version_chain); // Warming up + buildVersionChain(*dm_context, *segment_snapshot, version_chain); // Warming up RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); } for (auto _ : state) { VersionChain version_chain; - buildVersionChain(*dm_context, *segment_snapshot, version_chain); + buildVersionChain(*dm_context, *segment_snapshot, version_chain); RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); } } diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp index 6672cd83d32..4a95c17f4f2 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp @@ -41,7 +41,7 @@ try VersionChain version_chain; - buildVersionChain(*dm_context, *segment_snapshot, version_chain); + buildVersionChain(*dm_context, *segment_snapshot, version_chain); ASSERT_EQ(version_chain.getReplayedRows(), delta_rows); auto rs_results = loadPackFilterResults(*dm_context, segment_snapshot, {segment->getRowKeyRange()}); @@ -72,51 +72,29 @@ try } CATCH +static constexpr UInt32 max_delta_rows = 8 << 13; + TEST(TestVersionChain, randomMVCCBitmapVerify) { - std::vector delta_rows{ - 1, - 10, - 50, - 100, - 500, - 1000, - 5000, - 10000, - 20000, - 30000, - 40000, - 50000, - 60000, - 70000, - 80000, - 90000, - 100000}; - for (auto rows : delta_rows) - randomMVCCBitmapVerify(rows); + for (UInt32 delta_rows = 1; delta_rows <= max_delta_rows; delta_rows *= 8) + randomMVCCBitmapVerify(delta_rows); } TEST(TestVersionChain, randomMVCCBitmapVerify_CommonHandle) { - std::vector delta_rows{ - 1, - 10, - 50, - 100, - 500, - 1000, - 5000, - 10000, - 20000, - 30000, - 40000, - 50000, - 60000, - 70000, - 80000, - 90000, - 100000}; - for (auto rows : delta_rows) - randomMVCCBitmapVerify(rows); + for (UInt32 delta_rows = 1; delta_rows <= max_delta_rows; delta_rows *= 8) + randomMVCCBitmapVerify(delta_rows); +} + +TEST(TestVersionChain, loadPackFilterResults) +{ + constexpr bool is_common_handle = false; + auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] + = initialize(is_common_handle, 100); + SCOPE_EXIT({ context->shutdown(); }); + auto rs_results = loadPackFilterResults(*dm_context, segment_snapshot, {segment->getRowKeyRange()}); + + fmt::println("PackRes: {}", rs_results[0]->getPackRes()); + fmt::println("HandleRes: {}", rs_results[0]->getHandleRes()); } } // namespace DB::DM::tests From 1ca81ad3425b51140128be8344f64f99cc52ce80 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 16 Jan 2025 16:51:34 +0800 Subject: [PATCH 31/48] ci --- dbms/src/Storages/DeltaMerge/Segment.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index ed913d5de9d..824edcd568c 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -3085,7 +3085,7 @@ BitmapFilterPtr Segment::buildBitmapFilter( sanitizeCheckReadRanges(__FUNCTION__, read_ranges, rowkey_range, log); if (use_version_chain) { - return buildBitmapFilter( + return ::DB::DM::buildBitmapFilter( dm_context, *segment_snap, read_ranges, From 9e44b71d8ebdeed3bc8dfda1e1e27d7ccfabbcdb Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 16 Jan 2025 17:25:27 +0800 Subject: [PATCH 32/48] ci --- .../VersionChain/BuildBitmapFilter.cpp | 5 +-- .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 33 +++++++++---------- .../DeltaMerge/VersionChain/RowKeyFilter.h | 2 +- .../VersionChain/tests/gtest_mvcc_bitmap.cpp | 12 ------- 4 files changed, 17 insertions(+), 35 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index d8d03a8994a..2b63f1faa9a 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -43,10 +43,7 @@ BitmapFilterPtr buildBitmapFilter( auto & filter = bitmap_filter->getFilter(); RUNTIME_CHECK(pack_filter_results.size() == 1, pack_filter_results.size()); - const auto stable_pack_res = pack_filter_results.front()->getPackRes(); - - //fmt::println("stable_rows={}, delta_rows={}, stable_pack_res={}", stable_rows, delta_rows, stable_pack_res); - buildRowKeyFilter(dm_context, snapshot, read_ranges, stable_pack_res, filter); + buildRowKeyFilter(dm_context, snapshot, read_ranges, pack_filter_results[0], filter); buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); buildDeletedFilter(dm_context, snapshot, filter); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index f081e080c94..776c4584a8d 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -80,7 +80,7 @@ UInt32 buildRowKeyFilterDMFile( const std::optional & segment_range, const RowKeyRanges & delete_ranges, const RowKeyRanges & read_ranges, - const RSResults * stable_pack_res, + const DMFilePackFilterResultPtr & stable_filter_res, const UInt32 start_row_id, IColumn::Filter & filter) { @@ -88,22 +88,19 @@ UInt32 buildRowKeyFilterDMFile( if (unlikely(valid_handle_res.empty())) return 0; - if (stable_pack_res) + // Filter out these packs that don't need to read. + if (stable_filter_res) { - const auto & s_pack_res = *stable_pack_res; - RUNTIME_CHECK(s_pack_res.size() == valid_handle_res.size(), s_pack_res.size(), valid_handle_res.size()); - // stable_pack_res is the result of `read_ranges` && `rs_filter`. - // If the result of a pack filtered by read_ranges is All, but filtered by `filter` is Some, its result - // is Some in stable_pack_res. - // Result of `rs_filter` is no help here, because we just want to filter out rowkey here. - // So only use the None results of stable_pack_res + RUNTIME_CHECK(valid_start_pack_id == 0, valid_start_pack_id); + const auto & pack_res = stable_filter_res->getPackRes(); + RUNTIME_CHECK(pack_res.size() == valid_handle_res.size(), pack_res.size(), valid_handle_res.size()); for (UInt32 i = 0; i < valid_handle_res.size(); ++i) - if (!s_pack_res[i].isUse()) + if (!pack_res[i].isUse()) valid_handle_res[i] = RSResult::None; } // RSResult of read_ranges. - const auto read_ranges_handle_res = getRSResultsByRanges(dm_context, dmfile, read_ranges); + const auto & read_ranges_handle_res = stable_filter_res ? stable_filter_res->getHandleRes() : getRSResultsByRanges(dm_context, dmfile, read_ranges); for (UInt32 i = 0; i < valid_handle_res.size(); ++i) valid_handle_res[i] = valid_handle_res[i] && read_ranges_handle_res[valid_start_pack_id + i]; @@ -180,7 +177,7 @@ UInt32 buildRowKeyFilterColumnFileBig( cf_big.getRange(), delete_ranges, Segment::shrinkRowKeyRanges(cf_big.getRange(), read_ranges), - /*stable_pack_res*/ nullptr, + /*stable_filter_res*/ nullptr, start_row_id, filter); } @@ -191,7 +188,7 @@ UInt32 buildRowKeyFilterStable( const StableValueSpace::Snapshot & stable, const RowKeyRanges & delete_ranges, const RowKeyRanges & read_ranges, - const RSResults & stable_pack_res, + const DMFilePackFilterResultPtr & stable_filter_res, IColumn::Filter & filter) { const auto & dmfiles = stable.getDMFiles(); @@ -205,7 +202,7 @@ UInt32 buildRowKeyFilterStable( /*segment_range*/ std::nullopt, delete_ranges, read_ranges, - &stable_pack_res, + stable_filter_res, /*start_row_id*/ 0, filter); } @@ -217,7 +214,7 @@ void buildRowKeyFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, - const RSResults & stable_pack_res, + const DMFilePackFilterResultPtr & stable_filter_res, IColumn::Filter & filter) { const auto & delta = *(snapshot.delta); @@ -277,7 +274,7 @@ void buildRowKeyFilter( RUNTIME_CHECK(read_rows == delta_rows, read_rows, delta_rows); const auto n - = buildRowKeyFilterStable(dm_context, stable, delete_ranges, read_ranges, stable_pack_res, filter); + = buildRowKeyFilterStable(dm_context, stable, delete_ranges, read_ranges, stable_filter_res, filter); RUNTIME_CHECK(n == stable_rows, n, stable_rows); } @@ -285,13 +282,13 @@ template void buildRowKeyFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, - const RSResults & stable_pack_res, + const DMFilePackFilterResultPtr & filter_res, IColumn::Filter & filter); template void buildRowKeyFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, - const RSResults & stable_pack_res, + const DMFilePackFilterResultPtr & filter_res, IColumn::Filter & filter); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h index a66037989e7..7810ac10ae3 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.h @@ -29,6 +29,6 @@ void buildRowKeyFilter( const DMContext & dm_context, const SegmentSnapshot & snapshot, const RowKeyRanges & read_ranges, - const RSResults & stable_pack_res, + const DMFilePackFilterResultPtr & stable_filter_res, IColumn::Filter & filter); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp index 4a95c17f4f2..206452f50d6 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp @@ -85,16 +85,4 @@ TEST(TestVersionChain, randomMVCCBitmapVerify_CommonHandle) for (UInt32 delta_rows = 1; delta_rows <= max_delta_rows; delta_rows *= 8) randomMVCCBitmapVerify(delta_rows); } - -TEST(TestVersionChain, loadPackFilterResults) -{ - constexpr bool is_common_handle = false; - auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] - = initialize(is_common_handle, 100); - SCOPE_EXIT({ context->shutdown(); }); - auto rs_results = loadPackFilterResults(*dm_context, segment_snapshot, {segment->getRowKeyRange()}); - - fmt::println("PackRes: {}", rs_results[0]->getPackRes()); - fmt::println("HandleRes: {}", rs_results[0]->getHandleRes()); -} } // namespace DB::DM::tests From a8b8ae4c198691e314f9859fba95e6da6e273ac4 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 16 Jan 2025 17:26:20 +0800 Subject: [PATCH 33/48] ci --- .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index 776c4584a8d..a1ed43cb896 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -100,7 +100,8 @@ UInt32 buildRowKeyFilterDMFile( } // RSResult of read_ranges. - const auto & read_ranges_handle_res = stable_filter_res ? stable_filter_res->getHandleRes() : getRSResultsByRanges(dm_context, dmfile, read_ranges); + const auto & read_ranges_handle_res + = stable_filter_res ? stable_filter_res->getHandleRes() : getRSResultsByRanges(dm_context, dmfile, read_ranges); for (UInt32 i = 0; i < valid_handle_res.size(); ++i) valid_handle_res[i] = valid_handle_res[i] && read_ranges_handle_res[valid_start_pack_id + i]; @@ -273,8 +274,13 @@ void buildRowKeyFilter( } RUNTIME_CHECK(read_rows == delta_rows, read_rows, delta_rows); - const auto n - = buildRowKeyFilterStable(dm_context, stable, delete_ranges, read_ranges, stable_filter_res, filter); + const auto n = buildRowKeyFilterStable( + dm_context, + stable, + delete_ranges, + read_ranges, + stable_filter_res, + filter); RUNTIME_CHECK(n == stable_rows, n, stable_rows); } From 9ba7936b3a8fc725968bcf08380f731b7b3c16cc Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 17 Jan 2025 15:13:43 +0800 Subject: [PATCH 34/48] ci --- .../VersionChain/tests/bench_version_chain.cpp | 17 +++++++++++------ .../VersionChain/tests/mvcc_test_utils.h | 4 ++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index e6e14a70e11..8171ff0f6df 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -183,18 +183,23 @@ try CATCH // [ 1, 8, 64, 512, 4k, 8k, 64k ] -BENCHMARK_CAPTURE(MVCCFullPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) +BENCHMARK_CAPTURE(MVCCFullPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, NotCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCFullPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) +BENCHMARK_CAPTURE(MVCCFullPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, NotCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCIncrementalPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) +BENCHMARK_CAPTURE(MVCCIncrementalPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, NotCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCIncrementalPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) +BENCHMARK_CAPTURE(MVCCIncrementalPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, NotCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCBuildBitmap, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsNotCommonHandle) +BENCHMARK_CAPTURE(MVCCBuildBitmap, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, NotCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCBuildBitmap, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsNotCommonHandle) +BENCHMARK_CAPTURE(MVCCBuildBitmap, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, NotCommonHandle) + ->Range(1, 8 << 13); + +BENCHMARK_CAPTURE(MVCCFullPlace, IndexCommonHandle, BenchType::DeltaIndex, WriteLoad::RandomUpdate, CommonHandle) + ->Range(1, 8 << 13); +BENCHMARK_CAPTURE(MVCCFullPlace, ChainCommonHandle, BenchType::VersionChain, WriteLoad::RandomUpdate, CommonHandle) ->Range(1, 8 << 13); } // namespace diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h index 140584daa1b..4af157eee2f 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h @@ -50,8 +50,8 @@ enum class WriteLoad RandomInsert = 3, }; -// constexpr bool IsCommonHandle = true; -constexpr bool IsNotCommonHandle = false; +constexpr bool CommonHandle = true; +constexpr bool NotCommonHandle = false; auto loadPackFilterResults(const DMContext & dm_context, const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) From cece60290112c1f2bb559edebb503e784ab07138 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 17 Jan 2025 16:20:03 +0800 Subject: [PATCH 35/48] ci --- .../tests/bench_version_chain.cpp | 108 +++++++++++------- .../VersionChain/tests/mvcc_test_utils.h | 2 +- 2 files changed, 69 insertions(+), 41 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 8171ff0f6df..34156582d00 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -62,17 +62,23 @@ try } else if (type == BenchType::VersionChain) { - { - VersionChain version_chain; - buildVersionChain(*dm_context, *segment_snapshot, version_chain); // Warming up - RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); - } - for (auto _ : state) - { - VersionChain version_chain; - buildVersionChain(*dm_context, *segment_snapshot, version_chain); - RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); - } + auto bench_impl = [&](auto handle_type) { + { + VersionChain version_chain; + buildVersionChain(*dm_context, *segment_snapshot, version_chain); // Warming up + RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); + } + for (auto _ : state) + { + VersionChain version_chain; + buildVersionChain(*dm_context, *segment_snapshot, version_chain); + RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); + } + }; + if (is_common_handle) + bench_impl(String{}); + else + bench_impl(Int64{}); } } CATCH @@ -112,19 +118,25 @@ try } else if (type == BenchType::VersionChain) { - VersionChain base_version_chain; - buildVersionChain(*dm_context, *segment_snapshot, base_version_chain); - RUNTIME_ASSERT(base_version_chain.getReplayedRows() == prepared_delta_rows); - writeDelta(*dm_context, is_common_handle, *segment, incremental_delta_rows, random_sequences); - segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); - RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); - for (auto _ : state) - { - auto version_chain = base_version_chain; - RUNTIME_ASSERT(version_chain.getReplayedRows() == prepared_delta_rows); - buildVersionChain(*dm_context, *segment_snapshot, version_chain); - RUNTIME_ASSERT(version_chain.getReplayedRows() == prepared_delta_rows + incremental_delta_rows); - } + auto bench_impl = [&](auto handle_type) { + VersionChain base_version_chain; + buildVersionChain(*dm_context, *segment_snapshot, base_version_chain); + RUNTIME_ASSERT(base_version_chain.getReplayedRows() == prepared_delta_rows); + writeDelta(*dm_context, is_common_handle, *segment, incremental_delta_rows, random_sequences); + segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); + RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); + for (auto _ : state) + { + auto version_chain = base_version_chain; + RUNTIME_ASSERT(version_chain.getReplayedRows() == prepared_delta_rows); + buildVersionChain(*dm_context, *segment_snapshot, version_chain); + RUNTIME_ASSERT(version_chain.getReplayedRows() == prepared_delta_rows + incremental_delta_rows); + } + }; + if (is_common_handle) + bench_impl(String{}); + else + bench_impl(Int64{}); } } CATCH @@ -164,20 +176,26 @@ try } else if (type == BenchType::VersionChain) { - VersionChain version_chain; - buildVersionChain(*dm_context, *segment_snapshot, version_chain); - RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); - for (auto _ : state) - { - auto bitmap_filter = buildBitmapFilter( - *dm_context, - *segment_snapshot, - {segment->getRowKeyRange()}, - rs_results, - std::numeric_limits::max(), - version_chain); - benchmark::DoNotOptimize(bitmap_filter); - } + auto bench_impl = [&](auto handle_type) { + VersionChain version_chain; + buildVersionChain(*dm_context, *segment_snapshot, version_chain); + RUNTIME_ASSERT(version_chain.getReplayedRows() == delta_rows); + for (auto _ : state) + { + auto bitmap_filter = buildBitmapFilter( + *dm_context, + *segment_snapshot, + {segment->getRowKeyRange()}, + rs_results, + std::numeric_limits::max(), + version_chain); + benchmark::DoNotOptimize(bitmap_filter); + } + }; + if (is_common_handle) + bench_impl(String{}); + else + bench_impl(Int64{}); } } CATCH @@ -198,8 +216,18 @@ BENCHMARK_CAPTURE(MVCCBuildBitmap, Index, BenchType::DeltaIndex, WriteLoad::Rand BENCHMARK_CAPTURE(MVCCBuildBitmap, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, NotCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCFullPlace, IndexCommonHandle, BenchType::DeltaIndex, WriteLoad::RandomUpdate, CommonHandle) +BENCHMARK_CAPTURE(MVCCFullPlace, CommonHandleIndex, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsCommonHandle) + ->Range(1, 8 << 13); +BENCHMARK_CAPTURE(MVCCFullPlace, CommonHandleChain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsCommonHandle) + ->Range(1, 8 << 13); + +BENCHMARK_CAPTURE(MVCCIncrementalPlace, CommonHandleIndex, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsCommonHandle) + ->Range(1, 8 << 13); +BENCHMARK_CAPTURE(MVCCIncrementalPlace, CommonHandleChain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsCommonHandle) + ->Range(1, 8 << 13); + +BENCHMARK_CAPTURE(MVCCBuildBitmap, CommonHandleIndex, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCFullPlace, ChainCommonHandle, BenchType::VersionChain, WriteLoad::RandomUpdate, CommonHandle) +BENCHMARK_CAPTURE(MVCCBuildBitmap, CommonHandleChain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsCommonHandle) ->Range(1, 8 << 13); } // namespace diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h index 4af157eee2f..375353dcfad 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h @@ -50,7 +50,7 @@ enum class WriteLoad RandomInsert = 3, }; -constexpr bool CommonHandle = true; +constexpr bool IsCommonHandle = true; constexpr bool NotCommonHandle = false; From 95249489684132c7782d01aedbf9debfa8e337e0 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Fri, 17 Jan 2025 18:02:49 +0800 Subject: [PATCH 36/48] ci --- .../tests/bench_version_chain.cpp | 30 ++-- .../VersionChain/tests/mvcc_test_utils.h | 145 ++++++++++-------- 2 files changed, 104 insertions(+), 71 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index 34156582d00..fa9090aef09 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -42,8 +42,8 @@ try { const auto [type, write_load, is_common_handle] = std::make_tuple(std::move(args)...); const UInt32 delta_rows = state.range(0); - auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] - = initialize(is_common_handle, delta_rows); + auto [context, dm_context, cols, segment, segment_snapshot, write_seq] + = initialize(write_load, is_common_handle, delta_rows); SCOPE_EXIT({ context->shutdown(); }); if (type == BenchType::DeltaIndex) @@ -90,8 +90,8 @@ try const auto [type, write_load, is_common_handle] = std::make_tuple(std::move(args)...); const UInt32 incremental_delta_rows = state.range(0); constexpr UInt32 prepared_delta_rows = 10000; - auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] - = initialize(is_common_handle, prepared_delta_rows); + auto [context, dm_context, cols, segment, segment_snapshot, write_seq] + = initialize(write_load, is_common_handle, prepared_delta_rows); SCOPE_EXIT({ context->shutdown(); }); if (type == BenchType::DeltaIndex) @@ -103,7 +103,7 @@ try segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*base_delta_index); RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == prepared_delta_rows); - writeDelta(*dm_context, is_common_handle, *segment, incremental_delta_rows, random_sequences); + writeDelta(*dm_context, is_common_handle, *segment, incremental_delta_rows, *write_seq); segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); RUNTIME_ASSERT(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first == prepared_delta_rows); RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); @@ -122,7 +122,7 @@ try VersionChain base_version_chain; buildVersionChain(*dm_context, *segment_snapshot, base_version_chain); RUNTIME_ASSERT(base_version_chain.getReplayedRows() == prepared_delta_rows); - writeDelta(*dm_context, is_common_handle, *segment, incremental_delta_rows, random_sequences); + writeDelta(*dm_context, is_common_handle, *segment, incremental_delta_rows, *write_seq); segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); RUNTIME_ASSERT(segment_snapshot->delta->getRows() == prepared_delta_rows + incremental_delta_rows); for (auto _ : state) @@ -147,8 +147,8 @@ try { const auto [type, write_load, is_common_handle] = std::make_tuple(std::move(args)...); const UInt32 delta_rows = state.range(0); - auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] - = initialize(is_common_handle, delta_rows); + auto [context, dm_context, cols, segment, segment_snapshot, write_seq] + = initialize(write_load, is_common_handle, delta_rows); SCOPE_EXIT({ context->shutdown(); }); auto rs_results = loadPackFilterResults(*dm_context, segment_snapshot, {segment->getRowKeyRange()}); @@ -221,9 +221,19 @@ BENCHMARK_CAPTURE(MVCCFullPlace, CommonHandleIndex, BenchType::DeltaIndex, Write BENCHMARK_CAPTURE(MVCCFullPlace, CommonHandleChain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCIncrementalPlace, CommonHandleIndex, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsCommonHandle) +BENCHMARK_CAPTURE( + MVCCIncrementalPlace, + CommonHandleIndex, + BenchType::DeltaIndex, + WriteLoad::RandomUpdate, + IsCommonHandle) ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCIncrementalPlace, CommonHandleChain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsCommonHandle) +BENCHMARK_CAPTURE( + MVCCIncrementalPlace, + CommonHandleChain, + BenchType::VersionChain, + WriteLoad::RandomUpdate, + IsCommonHandle) ->Range(1, 8 << 13); BENCHMARK_CAPTURE(MVCCBuildBitmap, CommonHandleIndex, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsCommonHandle) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h index 375353dcfad..d120c661d51 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h @@ -32,9 +32,11 @@ extern const Metric DT_SnapshotOfRead; namespace DB::DM::tests::MVCC { -const String db_name = "test"; +inline const String db_name = "test"; +inline UInt64 version = 1; -UInt64 version = 1; +inline constexpr bool IsCommonHandle = true; +inline constexpr bool NotCommonHandle = false; enum class BenchType { @@ -50,11 +52,10 @@ enum class WriteLoad RandomInsert = 3, }; -constexpr bool IsCommonHandle = true; -constexpr bool NotCommonHandle = false; - - -auto loadPackFilterResults(const DMContext & dm_context, const SegmentSnapshotPtr & snap, const RowKeyRanges & ranges) +inline auto loadPackFilterResults( + const DMContext & dm_context, + const SegmentSnapshotPtr & snap, + const RowKeyRanges & ranges) { DMFilePackFilterResults results; results.reserve(snap->stable->getDMFiles().size()); @@ -66,7 +67,7 @@ auto loadPackFilterResults(const DMContext & dm_context, const SegmentSnapshotPt return results; } -auto getContext() +inline auto getContext() { const auto table_name = std::to_string(clock_gettime_ns()); const auto testdata_path = fmt::format("/tmp/{}", table_name); @@ -75,7 +76,7 @@ auto getContext() return std::pair{TiFlashTestEnv::getContext(), std::move(table_name)}; } -auto getDMContext(Context & context, const String & table_name, bool is_common_handle) +inline auto getDMContext(Context & context, const String & table_name, bool is_common_handle) { auto storage_path_pool = std::make_shared(context.getPathPool().withTable(db_name, table_name, false)); @@ -105,7 +106,7 @@ auto getDMContext(Context & context, const String & table_name, bool is_common_h return std::pair{std::move(dm_context), std::move(cols)}; } -SegmentPtr createSegment(DMContext & dm_context, const ColumnDefinesPtr & cols, bool is_common_handle) +inline SegmentPtr createSegment(DMContext & dm_context, const ColumnDefinesPtr & cols, bool is_common_handle) { return Segment::newSegment( Logger::get(), @@ -116,32 +117,44 @@ SegmentPtr createSegment(DMContext & dm_context, const ColumnDefinesPtr & cols, 0); } -constexpr Int64 MaxHandle = 1000000; +class WriteSequence +{ +public: + virtual ~WriteSequence() = default; + virtual std::vector getStable() = 0; + virtual std::vector getDelta(UInt32 n) = 0; +}; -class RandomSequence +class RandomUpdateSequence : public WriteSequence { public: - RandomSequence(UInt32 n) - : v(randomInt64s(n)) - , pos(v.begin()) + RandomUpdateSequence() + : max_handle(1000000) + , rnd_v(randomInt64s(max_handle)) + , pos(rnd_v.begin()) {} - std::vector get(UInt32 n) + std::vector getStable() override + { + std::vector v(max_handle); + std::iota(v.begin(), v.end(), 0); + return v; + } + + std::vector getDelta(UInt32 n) override { std::vector res; while (res.size() < n) { - auto copied = std::min(std::distance(pos, v.end()), static_cast(n - res.size())); + auto copied = std::min(std::distance(pos, rnd_v.end()), static_cast(n - res.size())); res.insert(res.end(), pos, pos + copied); std::advance(pos, copied); - if (pos == v.end()) - reset(); + if (pos == rnd_v.end()) + pos = rnd_v.begin(); } return res; } - void reset() { pos = v.begin(); } - private: std::vector randomInt64s(UInt32 n) { @@ -150,16 +163,29 @@ class RandomSequence std::vector v(n); for (UInt32 i = 0; i < n; ++i) { - v[i] = g() % MaxHandle; + v[i] = g() % max_handle; } return v; } - std::vector v; - std::vector::iterator pos; + const UInt32 max_handle; + const std::vector rnd_v; + std::vector::const_iterator pos; }; -Strings toMockCommonHandles(const std::vector & v) +std::unique_ptr createWriteSequence(WriteLoad write_load) +{ + switch (write_load) + { + case WriteLoad::RandomUpdate: + return std::make_unique(); + case WriteLoad::AppendOnly: + case WriteLoad::RandomInsert: + return nullptr; + } +} + +inline Strings toMockCommonHandles(const std::vector & v) { Strings handles; for (Int64 i : v) @@ -167,61 +193,58 @@ Strings toMockCommonHandles(const std::vector & v) return handles; } -void writeDelta( +template +Block createBlock(std::vector handles) +{ + Block block; + block.insert(createColumn(handles, MutSup::extra_handle_column_name, MutSup::extra_handle_id)); + + static UInt64 version = 1; + block.insert(createColumn( + std::vector(block.rows(), version++), + MutSup::version_column_name, + MutSup::version_col_id)); + + block.insert(createColumn( + std::vector(block.rows(), /*deleted*/ 0), + MutSup::delmark_column_name, + MutSup::delmark_col_id)); + return block; +} + +inline void writeDelta( DMContext & dm_context, bool is_common_handle, Segment & seg, UInt32 delta_rows, - RandomSequence & random_sequences) + WriteSequence & write_seq) { for (UInt32 i = 0; i < delta_rows; i += 2048) { - Block block; const auto n = std::min(delta_rows - i, 2048U); - const auto v = random_sequences.get(n); - if (is_common_handle) - block.insert(createColumn( - toMockCommonHandles(v), - MutSup::extra_handle_column_name, - MutSup::extra_handle_id)); - else - block.insert(createColumn(v, MutSup::extra_handle_column_name, MutSup::extra_handle_id)); - block.insert(createColumn( - std::vector(n, version++), - MutSup::version_column_name, - MutSup::version_col_id)); - block.insert(createColumn( - std::vector(n, /*deleted*/ 0), - MutSup::delmark_column_name, - MutSup::delmark_col_id)); + const auto v = write_seq.getDelta(n); + auto block = is_common_handle ? createBlock(toMockCommonHandles(v)) : createBlock(std::move(v)); seg.write(dm_context, block, false); } } -SegmentPtr createSegmentWithData( +inline SegmentPtr createSegmentWithData( DMContext & dm_context, const ColumnDefinesPtr & cols, bool is_common_handle, UInt32 delta_rows, - RandomSequence & random_sequences) + WriteSequence & write_seq) { auto seg = createSegment(dm_context, cols, is_common_handle); - auto block = DMTestEnv::prepareSimpleWriteBlock( - 0, - 1000000, - false, - version++, - MutSup::extra_handle_column_name, - MutSup::extra_handle_id, - is_common_handle ? MutSup::getExtraHandleColumnStringType() : MutSup::getExtraHandleColumnIntType(), - is_common_handle); + auto v = write_seq.getStable(); + auto block = is_common_handle ? createBlock(toMockCommonHandles(v)) : createBlock(std::move(v)); seg->write(dm_context, block, false); seg = seg->mergeDelta(dm_context, cols); - writeDelta(dm_context, is_common_handle, *seg, delta_rows, random_sequences); + writeDelta(dm_context, is_common_handle, *seg, delta_rows, write_seq); return seg; } -DeltaIndexPtr buildDeltaIndex( +inline DeltaIndexPtr buildDeltaIndex( const DMContext & dm_context, const ColumnDefines & cols, const SegmentSnapshotPtr & snapshot, @@ -255,12 +278,12 @@ auto buildVersionChain(const DMContext & dm_context, const SegmentSnapshot & sna return version_chain.replaySnapshot(dm_context, snapshot); } -auto initialize(bool is_common_handle, UInt32 delta_rows) +inline auto initialize(WriteLoad write_load, bool is_common_handle, UInt32 delta_rows) { auto [context, table_name] = getContext(); auto [dm_context, cols] = getDMContext(*context, table_name, is_common_handle); - RandomSequence random_sequences{10 * MaxHandle}; - auto segment = createSegmentWithData(*dm_context, cols, is_common_handle, delta_rows, random_sequences); + auto write_seq = createWriteSequence(write_load); + auto segment = createSegmentWithData(*dm_context, cols, is_common_handle, delta_rows, *write_seq); auto segment_snapshot = segment->createSnapshot(*dm_context, false, CurrentMetrics::DT_SnapshotOfRead); return std::tuple{ std::move(context), @@ -268,7 +291,7 @@ auto initialize(bool is_common_handle, UInt32 delta_rows) std::move(cols), std::move(segment), std::move(segment_snapshot), - std::move(random_sequences)}; + std::move(write_seq)}; } } // namespace DB::DM::tests::MVCC From ce48594692a56c4415eb68b2b7e9dda50c9d35b1 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Mon, 20 Jan 2025 15:51:34 +0800 Subject: [PATCH 37/48] ci --- .../VersionChain/tests/mvcc_test_utils.h | 76 ++++++++++++++----- 1 file changed, 59 insertions(+), 17 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h index d120c661d51..1170375be1b 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/mvcc_test_utils.h @@ -123,6 +123,23 @@ class WriteSequence virtual ~WriteSequence() = default; virtual std::vector getStable() = 0; virtual std::vector getDelta(UInt32 n) = 0; + +protected: + std::vector getVec(Int64 stable_start, Int64 stable_end) + { + std::vector v(stable_end - stable_start); + std::iota(v.begin(), v.end(), stable_start); + return v; + } + + std::vector getRandoms(Int64 rand_start, Int64 rand_end) + { + auto v = getVec(rand_start, rand_end); + static constexpr int rnd_seed = 573172; + std::mt19937 g(rnd_seed); + std::shuffle(v.begin(), v.end(), g); + return v; + } }; class RandomUpdateSequence : public WriteSequence @@ -130,16 +147,11 @@ class RandomUpdateSequence : public WriteSequence public: RandomUpdateSequence() : max_handle(1000000) - , rnd_v(randomInt64s(max_handle)) + , rnd_v(getRandoms(0, max_handle)) , pos(rnd_v.begin()) {} - std::vector getStable() override - { - std::vector v(max_handle); - std::iota(v.begin(), v.end(), 0); - return v; - } + std::vector getStable() override { return getVec(0, max_handle); } std::vector getDelta(UInt32 n) override { @@ -156,20 +168,49 @@ class RandomUpdateSequence : public WriteSequence } private: - std::vector randomInt64s(UInt32 n) + const Int64 max_handle; + const std::vector rnd_v; + std::vector::const_iterator pos; +}; + +class AppendOnlySequence : public WriteSequence +{ +public: + std::vector getStable() override { return getVec(0, curr_handle); } + + std::vector getDelta(UInt32 n) override { - static constexpr int rnd_seed = 573172; - std::mt19937 g(rnd_seed); std::vector v(n); - for (UInt32 i = 0; i < n; ++i) - { - v[i] = g() % max_handle; - } + std::iota(v.begin(), v.end(), curr_handle); + curr_handle += n; return v; } - const UInt32 max_handle; - const std::vector rnd_v; +private: + Int64 curr_handle = 1000000; +}; + +class RandomInsertSequence : public WriteSequence +{ +public: + RandomInsertSequence() + : rnds(getRandoms(0, 10000000)) // 1kw + , pos(rnds.begin()) + {} + + std::vector getStable() override { return std::vector(rnds.begin(), rnds.begin() + 1000000); } + + std::vector getDelta(UInt32 n) override + { + RUNTIME_CHECK(std::distance(pos, rnds.end()) >= n, std::distance(pos, rnds.end()), n); + std::vector res; + res.insert(res.end(), pos, pos + n); + std::advance(pos, n); + return res; + } + +private: + const std::vector rnds; std::vector::const_iterator pos; }; @@ -180,8 +221,9 @@ std::unique_ptr createWriteSequence(WriteLoad write_load) case WriteLoad::RandomUpdate: return std::make_unique(); case WriteLoad::AppendOnly: + return std::make_unique(); case WriteLoad::RandomInsert: - return nullptr; + return std::make_unique(); } } From 90528a90e03e4110c01fa5a91205a1234926c346 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Mon, 20 Jan 2025 21:42:05 +0800 Subject: [PATCH 38/48] ci --- .../tests/bench_version_chain.cpp | 98 +++++++++++-------- 1 file changed, 59 insertions(+), 39 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp index fa9090aef09..7922034b7f6 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_version_chain.cpp @@ -201,43 +201,63 @@ try CATCH // [ 1, 8, 64, 512, 4k, 8k, 64k ] -BENCHMARK_CAPTURE(MVCCFullPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, NotCommonHandle) - ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCFullPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, NotCommonHandle) - ->Range(1, 8 << 13); - -BENCHMARK_CAPTURE(MVCCIncrementalPlace, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, NotCommonHandle) - ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCIncrementalPlace, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, NotCommonHandle) - ->Range(1, 8 << 13); - -BENCHMARK_CAPTURE(MVCCBuildBitmap, Index, BenchType::DeltaIndex, WriteLoad::RandomUpdate, NotCommonHandle) - ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCBuildBitmap, Chain, BenchType::VersionChain, WriteLoad::RandomUpdate, NotCommonHandle) - ->Range(1, 8 << 13); - -BENCHMARK_CAPTURE(MVCCFullPlace, CommonHandleIndex, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsCommonHandle) - ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCFullPlace, CommonHandleChain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsCommonHandle) - ->Range(1, 8 << 13); - -BENCHMARK_CAPTURE( - MVCCIncrementalPlace, - CommonHandleIndex, - BenchType::DeltaIndex, - WriteLoad::RandomUpdate, - IsCommonHandle) - ->Range(1, 8 << 13); -BENCHMARK_CAPTURE( - MVCCIncrementalPlace, - CommonHandleChain, - BenchType::VersionChain, - WriteLoad::RandomUpdate, - IsCommonHandle) - ->Range(1, 8 << 13); - -BENCHMARK_CAPTURE(MVCCBuildBitmap, CommonHandleIndex, BenchType::DeltaIndex, WriteLoad::RandomUpdate, IsCommonHandle) - ->Range(1, 8 << 13); -BENCHMARK_CAPTURE(MVCCBuildBitmap, CommonHandleChain, BenchType::VersionChain, WriteLoad::RandomUpdate, IsCommonHandle) - ->Range(1, 8 << 13); +#define MVCC_BENCHMARK(FUNC) \ + BENCHMARK_CAPTURE(FUNC, Index / RandomUpdate, BenchType::DeltaIndex, WriteLoad::RandomUpdate, NotCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE(FUNC, Chain / RandomUpdate, BenchType::VersionChain, WriteLoad::RandomUpdate, NotCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE(FUNC, Index / AppendOnly, BenchType::DeltaIndex, WriteLoad::AppendOnly, NotCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE(FUNC, Chain / AppendOnly, BenchType::VersionChain, WriteLoad::AppendOnly, NotCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE(FUNC, Index / RandomInsert, BenchType::DeltaIndex, WriteLoad::RandomInsert, NotCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE(FUNC, Chain / RandomInsert, BenchType::VersionChain, WriteLoad::RandomInsert, NotCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE( \ + FUNC, \ + CommonHandle / Index / RandomUpdate, \ + BenchType::DeltaIndex, \ + WriteLoad::RandomUpdate, \ + IsCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE( \ + FUNC, \ + CommonHandle / Chain / RandomUpdate, \ + BenchType::VersionChain, \ + WriteLoad::RandomUpdate, \ + IsCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE( \ + FUNC, \ + CommonHandle / Index / AppendOnly, \ + BenchType::DeltaIndex, \ + WriteLoad::AppendOnly, \ + IsCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE( \ + FUNC, \ + CommonHandle / Chain / AppendOnly, \ + BenchType::VersionChain, \ + WriteLoad::AppendOnly, \ + IsCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE( \ + FUNC, \ + CommonHandle / Index / RandomInsert, \ + BenchType::DeltaIndex, \ + WriteLoad::RandomInsert, \ + IsCommonHandle) \ + ->Range(1, 8 << 13); \ + BENCHMARK_CAPTURE( \ + FUNC, \ + CommonHandle / Chain / RandomInsert, \ + BenchType::VersionChain, \ + WriteLoad::RandomInsert, \ + IsCommonHandle) \ + ->Range(1, 8 << 13); + +MVCC_BENCHMARK(MVCCFullPlace) +MVCC_BENCHMARK(MVCCIncrementalPlace) +MVCC_BENCHMARK(MVCCBuildBitmap) } // namespace From a819c81dca92459273c42f8cb20528d3101ca141 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 22 Jan 2025 11:30:22 +0800 Subject: [PATCH 39/48] ci --- dbms/src/Storages/DeltaMerge/File/DMFile.h | 5 - .../DeltaMerge/tests/bench_vector_index.cpp | 98 ---------- .../tests/bench_vector_index_utils.h | 177 ------------------ 3 files changed, 280 deletions(-) delete mode 100644 dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp delete mode 100644 dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.h b/dbms/src/Storages/DeltaMerge/File/DMFile.h index 2bca2e39f7f..6b2ed49ffaa 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.h @@ -305,11 +305,6 @@ class DMFile : private boost::noncopyable EncryptionPath encryptionIndexPath(const FileNameBase & file_name_base) const; EncryptionPath encryptionMarkPath(const FileNameBase & file_name_base) const; - static FileNameBase getFileNameBase(ColId col_id, const IDataType::SubstreamPath & substream = {}) - { - return IDataType::getFileNameForStream(DB::toString(col_id), substream); - } - static String localIndexFileName(IndexID index_id, TiDB::ColumnarIndexKind kind) { // Note: Keep sync with FileCache::getFileType() diff --git a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp b/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp deleted file mode 100644 index 7792664168a..00000000000 --- a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index.cpp +++ /dev/null @@ -1,98 +0,0 @@ -// Copyright 2024 PingCAP, Inc. -// -// 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 -#include -#include - -namespace DB::DM::bench -{ - -static void VectorIndexBuild(::benchmark::State & state) -try -{ - const auto & dataset = DatasetMnist::get(); - - auto train_data = dataset.buildDataTrainColumn(/* max_rows= */ 10000); - auto index_def = dataset.createIndexDef(tipb::VectorIndexKind::HNSW); - for (auto _ : state) - { - auto builder = std::make_unique(0, index_def); - builder->addBlock(*train_data, nullptr, []() { return true; }); - } -} -CATCH - -static void VectorIndexSearchTop10(::benchmark::State & state) -try -{ - const auto & dataset = DatasetMnist::get(); - - auto index_path = DB::tests::TiFlashTestEnv::getTemporaryPath("vector_search_top_10/vector_index.idx"); - VectorIndexBenchUtils::saveVectorIndex( // - index_path, - dataset, - /* max_rows= */ 10000); - - auto viewer = VectorIndexBenchUtils::viewVectorIndex(index_path, dataset); - - std::random_device rd; - std::mt19937 rng(rd()); - std::uniform_int_distribution dist(0, dataset.dataTestSize() - 1); - - for (auto _ : state) - { - auto test_index = dist(rng); - const auto & query_vector = DatasetMnist::get().dataTestAt(test_index); - auto keys = VectorIndexBenchUtils::queryTopK(viewer, query_vector, 10, state); - RUNTIME_CHECK(keys.size() == 10); - } -} -CATCH - -static void VectorIndexSearchTop100(::benchmark::State & state) -try -{ - const auto & dataset = DatasetMnist::get(); - - auto index_path = DB::tests::TiFlashTestEnv::getTemporaryPath("vector_search_top_10/vector_index.idx"); - VectorIndexBenchUtils::saveVectorIndex( // - index_path, - dataset, - /* max_rows= */ 10000); - - auto viewer = VectorIndexBenchUtils::viewVectorIndex(index_path, dataset); - - std::random_device rd; - std::mt19937 rng(rd()); - std::uniform_int_distribution dist(0, dataset.dataTestSize() - 1); - - for (auto _ : state) - { - auto test_index = dist(rng); - const auto & query_vector = DatasetMnist::get().dataTestAt(test_index); - auto keys = VectorIndexBenchUtils::queryTopK(viewer, query_vector, 100, state); - RUNTIME_CHECK(keys.size() == 100); - } -} -CATCH - -BENCHMARK(VectorIndexBuild); - -BENCHMARK(VectorIndexSearchTop10); - -BENCHMARK(VectorIndexSearchTop100); - -} // namespace DB::DM::bench diff --git a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h b/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h deleted file mode 100644 index 27e6c3cfa7e..00000000000 --- a/dbms/src/Storages/DeltaMerge/tests/bench_vector_index_utils.h +++ /dev/null @@ -1,177 +0,0 @@ -// Copyright 2024 PingCAP, Inc. -// -// 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 -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace DB::DM::bench -{ - -/** - * @brief Compatible with datasets on ANN-Benchmark: - * https://github.com/erikbern/ann-benchmarks - */ -class Dataset -{ -public: - explicit Dataset(std::string_view file_name) - { - auto dataset_directory = std::filesystem::path(__FILE__).parent_path().string() + "/bench_dataset"; - auto dataset_path = fmt::format("{}/{}", dataset_directory, file_name); - - if (!std::filesystem::exists(dataset_path)) - { - throw Exception(fmt::format( - "Benchmark cannot run because dataset file {} not found. See {}/README.md for setup instructions.", - dataset_path, - dataset_directory)); - } - - auto file = HighFive::File(dataset_path, HighFive::File::ReadOnly); - - auto dataset_train = file.getDataSet("train"); - dataset_train.read(data_train); - - auto dataset_test = file.getDataSet("test"); - dataset_test.read(data_test); - } - - virtual ~Dataset() = default; - - virtual UInt32 dimension() const = 0; - - virtual tipb::VectorDistanceMetric distanceMetric() const = 0; - -public: - MutableColumnPtr buildDataTrainColumn(std::optional max_rows = std::nullopt) const - { - auto vec_column = ColumnArray::create(ColumnFloat32::create()); - size_t rows = data_train.size(); - if (max_rows.has_value()) - rows = std::min(rows, *max_rows); - for (size_t i = 0; i < rows; ++i) - { - const auto & row = data_train[i]; - vec_column->insertData(reinterpret_cast(row.data()), row.size() * sizeof(Float32)); - } - return vec_column; - } - - size_t dataTestSize() const { return data_test.size(); } - - const std::vector & dataTestAt(size_t index) const { return data_test.at(index); } - - TiDB::VectorIndexDefinitionPtr createIndexDef(tipb::VectorIndexKind kind) const - { - return std::make_shared(TiDB::VectorIndexDefinition{ - .kind = kind, - .dimension = dimension(), - .distance_metric = distanceMetric(), - }); - } - -protected: - std::vector> data_train; - std::vector> data_test; -}; - -class DatasetMnist : public Dataset -{ -public: - DatasetMnist() - : Dataset("fashion-mnist-784-euclidean.hdf5") - { - RUNTIME_CHECK(data_train[0].size() == dimension()); - RUNTIME_CHECK(data_test[0].size() == dimension()); - } - - UInt32 dimension() const override { return 784; } - - tipb::VectorDistanceMetric distanceMetric() const override { return tipb::VectorDistanceMetric::L2; } - - static const DatasetMnist & get() - { - static DatasetMnist dataset; - return dataset; - } -}; - -class VectorIndexBenchUtils -{ -public: - template - static void saveVectorIndex( - std::string_view index_path, - const Dataset & dataset, - std::optional max_rows = std::nullopt) - { - Poco::File(index_path.data()).createDirectories(); - - auto train_data = dataset.buildDataTrainColumn(max_rows); - auto index_def = dataset.createIndexDef(Builder::kind()); - auto builder = std::make_unique(0, index_def); - builder->addBlock(*train_data, nullptr, []() { return true; }); - builder->saveToFile(index_path); - } - - template - static auto viewVectorIndex(std::string_view index_path, const Dataset & dataset) - { - auto index_view_props = dtpb::VectorIndexFileProps(); - index_view_props.set_index_kind(tipb::VectorIndexKind_Name(Viewer::kind())); - index_view_props.set_dimensions(dataset.dimension()); - index_view_props.set_distance_metric(tipb::VectorDistanceMetric_Name(dataset.distanceMetric())); - return Viewer::view(index_view_props, index_path); - } - - static auto queryTopK( - VectorIndexViewerPtr viewer, - const std::vector & ref, - UInt32 top_k, - std::optional> state = std::nullopt) - { - if (state.has_value()) - state->get().PauseTiming(); - - auto ann_query_info = std::make_shared(); - auto distance_metric = tipb::VectorDistanceMetric::INVALID_DISTANCE_METRIC; - tipb::VectorDistanceMetric_Parse(viewer->file_props.distance_metric(), &distance_metric); - ann_query_info->set_distance_metric(distance_metric); - ann_query_info->set_top_k(top_k); - ann_query_info->set_ref_vec_f32(DB::DM::tests::VectorIndexTestUtils::encodeVectorFloat32(ref)); - - auto filter = BitmapFilterView::createWithFilter(viewer->size(), true); - - if (state.has_value()) - state->get().ResumeTiming(); - - return viewer->search(ann_query_info, filter); - } -}; - - -} // namespace DB::DM::bench From 3dc5d1a3122d6ce0385071d4f3c79f763d83f8ce Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 22 Jan 2025 13:54:36 +0800 Subject: [PATCH 40/48] ci --- .../VersionChain/tests/gtest_mvcc_bitmap.cpp | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp index 206452f50d6..0bb2bef4a80 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/gtest_mvcc_bitmap.cpp @@ -25,12 +25,12 @@ using namespace DB::DM::tests::MVCC; namespace DB::DM::tests { template -void randomMVCCBitmapVerify(UInt32 delta_rows) +void randomMVCCBitmapVerify(WriteLoad write_load, UInt32 delta_rows) try { constexpr bool is_common_handle = std::is_same_v; auto [context, dm_context, cols, segment, segment_snapshot, random_sequences] - = initialize(is_common_handle, delta_rows); + = initialize(write_load, is_common_handle, delta_rows); SCOPE_EXIT({ context->shutdown(); }); ASSERT_EQ(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first, 0); @@ -39,7 +39,6 @@ try segment_snapshot->delta->getSharedDeltaIndex()->updateIfAdvanced(*delta_index); ASSERT_EQ(segment_snapshot->delta->getSharedDeltaIndex()->getPlacedStatus().first, delta_rows); - VersionChain version_chain; buildVersionChain(*dm_context, *segment_snapshot, version_chain); ASSERT_EQ(version_chain.getReplayedRows(), delta_rows); @@ -66,8 +65,13 @@ try RUNTIME_ASSERT(filter1.size() == filter2.size()); for (UInt32 i = 0; i < filter1.size(); ++i) { - ASSERT_EQ(filter1[i], filter2[i]) - << fmt::format("i={}, filter1={}, filter2={}, delta_rows={}", i, filter1[i], filter2[i], delta_rows); + ASSERT_EQ(filter1[i], filter2[i]) << fmt::format( + "i={}, filter1={}, filter2={}, write_load={}, delta_rows={}", + i, + filter1[i], + filter2[i], + magic_enum::enum_name(write_load), + delta_rows); } } CATCH @@ -76,13 +80,15 @@ static constexpr UInt32 max_delta_rows = 8 << 13; TEST(TestVersionChain, randomMVCCBitmapVerify) { - for (UInt32 delta_rows = 1; delta_rows <= max_delta_rows; delta_rows *= 8) - randomMVCCBitmapVerify(delta_rows); + for (auto write_load : magic_enum::enum_values()) + for (UInt32 delta_rows = 1; delta_rows <= max_delta_rows; delta_rows *= 8) + randomMVCCBitmapVerify(write_load, delta_rows); } TEST(TestVersionChain, randomMVCCBitmapVerify_CommonHandle) { - for (UInt32 delta_rows = 1; delta_rows <= max_delta_rows; delta_rows *= 8) - randomMVCCBitmapVerify(delta_rows); + for (auto write_load : magic_enum::enum_values()) + for (UInt32 delta_rows = 1; delta_rows <= max_delta_rows; delta_rows *= 8) + randomMVCCBitmapVerify(write_load, delta_rows); } } // namespace DB::DM::tests From 2c1e34420630fb0051ae69c2b6d0c07d178a7a96 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 22 Jan 2025 15:34:48 +0800 Subject: [PATCH 41/48] ci --- dbms/src/Storages/DeltaMerge/Segment.cpp | 1 + dbms/src/Storages/DeltaMerge/Segment.h | 2 +- .../VersionChain/BuildBitmapFilter.cpp | 19 +++++++++++++++++++ .../VersionChain/BuildBitmapFilter.h | 8 ++++++++ .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 4 ++-- .../DeltaMerge/VersionChain/VersionChain.cpp | 4 ++-- .../DeltaMerge/VersionChain/VersionChain.h | 8 ++++++++ .../DeltaMerge/VersionChain/VersionFilter.cpp | 4 ++-- 8 files changed, 43 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 824edcd568c..6d250527b7a 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -296,6 +296,7 @@ Segment::Segment( // , stable(stable_) , parent_log(parent_log_) , log(parent_log_->getChild(fmt::format("segment_id={} epoch={}", segment_id, epoch))) + , version_chain(createVersionChain(is_common_handle)) { if (delta != nullptr) delta->resetLogger(log); diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index 1e666f89d52..b430d2e6a94 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -843,7 +843,7 @@ class Segment const LoggerPtr parent_log; // Used when constructing new segments in split const LoggerPtr log; - VersionChain version_chain; // TODO: support common handle + std::variant, VersionChain> version_chain; }; void readSegmentMetaInfo(ReadBuffer & buf, Segment::SegmentMetaInfo & segment_info); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 2b63f1faa9a..92d7f8ce645 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -66,4 +66,23 @@ template BitmapFilterPtr buildBitmapFilter( const DMFilePackFilterResults & pack_filter_results, const UInt64 read_ts, VersionChain & version_chain); + +BitmapFilterPtr buildBitmapFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const DMFilePackFilterResults & pack_filter_results, + const UInt64 read_ts, + std::variant, VersionChain> & variant_version_chain) +{ + return std::visit([&](auto & version_chain) { + using T = std::decay_t; + if constexpr (std::is_same_v>) + return buildBitmapFilter(dm_context, snapshot, read_ranges, pack_filter_results, read_ts, version_chain); + else if constexpr (std::is_same_v>) + return buildBitmapFilter(dm_context, snapshot, read_ranges, pack_filter_results, read_ts, version_chain); + else + static_assert(false, "Only VersionChain and VersionChain is supported"); + }, variant_version_chain); +} } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h index 231ccb6a964..0c9a0d67323 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.h @@ -36,4 +36,12 @@ BitmapFilterPtr buildBitmapFilter( const UInt64 read_ts, VersionChain & version_chain); + +BitmapFilterPtr buildBitmapFilter( + const DMContext & dm_context, + const SegmentSnapshot & snapshot, + const RowKeyRanges & read_ranges, + const DMFilePackFilterResults & pack_filter_results, + const UInt64 read_ts, + std::variant, VersionChain> & variant_version_chain); } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index a1ed43cb896..c5fc7e642b1 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -230,9 +231,8 @@ void buildRowKeyFilter( RowKeyRanges delete_ranges; UInt32 read_rows = 0; // Read ColumnFiles from new to old for handling delete ranges - for (auto itr = cfs.rbegin(); itr != cfs.rend(); ++itr) + for (const auto & cf : cfs | std::views::reverse) { - const auto & cf = *itr; if (const auto * cf_delete_range = cf->tryToDeleteRange(); cf_delete_range) { delete_ranges.push_back(cf_delete_range->getDeleteRange()); diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index 3021ea27476..c0ed19c05b4 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -199,9 +200,8 @@ std::optional VersionChain::findBaseVersionFromDMFileOrDelete HandleRef h) { // From from new to old - for (auto itr = dmfile_or_delete_range_list.rbegin(); itr != dmfile_or_delete_range_list.rend(); ++itr) + for (auto & dmfile_or_delete_range : dmfile_or_delete_range_list | std::views::reverse) { - auto & dmfile_or_delete_range = *itr; if (auto * dmfile_index = std::get_if>(&dmfile_or_delete_range); dmfile_index) { if (auto row_id = dmfile_index->getBaseVersion(dm_context, h); row_id) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index d8e7a501bca..ccb75754072 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -86,4 +86,12 @@ class VersionChain using DMFileOrDeleteRange = std::variant>; std::vector dmfile_or_delete_range_list; }; + +inline std::variant, VersionChain> createVersionChain(bool is_common_handle) +{ + if (is_common_handle) + return std::variant, VersionChain>{std::in_place_type>}; + else + return std::variant, VersionChain>{std::in_place_type>}; +} } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp index 16cc457ed07..a9ff5ace5c7 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -235,9 +236,8 @@ void buildVersionFilter( // Delta MVCC UInt32 read_rows = 0; // Read versions from new to old - for (auto itr = cfs.rbegin(); itr != cfs.rend(); ++itr) + for (const auto & cf : cfs | std::views::reverse) { - const auto & cf = *itr; if (cf->isDeleteRange()) // Delete range is handled by RowKeyFilter. continue; From 6c5c3807a70ad156f55e42f250734c28a7d7be89 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 22 Jan 2025 15:36:15 +0800 Subject: [PATCH 42/48] ci --- .../VersionChain/BuildBitmapFilter.cpp | 32 +++++++++++++------ .../DeltaMerge/VersionChain/RowKeyFilter.cpp | 3 +- .../DeltaMerge/VersionChain/VersionChain.cpp | 3 +- .../DeltaMerge/VersionChain/VersionFilter.cpp | 3 +- 4 files changed, 29 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 92d7f8ce645..905f910e344 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -75,14 +75,28 @@ BitmapFilterPtr buildBitmapFilter( const UInt64 read_ts, std::variant, VersionChain> & variant_version_chain) { - return std::visit([&](auto & version_chain) { - using T = std::decay_t; - if constexpr (std::is_same_v>) - return buildBitmapFilter(dm_context, snapshot, read_ranges, pack_filter_results, read_ts, version_chain); - else if constexpr (std::is_same_v>) - return buildBitmapFilter(dm_context, snapshot, read_ranges, pack_filter_results, read_ts, version_chain); - else - static_assert(false, "Only VersionChain and VersionChain is supported"); - }, variant_version_chain); + return std::visit( + [&](auto & version_chain) { + using T = std::decay_t; + if constexpr (std::is_same_v>) + return buildBitmapFilter( + dm_context, + snapshot, + read_ranges, + pack_filter_results, + read_ts, + version_chain); + else if constexpr (std::is_same_v>) + return buildBitmapFilter( + dm_context, + snapshot, + read_ranges, + pack_filter_results, + read_ts, + version_chain); + else + static_assert(false, "Only VersionChain and VersionChain is supported"); + }, + variant_version_chain); } } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp index c5fc7e642b1..2e7ee5db7ba 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/RowKeyFilter.cpp @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include #include @@ -21,6 +20,8 @@ #include #include +#include + namespace DB::DM { namespace diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index c0ed19c05b4..8f838bb8eb1 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -12,12 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include #include #include +#include + namespace DB::DM { diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp index a9ff5ace5c7..957f23882d4 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionFilter.cpp @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include #include @@ -21,6 +20,8 @@ #include #include +#include + namespace DB::DM { [[nodiscard]] UInt32 buildVersionFilterBlock( From 80fd24077469f9a567d53944747df319059daab3 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 22 Jan 2025 15:40:11 +0800 Subject: [PATCH 43/48] ci --- .../VersionChain/BuildBitmapFilter.cpp | 20 +------------------ 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 905f910e344..2ea5e95da2b 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -77,25 +77,7 @@ BitmapFilterPtr buildBitmapFilter( { return std::visit( [&](auto & version_chain) { - using T = std::decay_t; - if constexpr (std::is_same_v>) - return buildBitmapFilter( - dm_context, - snapshot, - read_ranges, - pack_filter_results, - read_ts, - version_chain); - else if constexpr (std::is_same_v>) - return buildBitmapFilter( - dm_context, - snapshot, - read_ranges, - pack_filter_results, - read_ts, - version_chain); - else - static_assert(false, "Only VersionChain and VersionChain is supported"); + return buildBitmapFilter(dm_context, snapshot, read_ranges, pack_filter_results, read_ts, version_chain); }, variant_version_chain); } From 0eceafbdc2980955db4e5631b497dba762c3f2f2 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 22 Jan 2025 15:50:00 +0800 Subject: [PATCH 44/48] ci --- .../VersionChain/tests/bench_search.cpp | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp index e2e28463a13..143d8d3ef20 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp @@ -41,17 +41,13 @@ void searchInt64(benchmark::State & state, Args &&... args) std::random_device rd; std::mt19937 g(rd()); - auto gen_always_found = [&]() { + std::function gen_always_found = [&]() { return g() % size; }; - auto gen_always_not_found = [&]() { + std::function gen_always_not_found = [&]() { return gen_always_found() + size; }; - std::function gen; - if (result == SearchResult::AlwaysFound) - gen = gen_always_found; - else - gen = gen_always_not_found; + auto gen = result == SearchResult::AlwaysFound ? gen_always_found : gen_always_not_found; if (type == SearchType::BinarySearch) { @@ -106,17 +102,13 @@ void searchStruct(benchmark::State & state, Args &&... args) std::random_device rd; std::mt19937 g(rd()); - auto gen_always_found = [&]() { + std::function gen_always_found = [&]() { return g() % size; }; - auto gen_always_not_found = [&]() { + std::function gen_always_not_found = [&]() { return gen_always_found() + size; }; - std::function gen; - if (result == SearchResult::AlwaysFound) - gen = gen_always_found; - else - gen = gen_always_not_found; + auto gen = result == SearchResult::AlwaysFound ? gen_always_found : gen_always_not_found; if (type == SearchType::BinarySearch) { From 621f4b0acd049b819d64554857f180d40d705f5c Mon Sep 17 00:00:00 2001 From: jinhelin Date: Wed, 22 Jan 2025 17:14:31 +0800 Subject: [PATCH 45/48] ci --- .../src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp index 143d8d3ef20..489bc00ed6e 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/tests/bench_search.cpp @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. - #include +#include //#include #include #include From a2fa2c79069abc4df8ab9cc55be0723a0eeafb52 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 23 Jan 2025 10:47:33 +0800 Subject: [PATCH 46/48] ci --- dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h | 1 + .../Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h index d3e1f3bb531..d88b0183b8d 100644 --- a/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h +++ b/dbms/src/Storages/DeltaMerge/BitmapFilter/BitmapFilter.h @@ -40,6 +40,7 @@ class BitmapFilter void rangeAnd(IColumn::Filter & f, UInt32 start, UInt32 limit) const; void runOptimize(); + void setAllMatch(bool all_match_) { all_match = all_match_; } String toDebugString() const; size_t count() const; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp index 2ea5e95da2b..b7bd52e7439 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/BuildBitmapFilter.cpp @@ -46,8 +46,8 @@ BitmapFilterPtr buildBitmapFilter( buildRowKeyFilter(dm_context, snapshot, read_ranges, pack_filter_results[0], filter); buildVersionFilter(dm_context, snapshot, *base_ver_snap, read_ts, filter); buildDeletedFilter(dm_context, snapshot, filter); - - bitmap_filter->runOptimize(); + // TODO: Make buildRowKeyFilter/buildVersionFilter/buildDeletedFilter returns filtered rows. + bitmap_filter->setAllMatch(false); return bitmap_filter; } From 66cefb0c85fa892ce597f644f68c7bdbba5cdb64 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Thu, 23 Jan 2025 17:09:02 +0800 Subject: [PATCH 47/48] ci --- .../Storages/DeltaMerge/VersionChain/VersionChain.cpp | 6 +++--- dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h | 9 ++++++++- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp index 8f838bb8eb1..b97fb11665c 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.cpp @@ -148,9 +148,9 @@ UInt32 VersionChain::replayBlock( { const auto h = *itr; const RowID curr_row_id = base_versions->size() + stable_rows; - if (auto itr = new_handle_to_row_ids.find(h); itr != new_handle_to_row_ids.end()) + if (auto t = new_handle_to_row_ids.find(h); t != new_handle_to_row_ids.end()) { - base_versions->push_back(itr->second); + base_versions->push_back(t->second); continue; } if (auto row_id = findBaseVersionFromDMFileOrDeleteRangeList(dm_context, h); row_id) @@ -159,7 +159,7 @@ UInt32 VersionChain::replayBlock( continue; } - new_handle_to_row_ids.insert(std::make_pair(h, curr_row_id)); + new_handle_to_row_ids[h] = curr_row_id; base_versions->push_back(NotExistRowID); } return column.size() - offset; diff --git a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h index ccb75754072..210e1afc88e 100644 --- a/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h +++ b/dbms/src/Storages/DeltaMerge/VersionChain/VersionChain.h @@ -20,6 +20,13 @@ #include + +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wdeprecated-builtins" + +#include +#include + namespace DB::DM { @@ -82,7 +89,7 @@ class VersionChain std::mutex mtx; UInt32 replayed_rows_and_deletes = 0; // delta.getRows() + delta.getDeletes() std::shared_ptr> base_versions; // base_versions->size() == delta.getRows() - std::map> new_handle_to_row_ids; + absl::btree_map new_handle_to_row_ids; using DMFileOrDeleteRange = std::variant>; std::vector dmfile_or_delete_range_list; }; From bf1e6c0a6592fae8bd8e73e0618226c2d19a6c09 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Sun, 26 Jan 2025 10:03:40 +0800 Subject: [PATCH 48/48] ci --- .../KVStore/tests/bench_parse_lock.cpp | 68 ------------------- 1 file changed, 68 deletions(-) delete mode 100644 dbms/src/Storages/KVStore/tests/bench_parse_lock.cpp diff --git a/dbms/src/Storages/KVStore/tests/bench_parse_lock.cpp b/dbms/src/Storages/KVStore/tests/bench_parse_lock.cpp deleted file mode 100644 index 69032ad1c22..00000000000 --- a/dbms/src/Storages/KVStore/tests/bench_parse_lock.cpp +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2024 PingCAP, Inc. -// -// 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 -#include -#include - -#include - -using namespace DB; - -namespace DB::tests -{ -using DB::RecordKVFormat::DecodedLockCFValue; - -DecodedLockCFValue::Inner * decodeLockCfValue(const DecodedLockCFValue & decoded); - -void parseTest(benchmark::State & state) -{ - try - { - std::string shor_value = "value"; - auto lock_for_update_ts = 7777, txn_size = 1; - const std::vector & async_commit = {"s1", "s2"}; - const std::vector & rollback = {3, 4}; - auto lock_value2 = encodeFullLockCfValue( - Region::DelFlag, - "primary key", - 421321, - std::numeric_limits::max(), - &shor_value, - 66666, - lock_for_update_ts, - txn_size, - async_commit, - rollback, - 1111); - - auto ori_key = std::make_shared(RecordKVFormat::genKey(1, 88888)); - for (auto _ : state) - { - auto lock2 = RecordKVFormat::DecodedLockCFValue( - ori_key, - std::make_shared(TiKVValue::copyFrom(lock_value2))); - benchmark::DoNotOptimize(lock2); - } - } - catch (...) - { - tryLogCurrentException(DB::Logger::get(), __PRETTY_FUNCTION__); - } -} - -BENCHMARK(parseTest); - -} // namespace DB::tests