From 60a78ae97588cefd9b6acc61bb0efd412b9aaf0c Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Thu, 19 Mar 2026 06:27:06 +0000 Subject: [PATCH 01/10] feat: support LookupMergeTreeCompactRewriter --- include/paimon/defs.h | 4 + src/paimon/CMakeLists.txt | 4 + src/paimon/common/data/generic_row.h | 6 +- src/paimon/common/defs.cpp | 1 + src/paimon/common/io/cache/cache_manager.cpp | 6 +- src/paimon/common/sst/sst_file_reader.cpp | 12 + src/paimon/core/append/append_only_writer.cpp | 2 +- src/paimon/core/core_options.cpp | 46 +- src/paimon/core/core_options.h | 4 +- src/paimon/core/core_options_test.cpp | 28 +- .../global_index/global_index_scan_impl.cpp | 7 +- .../global_index/global_index_write_task.cpp | 13 +- .../core/io/single_file_writer_test.cpp | 4 +- src/paimon/core/key_value.h | 20 +- .../manifest/manifest_entry_writer_test.cpp | 2 +- .../compact/changelog_merge_tree_rewriter.h | 175 +++ .../core/mergetree/compact/compact_rewriter.h | 2 +- .../compact/first_row_merge_function.h | 4 + .../first_row_merge_function_wrapper.h | 68 ++ .../first_row_merge_function_wrapper_test.cpp | 107 ++ .../lookup_changelog_merge_function_wrapper.h | 151 +++ ..._changelog_merge_function_wrapper_test.cpp | 198 ++++ .../mergetree/compact/lookup_merge_function.h | 79 +- .../compact/lookup_merge_function_test.cpp | 68 ++ .../lookup_merge_tree_compact_rewriter.h | 185 +++ ...ookup_merge_tree_compact_rewriter_test.cpp | 1017 +++++++++++++++++ .../compact/merge_tree_compact_rewriter.cpp | 95 +- .../compact/merge_tree_compact_rewriter.h | 50 +- .../merge_tree_compact_rewriter_test.cpp | 17 +- .../compact/reducer_merge_function_wrapper.h | 9 +- src/paimon/core/mergetree/lookup_levels.h | 6 + .../core/mergetree/lookup_levels_test.cpp | 14 +- .../core/mergetree/merge_tree_writer.cpp | 2 +- src/paimon/core/migrate/file_meta_utils.cpp | 2 +- .../append_only_file_store_write.cpp | 2 +- .../core/operation/expire_snapshots_test.cpp | 2 +- .../core/operation/file_store_commit.cpp | 13 +- .../core/operation/file_store_write.cpp | 11 +- .../key_value_file_store_scan_test.cpp | 13 +- .../operation/manifest_file_merger_test.cpp | 2 +- .../core/operation/merge_file_split_read.cpp | 5 + .../core/operation/merge_file_split_read.h | 8 +- .../operation/merge_file_split_read_test.cpp | 7 +- .../core/operation/orphan_files_cleaner.cpp | 11 +- .../operation/raw_file_split_read_test.cpp | 14 +- src/paimon/core/options/lookup_strategy.h | 42 + .../postpone_bucket_file_store_write.h | 13 +- .../core/postpone/postpone_bucket_writer.cpp | 2 +- src/paimon/core/table/source/table_read.cpp | 3 +- src/paimon/core/table/source/table_scan.cpp | 3 +- .../utils/file_store_path_factory_cache.h | 67 ++ .../file_store_path_factory_cache_test.cpp | 53 + .../utils/file_store_path_factory_test.cpp | 2 +- 53 files changed, 2513 insertions(+), 168 deletions(-) create mode 100644 src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h create mode 100644 src/paimon/core/mergetree/compact/first_row_merge_function_wrapper.h create mode 100644 src/paimon/core/mergetree/compact/first_row_merge_function_wrapper_test.cpp create mode 100644 src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h create mode 100644 src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp create mode 100644 src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h create mode 100644 src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp create mode 100644 src/paimon/core/options/lookup_strategy.h create mode 100644 src/paimon/core/utils/file_store_path_factory_cache.h create mode 100644 src/paimon/core/utils/file_store_path_factory_cache_test.cpp diff --git a/include/paimon/defs.h b/include/paimon/defs.h index f21e5303..4c5158d2 100644 --- a/include/paimon/defs.h +++ b/include/paimon/defs.h @@ -354,6 +354,10 @@ struct PAIMON_EXPORT Options { static const char SPILL_COMPRESSION_ZSTD_LEVEL[]; /// "cache-page-size" - Memory page size for caching. Default value is 64 kb. static const char CACHE_PAGE_SIZE[]; + /// "file.format.per.level" - Define different file format for different level, you can add the + /// conf like this: 'file.format.per.level' = '0:avro,3:parquet', if the file format for level + /// is not provided, the default format which set by FILE_FORMAT will be used. + static const char FILE_FORMAT_PER_LEVEL[]; }; static constexpr int64_t BATCH_WRITE_COMMIT_IDENTIFIER = std::numeric_limits::max(); diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index 55eb91e7..c2bfe0fb 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -560,7 +560,10 @@ if(PAIMON_BUILD_TESTS) core/mergetree/compact/aggregate/field_sum_agg_test.cpp core/mergetree/compact/deduplicate_merge_function_test.cpp core/mergetree/compact/first_row_merge_function_test.cpp + core/mergetree/compact/first_row_merge_function_wrapper_test.cpp core/mergetree/compact/interval_partition_test.cpp + core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp + core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp core/mergetree/compact/lookup_merge_function_test.cpp core/mergetree/compact/partial_update_merge_function_test.cpp core/mergetree/compact/reducer_merge_function_wrapper_test.cpp @@ -621,6 +624,7 @@ if(PAIMON_BUILD_TESTS) core/table/source/table_scan_test.cpp core/tag/tag_test.cpp core/utils/branch_manager_test.cpp + core/utils/file_store_path_factory_cache_test.cpp core/utils/field_mapping_test.cpp core/utils/fields_comparator_test.cpp core/utils/file_store_path_factory_test.cpp diff --git a/src/paimon/common/data/generic_row.h b/src/paimon/common/data/generic_row.h index 30409005..6c9070a7 100644 --- a/src/paimon/common/data/generic_row.h +++ b/src/paimon/common/data/generic_row.h @@ -103,8 +103,8 @@ class GenericRow : public InternalRow { return DataDefine::IsVariantNull(fields_[pos]); } - void AddDataHolder(std::unique_ptr&& holder) { - holders_.push_back(std::move(holder)); + void AddDataHolder(const std::shared_ptr& holder) { + holders_.push_back(holder); } bool GetBoolean(int32_t pos) const override { @@ -219,7 +219,7 @@ class GenericRow : public InternalRow { std::vector fields_; /// As GenericRow only holds string view for string data to avoid deep copy, original data must /// be held in holders_ - std::vector> holders_; + std::vector> holders_; /// The kind of change that a row describes in a changelog. const RowKind* kind_; }; diff --git a/src/paimon/common/defs.cpp b/src/paimon/common/defs.cpp index b80452e3..349f4822 100644 --- a/src/paimon/common/defs.cpp +++ b/src/paimon/common/defs.cpp @@ -101,4 +101,5 @@ const char Options::LOOKUP_CACHE_BLOOM_FILTER_FPP[] = "lookup.cache.bloom.filter const char Options::LOOKUP_CACHE_SPILL_COMPRESSION[] = "lookup.cache-spill-compression"; const char Options::SPILL_COMPRESSION_ZSTD_LEVEL[] = "spill-compression.zstd-level"; const char Options::CACHE_PAGE_SIZE[] = "cache-page-size"; +const char Options::FILE_FORMAT_PER_LEVEL[] = "file.format.per.level"; } // namespace paimon diff --git a/src/paimon/common/io/cache/cache_manager.cpp b/src/paimon/common/io/cache/cache_manager.cpp index 0e3e3ea4..01ccb60b 100644 --- a/src/paimon/common/io/cache/cache_manager.cpp +++ b/src/paimon/common/io/cache/cache_manager.cpp @@ -31,7 +31,11 @@ std::shared_ptr CacheManager::GetPage( auto ptr = std::make_shared(segment); return std::make_shared(ptr); }; - return cache->Get(key, supplier)->GetSegment(); + auto cache_value = cache->Get(key, supplier); + if (!cache_value) { + return nullptr; + } + return cache_value->GetSegment(); } void CacheManager::InvalidPage(const std::shared_ptr& key) { diff --git a/src/paimon/common/sst/sst_file_reader.cpp b/src/paimon/common/sst/sst_file_reader.cpp index f313b224..9d4c0db2 100644 --- a/src/paimon/common/sst/sst_file_reader.cpp +++ b/src/paimon/common/sst/sst_file_reader.cpp @@ -56,10 +56,16 @@ Result> SstFileReader::Create( auto trailer_data = block_cache->GetBlock(index_block_handle->Offset() + index_block_handle->Size(), BlockTrailer::ENCODED_LENGTH, true); + if (!trailer_data) { + return Status::Invalid("Read trailer error"); + } auto trailer_input = MemorySlice::Wrap(trailer_data)->ToInput(); auto trailer = BlockTrailer::ReadBlockTrailer(trailer_input); auto block_data = block_cache->GetBlock(index_block_handle->Offset(), index_block_handle->Size(), true); + if (!block_data) { + return Status::Invalid("Read block error"); + } PAIMON_ASSIGN_OR_RAISE(std::shared_ptr uncompressed_data, DecompressBlock(block_data, trailer, pool)); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr reader, @@ -124,9 +130,15 @@ Result> SstFileReader::ReadBlock( const std::shared_ptr& handle, bool index) { auto trailer_data = block_cache_->GetBlock(handle->Offset() + handle->Size(), BlockTrailer::ENCODED_LENGTH, true); + if (!trailer_data) { + return Status::Invalid("read block failed"); + } auto trailer_input = MemorySlice::Wrap(trailer_data)->ToInput(); auto trailer = BlockTrailer::ReadBlockTrailer(trailer_input); auto block_data = block_cache_->GetBlock(handle->Offset(), handle->Size(), index); + if (!block_data) { + return Status::Invalid("read block failed"); + } PAIMON_ASSIGN_OR_RAISE(std::shared_ptr uncompressed_data, DecompressBlock(block_data, trailer, pool_)); return BlockReader::Create(MemorySlice::Wrap(uncompressed_data), comparator_); diff --git a/src/paimon/core/append/append_only_writer.cpp b/src/paimon/core/append/append_only_writer.cpp index 97fe575a..32f5a12f 100644 --- a/src/paimon/core/append/append_only_writer.cpp +++ b/src/paimon/core/append/append_only_writer.cpp @@ -173,7 +173,7 @@ AppendOnlyWriter::SingleFileWriterCreator AppendOnlyWriter::GetDataFileWriterCre ::ArrowSchema arrow_schema; ScopeGuard guard([&arrow_schema]() { ArrowSchemaRelease(&arrow_schema); }); PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*schema, &arrow_schema)); - auto format = options_.GetWriteFileFormat(); + auto format = options_.GetWriteFileFormat(/*level=*/0); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr writer_builder, format->CreateWriterBuilder(&arrow_schema, options_.GetWriteBatchSize())); diff --git a/src/paimon/core/core_options.cpp b/src/paimon/core/core_options.cpp index 5f9c26c1..8d43bccb 100644 --- a/src/paimon/core/core_options.cpp +++ b/src/paimon/core/core_options.cpp @@ -123,7 +123,8 @@ class ConfigParser { std::string normalized_value = StringUtils::ToLowerCase(iter->second); PAIMON_ASSIGN_OR_RAISE(*value, Factory::Get(normalized_value, config_map_)); } else { - PAIMON_ASSIGN_OR_RAISE(*value, Factory::Get(default_identifier, config_map_)); + PAIMON_ASSIGN_OR_RAISE( + *value, Factory::Get(StringUtils::ToLowerCase(default_identifier), config_map_)); } return Status::OK(); } @@ -336,6 +337,7 @@ struct CoreOptions::Impl { double lookup_cache_bloom_filter_fpp = 0.05; CompressOptions lookup_compress_options{"zstd", 1}; int64_t cache_page_size = 64 * 1024; // 64KB + std::map> file_format_per_level; }; // Parse configurations from a map and return a populated CoreOptions object @@ -596,6 +598,32 @@ Result CoreOptions::FromMap( // Parse cache-page-size PAIMON_RETURN_NOT_OK(parser.ParseMemorySize(Options::CACHE_PAGE_SIZE, &impl->cache_page_size)); + // parse file.format.per.level + std::string file_format_per_level_str; + PAIMON_RETURN_NOT_OK( + parser.ParseString(Options::FILE_FORMAT_PER_LEVEL, &file_format_per_level_str)); + if (!file_format_per_level_str.empty()) { + auto level2format = + StringUtils::Split(file_format_per_level_str, std::string(","), std::string(":")); + for (const auto& single_level : level2format) { + if (single_level.size() != 2) { + return Status::Invalid( + fmt::format("fail to parse key {}, value {} (usage example: 0:avro,3:parquet)", + Options::FILE_FORMAT_PER_LEVEL, file_format_per_level_str)); + } + auto level = StringUtils::StringToValue(single_level[0]); + if (!level || level.value() < 0) { + return Status::Invalid( + fmt::format("fail to parse level {} from string to int in {}", single_level[0], + Options::FILE_FORMAT_PER_LEVEL)); + } + std::shared_ptr file_format; + PAIMON_RETURN_NOT_OK(parser.ParseObject( + "_no_use", /*default_identifier=*/single_level[1], &file_format)); + impl->file_format_per_level[level.value()] = file_format; + } + } + return options; } @@ -617,7 +645,11 @@ int32_t CoreOptions::GetBucket() const { return impl_->bucket; } -std::shared_ptr CoreOptions::GetWriteFileFormat() const { +std::shared_ptr CoreOptions::GetWriteFileFormat(int32_t level) const { + auto iter = impl_->file_format_per_level.find(level); + if (iter != impl_->file_format_per_level.end()) { + return iter->second; + } return impl_->file_format; } @@ -807,9 +839,13 @@ const std::map& CoreOptions::ToMap() const { } bool CoreOptions::NeedLookup() const { - return GetMergeEngine() == MergeEngine::FIRST_ROW || - GetChangelogProducer() == ChangelogProducer::LOOKUP || DeletionVectorsEnabled() || - impl_->force_lookup; + return GetLookupStrategy().need_lookup; +} + +LookupStrategy CoreOptions::GetLookupStrategy() const { + return {GetMergeEngine() == MergeEngine::FIRST_ROW, + GetChangelogProducer() == ChangelogProducer::LOOKUP, DeletionVectorsEnabled(), + impl_->force_lookup}; } bool CoreOptions::CompactionForceRewriteAllFiles() const { diff --git a/src/paimon/core/core_options.h b/src/paimon/core/core_options.h index 44796334..59e28390 100644 --- a/src/paimon/core/core_options.h +++ b/src/paimon/core/core_options.h @@ -27,6 +27,7 @@ #include "paimon/core/options/changelog_producer.h" #include "paimon/core/options/compress_options.h" #include "paimon/core/options/external_path_strategy.h" +#include "paimon/core/options/lookup_strategy.h" #include "paimon/core/options/merge_engine.h" #include "paimon/core/options/sort_engine.h" #include "paimon/format/file_format.h" @@ -53,7 +54,7 @@ class PAIMON_EXPORT CoreOptions { ~CoreOptions(); int32_t GetBucket() const; - std::shared_ptr GetWriteFileFormat() const; + std::shared_ptr GetWriteFileFormat(int32_t level) const; std::shared_ptr GetFileSystem() const; const std::string& GetFileCompression() const; int32_t GetFileCompressionZstdLevel() const; @@ -101,6 +102,7 @@ class PAIMON_EXPORT CoreOptions { int64_t DeletionVectorTargetFileSize() const; ChangelogProducer GetChangelogProducer() const; bool NeedLookup() const; + LookupStrategy GetLookupStrategy() const; bool FileIndexReadEnabled() const; std::map GetFieldsSequenceGroups() const; diff --git a/src/paimon/core/core_options_test.cpp b/src/paimon/core/core_options_test.cpp index 01ec10d5..a90e82a6 100644 --- a/src/paimon/core/core_options_test.cpp +++ b/src/paimon/core/core_options_test.cpp @@ -33,7 +33,8 @@ namespace paimon::test { TEST(CoreOptionsTest, TestDefaultValue) { ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap({})); ASSERT_EQ(core_options.GetManifestFormat()->Identifier(), "avro"); - ASSERT_EQ(core_options.GetWriteFileFormat()->Identifier(), "parquet"); + ASSERT_EQ(core_options.GetWriteFileFormat(0)->Identifier(), "parquet"); + ASSERT_EQ(core_options.GetWriteFileFormat(3)->Identifier(), "parquet"); ASSERT_TRUE(core_options.GetFileSystem()); ASSERT_EQ(-1, core_options.GetBucket()); ASSERT_EQ(64 * 1024L, core_options.GetPageSize()); @@ -82,6 +83,10 @@ TEST(CoreOptionsTest, TestDefaultValue) { ASSERT_EQ(2 * 1024 * 1024, core_options.DeletionVectorTargetFileSize()); ASSERT_EQ(ChangelogProducer::NONE, core_options.GetChangelogProducer()); ASSERT_FALSE(core_options.NeedLookup()); + LookupStrategy expected_lookup_strategy = {/*is_first_row=*/false, + /*produce_changelog=*/false, + /*deletion_vector=*/false, /*force_lookup=*/false}; + ASSERT_EQ(expected_lookup_strategy, core_options.GetLookupStrategy()); ASSERT_TRUE(core_options.GetFieldsSequenceGroups().empty()); ASSERT_FALSE(core_options.PartialUpdateRemoveRecordOnDelete()); ASSERT_TRUE(core_options.GetPartialUpdateRemoveRecordOnSequenceGroup().empty()); @@ -185,14 +190,17 @@ TEST(CoreOptionsTest, TestFromMap) { {Options::LOOKUP_CACHE_BLOOM_FILTER_FPP, "0.5"}, {Options::LOOKUP_CACHE_SPILL_COMPRESSION, "lz4"}, {Options::SPILL_COMPRESSION_ZSTD_LEVEL, "2"}, - {Options::CACHE_PAGE_SIZE, "6MB"}}; + {Options::CACHE_PAGE_SIZE, "6MB"}, + {Options::FILE_FORMAT_PER_LEVEL, "0:AVRO,3:parquet"}}; ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); auto fs = core_options.GetFileSystem(); ASSERT_TRUE(fs); - auto format = core_options.GetWriteFileFormat(); - ASSERT_EQ(format->Identifier(), "orc"); + auto format = core_options.GetWriteFileFormat(0); + ASSERT_EQ(format->Identifier(), "avro"); + ASSERT_EQ(core_options.GetWriteFileFormat(1)->Identifier(), "orc"); + ASSERT_EQ(core_options.GetWriteFileFormat(3)->Identifier(), "parquet"); auto manifest_format = core_options.GetManifestFormat(); ASSERT_EQ(manifest_format->Identifier(), "avro"); @@ -235,6 +243,11 @@ TEST(CoreOptionsTest, TestFromMap) { ASSERT_EQ(4 * 1024 * 1024, core_options.DeletionVectorTargetFileSize()); ASSERT_EQ(ChangelogProducer::FULL_COMPACTION, core_options.GetChangelogProducer()); ASSERT_TRUE(core_options.NeedLookup()); + LookupStrategy expected_lookup_strategy = {/*is_first_row=*/false, + /*produce_changelog=*/false, + /*deletion_vector=*/true, /*need_lookup=*/true}; + ASSERT_EQ(expected_lookup_strategy, core_options.GetLookupStrategy()); + std::map seq_grp; seq_grp["g_1,g_3"] = "c,d"; ASSERT_EQ(core_options.GetFieldsSequenceGroups(), seq_grp); @@ -294,6 +307,13 @@ TEST(CoreOptionsTest, TestInvalidCase) { "invalid changelog producer: invalid"); } +TEST(CoreOptionsTest, TestInvalidFileFormatPerLevel) { + ASSERT_NOK_WITH_MSG(CoreOptions::FromMap({{Options::FILE_FORMAT_PER_LEVEL, "0:AVRO:parquet"}}), + "fail to parse key file.format.per.level, value 0:AVRO:parquet"); + ASSERT_NOK_WITH_MSG(CoreOptions::FromMap({{Options::FILE_FORMAT_PER_LEVEL, "aaa:avro"}}), + "fail to parse level aaa from string to int in file.format.per.level"); +} + TEST(CoreOptionsTest, TestCreateExternalPath) { std::map options = { {Options::DATA_FILE_EXTERNAL_PATHS, diff --git a/src/paimon/core/global_index/global_index_scan_impl.cpp b/src/paimon/core/global_index/global_index_scan_impl.cpp index 0bf5733e..b4034cbf 100644 --- a/src/paimon/core/global_index/global_index_scan_impl.cpp +++ b/src/paimon/core/global_index/global_index_scan_impl.cpp @@ -110,9 +110,10 @@ Status GlobalIndexScanImpl::Scan() { path_factory_, FileStorePathFactory::Create( root_path_, arrow_schema, table_schema_->PartitionKeys(), - options_.GetPartitionDefaultName(), options_.GetWriteFileFormat()->Identifier(), - options_.DataFilePrefix(), options_.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, options_.IndexFileInDataFileDir(), pool_)); + options_.GetPartitionDefaultName(), + options_.GetWriteFileFormat(/*level=*/0)->Identifier(), options_.DataFilePrefix(), + options_.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, + options_.IndexFileInDataFileDir(), pool_)); PAIMON_ASSIGN_OR_RAISE( std::unique_ptr index_manifest_file, diff --git a/src/paimon/core/global_index/global_index_write_task.cpp b/src/paimon/core/global_index/global_index_write_task.cpp index e031e333..2beb8e02 100644 --- a/src/paimon/core/global_index/global_index_write_task.cpp +++ b/src/paimon/core/global_index/global_index_write_task.cpp @@ -43,12 +43,13 @@ Result> CreateGlobalIndexFileManager( core_options.CreateGlobalIndexExternalPath()); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr path_factory, - FileStorePathFactory::Create( - table_path, all_arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), core_options.GetWriteFileFormat()->Identifier(), - core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), - external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), - pool)); + FileStorePathFactory::Create(table_path, all_arrow_schema, table_schema->PartitionKeys(), + core_options.GetPartitionDefaultName(), + core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.DataFilePrefix(), + core_options.LegacyPartitionNameEnabled(), external_paths, + global_index_external_path, + core_options.IndexFileInDataFileDir(), pool)); std::shared_ptr index_path_factory = path_factory->CreateGlobalIndexFileFactory(); return std::make_shared(core_options.GetFileSystem(), diff --git a/src/paimon/core/io/single_file_writer_test.cpp b/src/paimon/core/io/single_file_writer_test.cpp index c68ee4b5..9e8823f8 100644 --- a/src/paimon/core/io/single_file_writer_test.cpp +++ b/src/paimon/core/io/single_file_writer_test.cpp @@ -59,7 +59,7 @@ TEST(SingleFileWriterTest, TestSimple) { ASSERT_OK_AND_ASSIGN( CoreOptions options, CoreOptions::FromMap({{Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, "orc"}})); - auto file_format = options.GetWriteFileFormat(); + auto file_format = options.GetWriteFileFormat(/*level=*/0); auto file_system = options.GetFileSystem(); ArrowSchema arrow_schema; ASSERT_TRUE(arrow::ExportType(*data_type, &arrow_schema).ok()); @@ -91,7 +91,7 @@ TEST(SingleFileWriterTest, TestInvalidConvert) { ASSERT_OK_AND_ASSIGN( CoreOptions options, CoreOptions::FromMap({{Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, "orc"}})); - auto file_format = options.GetWriteFileFormat(); + auto file_format = options.GetWriteFileFormat(/*level=*/0); auto file_system = options.GetFileSystem(); ArrowSchema arrow_schema; ASSERT_TRUE(arrow::ExportType(*data_type, &arrow_schema).ok()); diff --git a/src/paimon/core/key_value.h b/src/paimon/core/key_value.h index e110cde4..e020d88d 100644 --- a/src/paimon/core/key_value.h +++ b/src/paimon/core/key_value.h @@ -38,13 +38,29 @@ struct KeyValue { KeyValue() = default; KeyValue(const RowKind* _value_kind, int64_t _sequence_number, int32_t _level, - std::shared_ptr&& _key, std::unique_ptr&& _value) + std::shared_ptr _key, std::shared_ptr _value) : value_kind(_value_kind), sequence_number(_sequence_number), level(_level), key(std::move(_key)), value(std::move(_value)) {} + KeyValue(const KeyValue& other) noexcept { + *this = other; + } + + KeyValue& operator=(const KeyValue& other) noexcept { + if (&other == this) { + return *this; + } + value_kind = other.value_kind; + sequence_number = other.sequence_number; + level = other.level; + key = other.key; + value = other.value; + return *this; + } + KeyValue(KeyValue&& other) noexcept { *this = std::move(other); } @@ -67,7 +83,7 @@ struct KeyValue { // determined after read from file int32_t level = -1; std::shared_ptr key; - std::unique_ptr value; + std::shared_ptr value; }; struct KeyValueBatch { diff --git a/src/paimon/core/manifest/manifest_entry_writer_test.cpp b/src/paimon/core/manifest/manifest_entry_writer_test.cpp index c275908e..24f1b59b 100644 --- a/src/paimon/core/manifest/manifest_entry_writer_test.cpp +++ b/src/paimon/core/manifest/manifest_entry_writer_test.cpp @@ -88,7 +88,7 @@ class ManifestEntryWriterTest : public ::testing::Test { EXPECT_OK_AND_ASSIGN(CoreOptions options, CoreOptions::FromMap({{Options::FILE_FORMAT, "orc"}})); - auto file_format = options.GetWriteFileFormat(); + auto file_format = options.GetWriteFileFormat(/*level=*/0); std::shared_ptr data_type = VersionedObjectSerializer::VersionType(ManifestEntry::DataType()); ArrowSchema arrow_schema; diff --git a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h new file mode 100644 index 00000000..6a3f3668 --- /dev/null +++ b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h @@ -0,0 +1,175 @@ +/* + * Copyright 2026-present Alibaba 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 "arrow/api.h" +#include "paimon/core/core_options.h" +#include "paimon/core/io/data_file_meta.h" +#include "paimon/core/io/rolling_file_writer.h" +#include "paimon/core/key_value.h" +#include "paimon/core/mergetree/compact/merge_tree_compact_rewriter.h" +#include "paimon/core/mergetree/merge_tree_writer.h" +#include "paimon/core/operation/merge_file_split_read.h" +#include "paimon/core/schema/table_schema.h" +#include "paimon/core/utils/file_store_path_factory.h" +namespace paimon { +/// A `MergeTreeCompactRewriter` which produces changelog files while performing compaction. +class ChangelogMergeTreeRewriter : public MergeTreeCompactRewriter { + public: + Result Rewrite(int32_t output_level, bool drop_delete, + const std::vector>& sections) override { + if (RewriteChangelog(output_level, drop_delete, sections)) { + return RewriteOrProduceChangelog(output_level, sections, drop_delete, + /*rewrite_compact_file=*/true); + } else { + return RewriteCompaction(output_level, drop_delete, sections); + } + } + + Result Upgrade(int32_t output_level, + const std::shared_ptr& file) override { + UpgradeStrategy upgrade_strategy = GenerateUpgradeStrategy(output_level, file); + if (upgrade_strategy.changelog) { + return RewriteOrProduceChangelog(output_level, {{SortedRun::FromSingle(file)}}, + force_drop_delete_, upgrade_strategy.rewrite); + } else { + return MergeTreeCompactRewriter::Upgrade(output_level, file); + } + } + + protected: + ChangelogMergeTreeRewriter(int32_t max_level, bool force_drop_delete, + const BinaryRow& partition, int32_t bucket, int64_t schema_id, + const std::vector& trimmed_primary_keys, + const CoreOptions& options, + const std::shared_ptr& data_schema, + const std::shared_ptr& write_schema, + const std::shared_ptr& path_factory_cache, + std::unique_ptr&& merge_file_split_read, + MergeFunctionWrapperFactory merge_function_wrapper_factory, + const std::shared_ptr& pool) + : MergeTreeCompactRewriter(partition, bucket, schema_id, trimmed_primary_keys, options, + data_schema, write_schema, path_factory_cache, + std::move(merge_file_split_read), + std::move(merge_function_wrapper_factory), pool), + max_level_(max_level), + force_drop_delete_(force_drop_delete) {} + + struct UpgradeStrategy { + static UpgradeStrategy NoChangelogNoRewrite() { + static const UpgradeStrategy ret = {false, false}; + return ret; + } + static UpgradeStrategy ChangelogNoRewrite() { + static const UpgradeStrategy ret = {true, false}; + return ret; + } + static UpgradeStrategy ChangelogWithRewrite() { + static const UpgradeStrategy ret = {true, true}; + return ret; + } + + bool operator==(const UpgradeStrategy& other) const { + if (this == &other) { + return true; + } + return changelog == other.changelog && rewrite == other.rewrite; + } + bool changelog; + bool rewrite; + }; + + virtual UpgradeStrategy GenerateUpgradeStrategy( + int32_t output_level, const std::shared_ptr& file) const = 0; + + virtual bool RewriteChangelog(int32_t output_level, bool drop_delete, + const std::vector>& sections) const = 0; + + bool RewriteLookupChangelog(int32_t output_level, + const std::vector>& sections) const { + if (output_level == 0) { + return false; + } + for (const auto& runs : sections) { + for (const auto& run : runs) { + for (const auto& file : run.Files()) { + if (file->level == 0) { + return true; + } + } + } + } + return false; + } + + private: + Result RewriteOrProduceChangelog( + int32_t output_level, const std::vector>& sections, bool drop_delete, + bool rewrite_compact_file) { + PAIMON_ASSIGN_OR_RAISE(MergeTreeCompactRewriter::KeyValueConsumerCreator create_consumer, + GenerateKeyValueConsumer()); + std::vector> reader_holders; + + std::unique_ptr compact_file_writer; + if (rewrite_compact_file) { + compact_file_writer = CreateRollingRowWriter(output_level); + } + // TODO(xinyu.lxy): produce changelog + ScopeGuard write_guard([&]() -> void { + if (compact_file_writer) { + compact_file_writer->Abort(); + } + merge_file_split_read_.reset(); + for (const auto& reader : reader_holders) { + reader->Close(); + } + }); + + for (const auto& section : sections) { + PAIMON_RETURN_NOT_OK(MergeReadAndWrite(output_level, drop_delete, section, + create_consumer, compact_file_writer.get(), + &reader_holders)); + } + if (compact_file_writer) { + PAIMON_RETURN_NOT_OK(compact_file_writer->Close()); + } + auto before = ExtractFilesFromSections(sections); + std::vector> after; + if (compact_file_writer) { + PAIMON_ASSIGN_OR_RAISE(after, compact_file_writer->GetResult()); + } else { + after.reserve(before.size()); + for (const auto& file : before) { + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr new_file, + file->Upgrade(output_level)); + after.emplace_back(std::move(new_file)); + } + } + if (rewrite_compact_file) { + NotifyRewriteCompactBefore(before); + } + write_guard.Release(); + + after = NotifyRewriteCompactAfter(after); + return CompactResult(before, after); + } + + protected: + int32_t max_level_; + bool force_drop_delete_; +}; +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/compact_rewriter.h b/src/paimon/core/mergetree/compact/compact_rewriter.h index 393b98f1..c0a2b5b0 100644 --- a/src/paimon/core/mergetree/compact/compact_rewriter.h +++ b/src/paimon/core/mergetree/compact/compact_rewriter.h @@ -44,7 +44,7 @@ class CompactRewriter { /// @param file file to be updated /// @return compaction result virtual Result Upgrade(int32_t output_level, - const std::shared_ptr& file) const = 0; + const std::shared_ptr& file) = 0; /// Close rewriter. virtual Status Close() = 0; diff --git a/src/paimon/core/mergetree/compact/first_row_merge_function.h b/src/paimon/core/mergetree/compact/first_row_merge_function.h index 96d1e573..4043a904 100644 --- a/src/paimon/core/mergetree/compact/first_row_merge_function.h +++ b/src/paimon/core/mergetree/compact/first_row_merge_function.h @@ -36,6 +36,10 @@ class FirstRowMergeFunction : public MergeFunction { contains_high_level_ = false; } + bool ContainsHighLevel() const { + return contains_high_level_; + } + Status Add(KeyValue&& moved_kv) override { KeyValue kv = std::move(moved_kv); if (kv.value_kind->IsRetract()) { diff --git a/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper.h b/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper.h new file mode 100644 index 00000000..1f357024 --- /dev/null +++ b/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper.h @@ -0,0 +1,68 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/key_value.h" +#include "paimon/core/mergetree/compact/first_row_merge_function.h" +#include "paimon/core/mergetree/compact/merge_function_wrapper.h" +#include "paimon/result.h" +#include "paimon/status.h" + +namespace paimon { +/// Wrapper for `MergeFunction`s to produce changelog by lookup for first row. +class FirstRowMergeFunctionWrapper : public MergeFunctionWrapper { + public: + FirstRowMergeFunctionWrapper( + std::unique_ptr&& merge_function, + std::function(const std::shared_ptr&)> contains) + : merge_function_(std::move(merge_function)), contains_(std::move(contains)) {} + + void Reset() override { + merge_function_->Reset(); + } + + Status Add(KeyValue&& kv) override { + return merge_function_->Add(std::move(kv)); + } + + Result> GetResult() override { + PAIMON_ASSIGN_OR_RAISE(std::optional result, merge_function_->GetResult()); + if (merge_function_->ContainsHighLevel()) { + return result; + } + assert(result); + PAIMON_ASSIGN_OR_RAISE(bool contains, contains_(result.value().key)); + if (contains) { + // empty + return std::optional(); + } + // new record, output changelog + // TODO(xinyu.lxy) support changelog + Reset(); + return result; + } + + private: + std::unique_ptr merge_function_; + std::function(const std::shared_ptr&)> contains_; +}; + +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper_test.cpp b/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper_test.cpp new file mode 100644 index 00000000..39644acc --- /dev/null +++ b/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper_test.cpp @@ -0,0 +1,107 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/mergetree/compact/first_row_merge_function_wrapper.h" + +#include +#include + +#include "gtest/gtest.h" +#include "paimon/common/data/internal_row.h" +#include "paimon/memory/memory_pool.h" +#include "paimon/testing/utils/binary_row_generator.h" +#include "paimon/testing/utils/key_value_checker.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::test { +TEST(FirstRowMergeFunctionWrapperTest, TestSimple) { + auto pool = GetDefaultPool(); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/0, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/1, /*level=*/1, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({200}, pool.get())); + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/2, /*level=*/2, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({300}, pool.get())); + + auto mfunc = std::make_unique(/*ignore_delete=*/true); + + auto contains = [](const std::shared_ptr& row) { return true; }; + + FirstRowMergeFunctionWrapper wrapper(std::move(mfunc), std::move(contains)); + wrapper.Reset(); + ASSERT_OK(wrapper.Add(std::move(kv1))); + ASSERT_OK(wrapper.Add(std::move(kv2))); + ASSERT_OK(wrapper.Add(std::move(kv3))); + ASSERT_OK_AND_ASSIGN(auto result, wrapper.GetResult()); + ASSERT_TRUE(result); + ASSERT_EQ(result.value().sequence_number, 0); +} + +TEST(FirstRowMergeFunctionWrapperTest, TestAllLevel0WithContain) { + auto pool = GetDefaultPool(); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/0, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/1, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({200}, pool.get())); + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({300}, pool.get())); + + auto mfunc = std::make_unique(/*ignore_delete=*/true); + + auto contains = [](const std::shared_ptr& row) { return true; }; + + FirstRowMergeFunctionWrapper wrapper(std::move(mfunc), std::move(contains)); + wrapper.Reset(); + ASSERT_OK(wrapper.Add(std::move(kv1))); + ASSERT_OK(wrapper.Add(std::move(kv2))); + ASSERT_OK(wrapper.Add(std::move(kv3))); + ASSERT_OK_AND_ASSIGN(auto result, wrapper.GetResult()); + ASSERT_FALSE(result); +} + +TEST(FirstRowMergeFunctionWrapperTest, TestAllLevel0WithoutContain) { + auto pool = GetDefaultPool(); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/0, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/1, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({200}, pool.get())); + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({300}, pool.get())); + + auto mfunc = std::make_unique(/*ignore_delete=*/true); + + auto contains = [](const std::shared_ptr& row) { return false; }; + + FirstRowMergeFunctionWrapper wrapper(std::move(mfunc), std::move(contains)); + wrapper.Reset(); + ASSERT_OK(wrapper.Add(std::move(kv1))); + ASSERT_OK(wrapper.Add(std::move(kv2))); + ASSERT_OK(wrapper.Add(std::move(kv3))); + ASSERT_OK_AND_ASSIGN(auto result, wrapper.GetResult()); + ASSERT_TRUE(result); + ASSERT_EQ(result.value().sequence_number, 0); +} + +} // namespace paimon::test diff --git a/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h b/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h new file mode 100644 index 00000000..5a8aa1c0 --- /dev/null +++ b/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h @@ -0,0 +1,151 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/deletionvectors/bucketed_dv_maintainer.h" +#include "paimon/core/key_value.h" +#include "paimon/core/mergetree/compact/lookup_merge_function.h" +#include "paimon/core/mergetree/compact/merge_function_wrapper.h" +#include "paimon/core/mergetree/lookup/file_position.h" +#include "paimon/core/mergetree/lookup/positioned_key_value.h" +#include "paimon/core/options/lookup_strategy.h" +#include "paimon/core/utils/fields_comparator.h" +#include "paimon/result.h" +#include "paimon/status.h" +namespace paimon { +/// Wrapper for `MergeFunction`s to produce changelog by lookup during the compaction involving +/// level 0 files. +/// +/// Changelog records are generated in the process of the level-0 file participating in the +/// compaction, if during the compaction processing: +/// +/// Without level-0 records, no changelog. +/// With level-0 record, with level-x (x > 0) record, level-x record should be BEFORE, level-0 +/// should be AFTER. +/// With level-0 record, without level-x record, need to lookup the history value of the upper +/// level as BEFORE. +/// TODO(xinyu.lxy) : add changelog +template +class LookupChangelogMergeFunctionWrapper : public MergeFunctionWrapper { + public: + static Result> Create( + std::unique_ptr&& merge_function, + std::function>(const std::shared_ptr&)> lookup, + const LookupStrategy& lookup_strategy, + const std::shared_ptr& deletion_vectors_maintainer, + const std::shared_ptr& comparator) { + if (lookup_strategy.deletion_vector && !deletion_vectors_maintainer) { + return Status::Invalid("deletionVectorsMaintainer should not be null, there is a bug."); + } + return std::unique_ptr( + new LookupChangelogMergeFunctionWrapper(std::move(merge_function), std::move(lookup), + lookup_strategy, deletion_vectors_maintainer, + comparator)); + } + void Reset() override { + merge_function_->Reset(); + } + + Status Add(KeyValue&& kv) override { + return merge_function_->Add(std::move(kv)); + } + + Result> GetResult() override { + // 1. Find the latest high level record and compute containLevel0 + std::optional high_level = merge_function_->PickHighLevel(); + + // 2. Lookup if latest high level record is absent + if (high_level == std::nullopt) { + PAIMON_ASSIGN_OR_RAISE(std::optional lookup_result, + lookup_(merge_function_->GetKey())); + if (lookup_result) { + std::string file_name; + int64_t row_position = -1; + if constexpr (std::is_same_v) { + high_level = lookup_result->key_value; + file_name = lookup_result->file_name; + row_position = lookup_result->row_position; + } else if constexpr (std::is_same_v) { + file_name = lookup_result->file_name; + row_position = lookup_result->row_position; + } else if constexpr (std::is_same_v) { + high_level = lookup_result; + } else { + return Status::Invalid( + "deletion vector mode must have PositionedKeyValue or FilePosition " + "lookup result"); + } + if (lookup_strategy_.deletion_vector) { + PAIMON_RETURN_NOT_OK( + deletion_vectors_maintainer_->NotifyNewDeletion(file_name, row_position)); + } + } + if (high_level) { + merge_function_->InsertInto(std::move(high_level), comparator_); + } + } + + // 3. Calculate result + PAIMON_ASSIGN_OR_RAISE(std::optional result, merge_function_->GetResult()); + Reset(); + // 4. Set changelog when there's level-0 records + // TODO(liancheng.lsz): setChangelog + return result; + } + + private: + LookupChangelogMergeFunctionWrapper( + std::unique_ptr&& merge_function, + std::function>(const std::shared_ptr&)> lookup, + const LookupStrategy& lookup_strategy, + const std::shared_ptr& deletion_vectors_maintainer, + const std::shared_ptr& user_defined_seq_comparator) + : merge_function_(std::move(merge_function)), + lookup_(std::move(lookup)), + lookup_strategy_(lookup_strategy), + deletion_vectors_maintainer_(deletion_vectors_maintainer), + comparator_(CreateSequenceComparator(user_defined_seq_comparator)) {} + + static std::function CreateSequenceComparator( + const std::shared_ptr& user_defined_seq_comparator) { + auto cmp_func = [user_defined_seq_comparator](const KeyValue& o1, const KeyValue& o2) { + if (user_defined_seq_comparator == nullptr) { + return o1.sequence_number < o2.sequence_number; + } + auto user_defined_result = + user_defined_seq_comparator->CompareTo(*(o1.value), *(o2.value)); + if (user_defined_result != 0) { + return user_defined_result < 0; + } + return o1.sequence_number < o2.sequence_number; + }; + return cmp_func; + } + + private: + std::unique_ptr merge_function_; + std::function>(const std::shared_ptr&)> lookup_; + LookupStrategy lookup_strategy_; + std::shared_ptr deletion_vectors_maintainer_; + std::function comparator_; +}; + +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp b/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp new file mode 100644 index 00000000..a4ca1d18 --- /dev/null +++ b/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp @@ -0,0 +1,198 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h" + +#include +#include + +#include "gtest/gtest.h" +#include "paimon/common/data/internal_row.h" +#include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" +#include "paimon/core/mergetree/compact/aggregate/aggregate_merge_function.h" +#include "paimon/core/mergetree/compact/deduplicate_merge_function.h" +#include "paimon/memory/memory_pool.h" +#include "paimon/testing/utils/binary_row_generator.h" +#include "paimon/testing/utils/key_value_checker.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::test { +TEST(LookupChangelogMergeFunctionWrapperTest, TestSimple) { + auto pool = GetDefaultPool(); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/0, /*level=*/2, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/1, /*level=*/1, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({200}, pool.get())); + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({300}, pool.get())); + + auto mfunc = std::make_unique(/*ignore_delete=*/true); + auto lookup_mfunc = std::make_unique(std::move(mfunc)); + auto lookup = [&](const std::shared_ptr& key) -> Result> { + return std::optional( + KeyValue(RowKind::Insert(), /*sequence_number=*/1000, /*level=*/3, key, + BinaryRowGenerator::GenerateRowPtr({1001}, pool.get()))); + }; + LookupStrategy lookup_strategy(/*is_first_row=*/false, /*produce_changelog=*/false, + /*deletion_vector=*/false, /*force_lookup=*/true); + ASSERT_OK_AND_ASSIGN(auto wrapper, LookupChangelogMergeFunctionWrapper::Create( + std::move(lookup_mfunc), lookup, lookup_strategy, + /*deletion_vectors_maintainer=*/nullptr, + /*comparator=*/nullptr)); + + wrapper->Reset(); + ASSERT_OK(wrapper->Add(std::move(kv1))); + ASSERT_OK(wrapper->Add(std::move(kv2))); + ASSERT_OK(wrapper->Add(std::move(kv3))); + ASSERT_OK_AND_ASSIGN(auto result, wrapper->GetResult()); + ASSERT_TRUE(result); + ASSERT_EQ(result.value().sequence_number, 2); +} + +TEST(LookupChangelogMergeFunctionWrapperTest, TestWithLookup) { + auto pool = GetDefaultPool(); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/1, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({200}, pool.get())); + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/3, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({300}, pool.get())); + + auto mfunc = std::make_unique(/*ignore_delete=*/true); + auto lookup_mfunc = std::make_unique(std::move(mfunc)); + auto lookup = [&](const std::shared_ptr& key) -> Result> { + return std::optional( + KeyValue(RowKind::Insert(), /*sequence_number=*/0, /*level=*/3, key, + BinaryRowGenerator::GenerateRowPtr({1001}, pool.get()))); + }; + LookupStrategy lookup_strategy(/*is_first_row=*/false, /*produce_changelog=*/false, + /*deletion_vector=*/false, /*force_lookup=*/true); + ASSERT_OK_AND_ASSIGN(auto wrapper, LookupChangelogMergeFunctionWrapper::Create( + std::move(lookup_mfunc), lookup, lookup_strategy, + /*deletion_vectors_maintainer=*/nullptr, + /*comparator=*/nullptr)); + + wrapper->Reset(); + ASSERT_OK(wrapper->Add(std::move(kv1))); + ASSERT_OK(wrapper->Add(std::move(kv2))); + ASSERT_OK(wrapper->Add(std::move(kv3))); + ASSERT_OK_AND_ASSIGN(auto result, wrapper->GetResult()); + ASSERT_TRUE(result); + ASSERT_EQ(result.value().sequence_number, 3); +} + +TEST(LookupChangelogMergeFunctionWrapperTest, TestWithLookupWithDv) { + auto pool = GetDefaultPool(); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/1, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({200}, pool.get())); + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/3, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({300}, pool.get())); + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, + CoreOptions::FromMap({{Options::FIELDS_DEFAULT_AGG_FUNC, "sum"}})); + ASSERT_OK_AND_ASSIGN(auto mfunc, AggregateMergeFunction::Create( + arrow::schema({arrow::field("value", arrow::int32())}), + {"key"}, core_options)); + auto lookup_mfunc = std::make_unique(std::move(mfunc)); + auto lookup = + [&](const std::shared_ptr& key) -> Result> { + return std::optional( + {KeyValue(RowKind::Insert(), /*sequence_number=*/0, /*level=*/3, key, + BinaryRowGenerator::GenerateRowPtr({1001}, pool.get())), + "data.file", /*row_position=*/10}); + }; + auto dv_index_file = + std::make_shared(/*fs=*/nullptr, /*path_factory=*/nullptr, + /*bitmap64=*/false, pool); + std::map> deletion_vectors; + auto dv_maintainer = std::make_shared(dv_index_file, deletion_vectors); + + LookupStrategy lookup_strategy(/*is_first_row=*/false, /*produce_changelog=*/false, + /*deletion_vector=*/true, /*force_lookup=*/false); + ASSERT_OK_AND_ASSIGN(auto wrapper, + LookupChangelogMergeFunctionWrapper::Create( + std::move(lookup_mfunc), lookup, lookup_strategy, dv_maintainer, + /*comparator=*/nullptr)); + + wrapper->Reset(); + ASSERT_OK(wrapper->Add(std::move(kv1))); + ASSERT_OK(wrapper->Add(std::move(kv2))); + ASSERT_OK(wrapper->Add(std::move(kv3))); + ASSERT_OK_AND_ASSIGN(auto result, wrapper->GetResult()); + ASSERT_TRUE(result); + ASSERT_EQ(result.value().sequence_number, 3); + ASSERT_EQ(result.value().value->GetInt(0), 100 + 200 + 300 + 1001); + + auto dv = dv_maintainer->DeletionVectorOf("data.file"); + ASSERT_TRUE(dv); + ASSERT_FALSE(dv.value()->IsDeleted(0).value()); + ASSERT_TRUE(dv.value()->IsDeleted(10).value()); +} + +TEST(LookupChangelogMergeFunctionWrapperTest, TestCreateSequenceComparator) { + auto pool = GetDefaultPool(); + { + // test no user_defined_seq_comparator + auto cmp = LookupChangelogMergeFunctionWrapper::CreateSequenceComparator( + /*user_defined_seq_comparator=*/nullptr); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/1, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({500}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({200}, pool.get())); + ASSERT_TRUE(cmp(kv1, kv2)); + ASSERT_FALSE(cmp(kv2, kv1)); + } + { + // test with user_defined_seq_comparator + ASSERT_OK_AND_ASSIGN( + std::shared_ptr user_defined_seq_comparator, + FieldsComparator::Create({DataField(1, arrow::field("value", arrow::int32()))}, + /*is_ascending_order=*/true, /*use_view=*/true)); + auto cmp = LookupChangelogMergeFunctionWrapper::CreateSequenceComparator( + user_defined_seq_comparator); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/1, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({200}, pool.get())); + ASSERT_TRUE(cmp(kv1, kv2)); + ASSERT_FALSE(cmp(kv2, kv1)); + + // same sequence field, compare sequence number + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/1, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + KeyValue kv4(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({100}, pool.get())); + ASSERT_TRUE(cmp(kv3, kv4)); + ASSERT_FALSE(cmp(kv4, kv3)); + } +} +} // namespace paimon::test diff --git a/src/paimon/core/mergetree/compact/lookup_merge_function.h b/src/paimon/core/mergetree/compact/lookup_merge_function.h index 5384721c..381b1e9d 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_function.h +++ b/src/paimon/core/mergetree/compact/lookup_merge_function.h @@ -37,41 +37,80 @@ class LookupMergeFunction : public MergeFunction { void Reset() override { candidates_.clear(); + current_key_ = nullptr; + contain_level0_ = false; } Status Add(KeyValue&& kv) override { + current_key_ = kv.key; + if (kv.level == 0) { + contain_level0_ = true; + } candidates_.emplace_back(std::move(kv)); return Status::OK(); } - Result> GetResult() override { - // 1. Find the latest high level record - bool has_high_level = false; - std::vector target_candidates; - target_candidates.reserve(candidates_.size()); - for (int32_t i = static_cast(candidates_.size()) - 1; i >= 0; i--) { - if (candidates_[i].level > 0) { - if (has_high_level) { - continue; - } else { - has_high_level = true; - } - } - target_candidates.emplace_back(std::move(candidates_[i])); + bool ContainLevel0() const { + return contain_level0_; + } + + const std::shared_ptr& GetKey() const { + return current_key_; + } + + std::optional PickHighLevel() const { + int32_t idx = PickHighLevelIdx(); + if (idx == -1) { + return std::optional(); } - // 2. Do the merge for inputs + return candidates_[idx]; + } + + Result> GetResult() override { merge_function_->Reset(); - // step 1 visits candidates_ from end to begin, therefore elements in target_candidates need - // to be reversely accessed - for (int32_t i = static_cast(target_candidates.size()) - 1; i >= 0; i--) { - PAIMON_RETURN_NOT_OK(merge_function_->Add(std::move(target_candidates[i]))); + int32_t high_level_idx = PickHighLevelIdx(); + for (int32_t i = 0; i < static_cast(candidates_.size()); ++i) { + // records that has not been stored on the disk yet, such as the data in the write + // buffer being at level -1 + if (candidates_[i].level <= 0 || i == high_level_idx) { + PAIMON_RETURN_NOT_OK(merge_function_->Add(std::move(candidates_[i]))); + } } - candidates_.clear(); return merge_function_->GetResult(); } + void InsertInto(std::optional&& high_level, + std::function cmp_function) { + if (!high_level) { + return; + } + candidates_.push_back(std::move(high_level.value())); + std::sort(candidates_.begin(), candidates_.end(), cmp_function); + } + + private: + int32_t PickHighLevelIdx() const { + int32_t high_level_idx = -1; + for (int32_t i = 0; i < static_cast(candidates_.size()); i++) { + const auto& kv = candidates_[i]; + // records that has not been stored on the disk yet, such as the data in the write + // buffer being at level -1 + if (kv.level <= 0) { + continue; + } + // For high-level comparison logic (not involving Level 0), only the value of the + // minimum Level should be selected + if (high_level_idx == -1 || kv.level < candidates_[high_level_idx].level) { + high_level_idx = i; + } + } + return high_level_idx; + } + private: std::unique_ptr merge_function_; std::vector candidates_; + std::shared_ptr current_key_; + bool contain_level0_ = false; }; } // namespace paimon diff --git a/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp b/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp index ed80cfbb..9336a0d2 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp +++ b/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp @@ -119,4 +119,72 @@ TEST(LookupMergeFunctionTest, TestDeduplicate) { KeyValueChecker::CheckResult(expected2, result_kv, /*key_arity=*/1, /*value_arity=*/2); } +TEST(LookupMergeFunctionTest, TestPickHighLevel) { + auto deduplicate_merge_func = + std::make_unique(/*ignore_delete=*/false); + auto merge_func = std::make_unique(std::move(deduplicate_merge_func)); + + auto pool = GetDefaultPool(); + + merge_func->Reset(); + ASSERT_FALSE(merge_func->PickHighLevel()); + ASSERT_FALSE(merge_func->ContainLevel0()); + + merge_func->Reset(); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/1, /*level=*/2, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/2, /*level=*/1, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 200}, pool.get())); + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/4, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 300}, pool.get())); + + ASSERT_OK(merge_func->Add(std::move(kv1))); + ASSERT_OK(merge_func->Add(std::move(kv2))); + ASSERT_OK(merge_func->Add(std::move(kv3))); + ASSERT_TRUE(merge_func->ContainLevel0()); + ASSERT_EQ(merge_func->GetKey()->GetInt(0), 10); + auto result_kv = std::move(merge_func->PickHighLevel().value()); + KeyValue expected(RowKind::Insert(), /*sequence_number=*/2, /*level=*/1, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 200}, pool.get())); + KeyValueChecker::CheckResult(expected, result_kv, /*key_arity=*/1, /*value_arity=*/2); +} + +TEST(LookupMergeFunctionTest, TestInsertInto) { + auto deduplicate_merge_func = + std::make_unique(/*ignore_delete=*/false); + auto merge_func = std::make_unique(std::move(deduplicate_merge_func)); + + auto pool = GetDefaultPool(); + + merge_func->Reset(); + KeyValue kv1(RowKind::Insert(), /*sequence_number=*/1, /*level=*/0, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 100}, pool.get())); + KeyValue kv2(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 200}, pool.get())); + KeyValue kv3(RowKind::Insert(), /*sequence_number=*/0, /*level=*/3, /*key=*/ + BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 300}, pool.get())); + + ASSERT_OK(merge_func->Add(std::move(kv1))); + ASSERT_OK(merge_func->Add(std::move(kv2))); + merge_func->InsertInto(std::move(kv3), [](const KeyValue& o1, const KeyValue& o2) { + return o1.sequence_number < o2.sequence_number; + }); + ASSERT_EQ(merge_func->candidates_.size(), 3); + ASSERT_EQ(merge_func->candidates_[0].sequence_number, 0); + ASSERT_EQ(merge_func->candidates_[1].sequence_number, 1); + ASSERT_EQ(merge_func->candidates_[2].sequence_number, 2); + + auto result_kv = std::move(merge_func->GetResult().value().value()); + KeyValue expected(RowKind::Insert(), /*sequence_number=*/2, /*level=*/0, + /*key=*/BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), + /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 200}, pool.get())); + KeyValueChecker::CheckResult(expected, result_kv, /*key_arity=*/1, /*value_arity=*/2); +} } // namespace paimon::test diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h new file mode 100644 index 00000000..e1ba178f --- /dev/null +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h @@ -0,0 +1,185 @@ +/* + * Copyright 2026-present Alibaba 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 "arrow/api.h" +#include "paimon/common/table/special_fields.h" +#include "paimon/core/core_options.h" +#include "paimon/core/io/data_file_meta.h" +#include "paimon/core/io/rolling_file_writer.h" +#include "paimon/core/key_value.h" +#include "paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h" +#include "paimon/core/mergetree/compact/first_row_merge_function.h" +#include "paimon/core/mergetree/compact/first_row_merge_function_wrapper.h" +#include "paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h" +#include "paimon/core/mergetree/compact/lookup_merge_function.h" +#include "paimon/core/mergetree/lookup_levels.h" +#include "paimon/core/mergetree/merge_tree_writer.h" +#include "paimon/core/operation/merge_file_split_read.h" +#include "paimon/core/schema/table_schema.h" +#include "paimon/core/utils/file_store_path_factory.h" +namespace paimon { +/// A `MergeTreeCompactRewriter` which produces changelog files by lookup for the compaction +/// involving level 0 files. +template +class LookupMergeTreeCompactRewriter : public ChangelogMergeTreeRewriter { + public: + static Result> Create( + int32_t max_level, std::unique_ptr>&& lookup_levels, + const std::shared_ptr& dv_maintainer, + MergeFunctionWrapperFactory merge_function_wrapper_factory, int32_t bucket, + const BinaryRow& partition, const std::shared_ptr& table_schema, + const std::shared_ptr& path_factory_cache, + const CoreOptions& options, const std::shared_ptr& pool) { + PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, + table_schema->TrimmedPrimaryKeys()); + auto data_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); + auto write_schema = SpecialFields::CompleteSequenceAndValueKindField(data_schema); + + // TODO(xinyu.lxy): set executor + ReadContextBuilder read_context_builder(path_factory_cache->RootPath()); + read_context_builder.SetOptions(options.ToMap()).EnablePrefetch(true).WithMemoryPool(pool); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr read_context, + read_context_builder.Finish()); + + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr internal_context, + InternalReadContext::Create(read_context, table_schema, options.ToMap())); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr path_factory, + path_factory_cache->GetOrCreatePathFactory( + options.GetWriteFileFormat(/*level=*/0)->Identifier())); + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr merge_file_split_read, + MergeFileSplitRead::Create(path_factory, internal_context, pool, + CreateDefaultExecutor())); + return std::unique_ptr(new LookupMergeTreeCompactRewriter( + std::move(lookup_levels), dv_maintainer, max_level, partition, bucket, + table_schema->Id(), trimmed_primary_keys, options, data_schema, write_schema, + path_factory_cache, std::move(merge_file_split_read), + std::move(merge_function_wrapper_factory), pool)); + } + + Status Close() override { + return lookup_levels_->Close(); + } + + static std::shared_ptr> CreateFirstRowMergeFunctionWrapper( + std::unique_ptr&& merge_func, int32_t output_level, + LookupLevels* lookup_levels) { + auto contains = [output_level, + lookup_levels](const std::shared_ptr& key) -> Result { + PAIMON_ASSIGN_OR_RAISE(std::optional contain, + lookup_levels->Lookup(key, output_level + 1)); + return contain != std::nullopt; + }; + return std::make_shared(std::move(merge_func), + std::move(contains)); + } + + static Result>> CreateLookupMergeFunctionWrapper( + std::unique_ptr&& merge_func, int32_t output_level, + const std::shared_ptr& deletion_vectors_maintainer, + const LookupStrategy& lookup_strategy, + const std::shared_ptr& user_defined_seq_comparator, + LookupLevels* lookup_levels) { + auto lookup = [output_level, lookup_levels]( + const std::shared_ptr& key) -> Result> { + return lookup_levels->Lookup(key, output_level + 1); + }; + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr> wrapper, + LookupChangelogMergeFunctionWrapper::Create( + std::move(merge_func), std::move(lookup), lookup_strategy, + deletion_vectors_maintainer, user_defined_seq_comparator)); + return wrapper; + } + + private: + LookupMergeTreeCompactRewriter( + std::unique_ptr>&& lookup_levels, + const std::shared_ptr& dv_maintainer, int32_t max_level, + const BinaryRow& partition, int32_t bucket, int64_t schema_id, + const std::vector& trimmed_primary_keys, const CoreOptions& options, + const std::shared_ptr& data_schema, + const std::shared_ptr& write_schema, + const std::shared_ptr& path_factory_cache, + std::unique_ptr&& merge_file_split_read, + MergeFunctionWrapperFactory merge_function_wrapper_factory, + const std::shared_ptr& pool) + : ChangelogMergeTreeRewriter(max_level, /*force_drop_delete=*/dv_maintainer_ != nullptr, + partition, bucket, schema_id, trimmed_primary_keys, options, + data_schema, write_schema, path_factory_cache, + std::move(merge_file_split_read), + std::move(merge_function_wrapper_factory), pool), + lookup_levels_(std::move(lookup_levels)), + dv_maintainer_(dv_maintainer) {} + + bool RewriteChangelog(int32_t output_level, bool drop_delete, + const std::vector>& sections) const override { + return RewriteLookupChangelog(output_level, sections); + } + + UpgradeStrategy GenerateUpgradeStrategy( + int32_t output_level, const std::shared_ptr& file) const override { + if (file->level != 0) { + return UpgradeStrategy::NoChangelogNoRewrite(); + } + // forcing rewriting when upgrading from level 0 to level x with different file formats + if (options_.GetWriteFileFormat(file->level)->Identifier() != + options_.GetWriteFileFormat(output_level)->Identifier()) { + return UpgradeStrategy::ChangelogWithRewrite(); + } + + // In deletionVector mode, since drop delete is required, when delete row count > 0 rewrite + // is required. + if (dv_maintainer_ && (!file->delete_row_count || file->delete_row_count.value() > 0)) { + return UpgradeStrategy::ChangelogWithRewrite(); + } + + if (output_level == max_level_) { + return UpgradeStrategy::ChangelogNoRewrite(); + } + + // DEDUPLICATE retains the latest records as the final result, so merging has no impact on + // it at all. + if (options_.GetMergeEngine() == MergeEngine::DEDUPLICATE && + options_.GetSequenceField().empty()) { + return UpgradeStrategy::ChangelogNoRewrite(); + } + // other merge engines must rewrite file, because some records that are already at higher + // level may be merged + // See LookupMergeFunction, it just returns newly records. + return UpgradeStrategy::ChangelogWithRewrite(); + } + + void NotifyRewriteCompactBefore( + const std::vector>& files) override { + if (dv_maintainer_) { + for (const auto& file : files) { + dv_maintainer_->RemoveDeletionVectorOf(file->file_name); + } + } + } + + std::vector> NotifyRewriteCompactAfter( + const std::vector>& files) override { + // TODO(xinyu.lxy): support remoteLookupFileManager + return files; + } + + private: + std::unique_ptr> lookup_levels_; + std::shared_ptr dv_maintainer_; +}; +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp new file mode 100644 index 00000000..cdd3af2b --- /dev/null +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp @@ -0,0 +1,1017 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h" + +#include "arrow/api.h" +#include "arrow/c/abi.h" +#include "arrow/c/bridge.h" +#include "arrow/ipc/json_simple.h" +#include "gtest/gtest.h" +#include "paimon/catalog/catalog.h" +#include "paimon/common/factories/io_hook.h" +#include "paimon/common/table/special_fields.h" +#include "paimon/common/utils/path_util.h" +#include "paimon/core/core_options.h" +#include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" +#include "paimon/core/io/data_file_path_factory.h" +#include "paimon/core/mergetree/compact/aggregate/aggregate_merge_function.h" +#include "paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h" +#include "paimon/core/mergetree/compact/deduplicate_merge_function.h" +#include "paimon/core/mergetree/compact/interval_partition.h" +#include "paimon/core/mergetree/compact/reducer_merge_function_wrapper.h" +#include "paimon/core/mergetree/lookup/default_lookup_serializer_factory.h" +#include "paimon/core/mergetree/lookup/persist_empty_processor.h" +#include "paimon/core/mergetree/lookup/persist_position_processor.h" +#include "paimon/core/mergetree/lookup/persist_value_and_pos_processor.h" +#include "paimon/core/mergetree/lookup/persist_value_processor.h" +#include "paimon/core/mergetree/lookup/positioned_key_value.h" +#include "paimon/core/mergetree/lookup_levels.h" +#include "paimon/core/mergetree/merge_tree_writer.h" +#include "paimon/core/schema/schema_manager.h" +#include "paimon/core/table/source/data_split_impl.h" +#include "paimon/core/utils/fields_comparator.h" +#include "paimon/format/file_format_factory.h" +#include "paimon/memory/memory_pool.h" +#include "paimon/record_batch.h" +#include "paimon/scan_context.h" +#include "paimon/table/source/table_scan.h" +#include "paimon/testing/mock/mock_index_path_factory.h" +#include "paimon/testing/utils/binary_row_generator.h" +#include "paimon/testing/utils/io_exception_helper.h" +#include "paimon/testing/utils/read_result_collector.h" +#include "paimon/testing/utils/testharness.h" +namespace paimon::test { +class LookupMergeTreeCompactRewriterTest : public testing::Test { + public: + void SetUp() override { + pool_ = GetDefaultPool(); + arrow::FieldVector fields = { + arrow::field("key", arrow::int32()), + arrow::field("value", arrow::int32()), + }; + arrow_schema_ = arrow::schema(fields); + key_schema_ = arrow::schema({fields[0]}); + tmp_dir_ = UniqueTestDirectory::Create("local"); + dir_ = UniqueTestDirectory::Create("local"); + fs_ = dir_->GetFileSystem(); + } + + void TearDown() override {} + + Result> NewFiles(int32_t level, int64_t last_sequence_number, + const std::string& table_path, + const CoreOptions& options, + const std::string& src_array_str) const { + std::shared_ptr src_array = + arrow::ipc::internal::json::ArrayFromJSON(arrow::struct_(arrow_schema_->fields()), + src_array_str) + .ValueOrDie(); + + // prepare writer + PAIMON_ASSIGN_OR_RAISE(auto path_factory, CreateFileStorePathFactory(table_path, options)); + PAIMON_ASSIGN_OR_RAISE(auto data_path_factory, path_factory->CreateDataFilePathFactory( + BinaryRow::EmptyRow(), /*bucket=*/0)); + PAIMON_ASSIGN_OR_RAISE(auto key_comparator, CreateKeyComparator()); + auto mfunc = std::make_unique(/*ignore_delete=*/false); + auto merge_function_wrapper = + std::make_shared(std::move(mfunc)); + + auto writer = std::make_shared( + /*last_sequence_number=*/last_sequence_number, std::vector({"key"}), + data_path_factory, key_comparator, + /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper, /*schema_id=*/0, + arrow_schema_, options, pool_); + + // write data + ArrowArray c_src_array; + PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportArray(*src_array, &c_src_array)); + RecordBatchBuilder batch_builder(&c_src_array); + batch_builder.SetBucket(0); + PAIMON_ASSIGN_OR_RAISE(auto batch, batch_builder.Finish()); + PAIMON_RETURN_NOT_OK(writer->Write(std::move(batch))); + // get file meta + PAIMON_ASSIGN_OR_RAISE(auto commit_increment, + writer->PrepareCommit(/*wait_compaction=*/false)); + const auto& file_metas = commit_increment.GetNewFilesIncrement().NewFiles(); + EXPECT_EQ(file_metas.size(), 1); + auto file_meta = file_metas[0]; + file_meta->level = level; + return file_meta; + } + + Result> CreateKeyComparator() const { + std::vector key_fields = {DataField(0, key_schema_->field(0))}; + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr key_comparator, + FieldsComparator::Create(key_fields, + /*is_ascending_order=*/true, + /*use_view=*/false)); + return key_comparator; + } + + Result CreateTable(const std::map& options) const { + ::ArrowSchema c_schema; + PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*arrow_schema_, &c_schema)); + + PAIMON_ASSIGN_OR_RAISE(auto catalog, Catalog::Create(dir_->Str(), {})); + PAIMON_RETURN_NOT_OK(catalog->CreateDatabase("foo", {}, /*ignore_if_exists=*/false)); + PAIMON_RETURN_NOT_OK(catalog->CreateTable(Identifier("foo", "bar"), &c_schema, + /*partition_keys=*/{}, + /*primary_keys=*/{"key"}, options, + /*ignore_if_exists=*/false)); + return PathUtil::JoinPath(dir_->Str(), "foo.db/bar"); + } + + Result>> CreateCompactRewriterForFirstRow( + const std::string& table_path, const std::shared_ptr& table_schema, + const CoreOptions& options, std::unique_ptr>&& lookup_levels) const { + auto path_factory_cache = + std::make_shared(table_path, table_schema, options, pool_); + auto merge_function_wrapper_factory = + [lookup_levels_ptr = lookup_levels.get()]( + int32_t output_level) -> Result>> { + std::shared_ptr> merge_function_wrapper = + LookupMergeTreeCompactRewriter::CreateFirstRowMergeFunctionWrapper( + std::make_unique(/*ignore_delete=*/true), output_level, + lookup_levels_ptr); + return merge_function_wrapper; + }; + + return LookupMergeTreeCompactRewriter::Create( + /*max_level=*/5, std::move(lookup_levels), /*dv_maintainer=*/nullptr, + std::move(merge_function_wrapper_factory), + /*bucket=*/0, + /*partition=*/BinaryRow::EmptyRow(), table_schema, path_factory_cache, options, pool_); + } + + Result>> + CreateCompactRewriterForKeyValue( + const std::string& table_path, const std::shared_ptr& table_schema, + const CoreOptions& options, std::unique_ptr>&& lookup_levels) const { + auto path_factory_cache = + std::make_shared(table_path, table_schema, options, pool_); + auto merge_function_wrapper_factory = + [this, table_schema, options, lookup_levels_ptr = lookup_levels.get(), + lookup_strategy = options.GetLookupStrategy()]( + int32_t output_level) -> Result>> { + PAIMON_ASSIGN_OR_RAISE( + auto merge_func, + AggregateMergeFunction::Create( + arrow_schema_, table_schema->TrimmedPrimaryKeys().value(), options)); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr> merge_function_wrapper, + LookupMergeTreeCompactRewriter::CreateLookupMergeFunctionWrapper( + std::make_unique(std::move(merge_func)), output_level, + /*deletion_vectors_maintainer=*/nullptr, lookup_strategy, + /*user_defined_seq_comparator=*/nullptr, lookup_levels_ptr)); + return merge_function_wrapper; + }; + + return LookupMergeTreeCompactRewriter::Create( + /*max_level=*/5, std::move(lookup_levels), /*dv_maintainer=*/nullptr, + std::move(merge_function_wrapper_factory), /*bucket=*/0, + /*partition=*/BinaryRow::EmptyRow(), table_schema, path_factory_cache, options, pool_); + } + + Result>> + CreateCompactRewriterForFilePosition( + const std::string& table_path, const std::shared_ptr& table_schema, + const CoreOptions& options, + std::unique_ptr>&& lookup_levels) const { + auto path_factory_cache = + std::make_shared(table_path, table_schema, options, pool_); + + auto path_factory = std::make_shared(table_path + "/index/"); + auto dv_index_file = std::make_shared(fs_, path_factory, + /*bitmap64=*/false, pool_); + std::map> deletion_vectors; + auto dv_maintainer = + std::make_shared(dv_index_file, deletion_vectors); + + auto merge_function_wrapper_factory = + [lookup_levels_ptr = lookup_levels.get(), lookup_strategy = options.GetLookupStrategy(), + dv_maintainer_ptr = dv_maintainer]( + int32_t output_level) -> Result>> { + auto merge_func = std::make_unique(false); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr> merge_function_wrapper, + LookupMergeTreeCompactRewriter::CreateLookupMergeFunctionWrapper( + std::make_unique(std::move(merge_func)), output_level, + dv_maintainer_ptr, lookup_strategy, + /*user_defined_seq_comparator=*/nullptr, lookup_levels_ptr)); + return merge_function_wrapper; + }; + + return LookupMergeTreeCompactRewriter::Create( + /*max_level=*/5, std::move(lookup_levels), dv_maintainer, + std::move(merge_function_wrapper_factory), /*bucket=*/0, + /*partition=*/BinaryRow::EmptyRow(), table_schema, path_factory_cache, options, pool_); + } + + Result>> + CreateCompactRewriterForPositionedKeyValue( + const std::string& table_path, const std::shared_ptr& table_schema, + const CoreOptions& options, + std::unique_ptr>&& lookup_levels) const { + auto path_factory_cache = + std::make_shared(table_path, table_schema, options, pool_); + + auto path_factory = std::make_shared(table_path + "/index/"); + auto dv_index_file = std::make_shared(fs_, path_factory, + /*bitmap64=*/false, pool_); + std::map> deletion_vectors; + auto dv_maintainer = + std::make_shared(dv_index_file, deletion_vectors); + + auto merge_function_wrapper_factory = + [this, table_schema, options, lookup_levels_ptr = lookup_levels.get(), + lookup_strategy = options.GetLookupStrategy(), dv_maintainer_ptr = dv_maintainer]( + int32_t output_level) -> Result>> { + PAIMON_ASSIGN_OR_RAISE( + auto merge_func, + AggregateMergeFunction::Create( + arrow_schema_, table_schema->TrimmedPrimaryKeys().value(), options)); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr> merge_function_wrapper, + LookupMergeTreeCompactRewriter:: + CreateLookupMergeFunctionWrapper( + std::make_unique(std::move(merge_func)), output_level, + dv_maintainer_ptr, lookup_strategy, + /*user_defined_seq_comparator=*/nullptr, lookup_levels_ptr)); + return merge_function_wrapper; + }; + + return LookupMergeTreeCompactRewriter::Create( + /*max_level=*/5, std::move(lookup_levels), dv_maintainer, + std::move(merge_function_wrapper_factory), /*bucket=*/0, + /*partition=*/BinaryRow::EmptyRow(), table_schema, path_factory_cache, options, pool_); + } + + void CheckResult(const std::string& compact_file_name, + const std::shared_ptr& table_schema, + const std::string& file_format_str, + const std::shared_ptr& expected_array) const { + ASSERT_OK_AND_ASSIGN(auto file_format, + FileFormatFactory::Get(file_format_str, table_schema->Options())); + ASSERT_OK_AND_ASSIGN(auto reader_builder, + file_format->CreateReaderBuilder(/*batch_size=*/10)); + ASSERT_OK_AND_ASSIGN(std::shared_ptr input_stream, + fs_->Open(compact_file_name)); + ASSERT_OK_AND_ASSIGN(auto file_batch_reader, reader_builder->Build(input_stream)); + ASSERT_OK_AND_ASSIGN(auto result_array, + ReadResultCollector::CollectResult(file_batch_reader.get())); + // handle type nullable, as result_array does not have not null flag + result_array = result_array->View(expected_array->type()).ValueOrDie(); + + ASSERT_TRUE(expected_array->type()->Equals(result_array->type())) + << "result=" << result_array->type()->ToString() + << ", expected=" << expected_array->type()->ToString() << std::endl; + ASSERT_TRUE(expected_array->Equals(*result_array)) << result_array->ToString(); + } + + Result> CreateFileStorePathFactory( + const std::string& table_path, const CoreOptions& options) const { + PAIMON_ASSIGN_OR_RAISE(std::vector external_paths, + options.CreateExternalPaths()); + PAIMON_ASSIGN_OR_RAISE(std::optional global_index_external_path, + options.CreateGlobalIndexExternalPath()); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr path_factory, + FileStorePathFactory::Create( + table_path, arrow_schema_, /*partition_keys=*/{}, options.GetPartitionDefaultName(), + options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), + options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, + options.IndexFileInDataFileDir(), pool_)); + return path_factory; + } + + template + Result>> CreateLookupLevels( + const std::string& table_path, const std::shared_ptr& table_schema, + const std::shared_ptr::Factory>& processor_factory, + const std::vector>& files) const { + auto schema_manager = std::make_shared(fs_, table_path); + PAIMON_ASSIGN_OR_RAISE(auto key_comparator, CreateKeyComparator()); + PAIMON_ASSIGN_OR_RAISE(auto levels, + Levels::Create(key_comparator, files, /*num_levels=*/5)); + PAIMON_ASSIGN_OR_RAISE(CoreOptions options, CoreOptions::FromMap(table_schema->Options())); + + auto io_manager = IOManager::Create(tmp_dir_->Str()); + auto serializer_factory = std::make_shared(); + PAIMON_ASSIGN_OR_RAISE(auto lookup_key_comparator, + RowCompactedSerializer::CreateSliceComparator(key_schema_, pool_)); + PAIMON_ASSIGN_OR_RAISE(auto lookup_store_factory, + LookupStoreFactory::Create(lookup_key_comparator, options)); + PAIMON_ASSIGN_OR_RAISE(auto path_factory, CreateFileStorePathFactory(table_path, options)); + return LookupLevels::Create(fs_, BinaryRow::EmptyRow(), /*bucket=*/0, options, + schema_manager, std::move(io_manager), path_factory, + table_schema, std::move(levels), + /*deletion_file_map=*/{}, processor_factory, + serializer_factory, lookup_store_factory, pool_); + } + + Result>> GenerateSortedRuns( + const std::vector>& files) const { + PAIMON_ASSIGN_OR_RAISE(auto key_comparator, CreateKeyComparator()); + IntervalPartition interval_partition(files, key_comparator); + return interval_partition.Partition(); + } + + private: + std::shared_ptr pool_; + std::shared_ptr arrow_schema_; + std::shared_ptr key_schema_; + std::unique_ptr tmp_dir_; + std::unique_ptr dir_; + std::shared_ptr fs_; +}; + +TEST_F(LookupMergeTreeCompactRewriterTest, TestFirstRowRewrite) { + std::map options = {{Options::MERGE_ENGINE, "first-row"}, + {Options::FILE_FORMAT, "orc"}}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 2 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/0, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 5]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/0, /*last_sequence_number=*/2, table_path, + core_options, "[[2, 22], [5, 55]]")); + std::vector> files = {file0, file1}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN(auto lookup_levels, CreateLookupLevels(table_path, table_schema, + processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN(auto rewriter, + CreateCompactRewriterForFirstRow(table_path, table_schema, core_options, + std::move(lookup_levels))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns(files)); + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Rewrite( + /*output_level=*/5, /*drop_delete=*/true, runs)); + ASSERT_EQ(2, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + // check compact result + const auto& compact_file_meta = compact_result.After()[0]; + auto expected_file_meta = std::make_shared( + "file.orc", 100l, /*row_count=*/4, + /*min_key=*/BinaryRowGenerator::GenerateRow({1}, pool_.get()), + /*max_key=*/BinaryRowGenerator::GenerateRow({5}, pool_.get()), + /*key_stats=*/ + BinaryRowGenerator::GenerateStats({1}, {5}, {0}, pool_.get()), + /*value_stats=*/ + BinaryRowGenerator::GenerateStats({1, 5}, {5, 33}, {0, 0}, pool_.get()), + /*min_sequence_number=*/0l, /*max_sequence_number=*/3l, /*schema_id=*/0, /*level=*/5, + std::vector>(), Timestamp(0l, 0), /*delete_row_count=*/0, + nullptr, FileSource::Compact(), std::nullopt, std::nullopt, std::nullopt, std::nullopt); + ASSERT_TRUE(expected_file_meta->TEST_Equal(*compact_file_meta)); + + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = + arrow::struct_(SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[0, 0, 1, 11], +[3, 0, 2, 22], +[1, 0, 3, 33], +[2, 0, 5, 5] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "orc", expected_array); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestFirstRowUpgrade) { + std::map options = {{Options::MERGE_ENGINE, "first-row"}, + {Options::FILE_FORMAT, "orc"}}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 1 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/0, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 5]]")); + std::vector> files = {file0}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN(auto lookup_levels, CreateLookupLevels(table_path, table_schema, + processor_factory, files)); + + // upgrade and rewrite + ASSERT_OK_AND_ASSIGN(auto rewriter, + CreateCompactRewriterForFirstRow(table_path, table_schema, core_options, + std::move(lookup_levels))); + // output_level = max_level, with ChangelogNoRewrite UpgradeStrategy + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Upgrade( + /*output_level=*/5, file0)); + ASSERT_EQ(1, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + // check compact result + const auto& compact_file_meta = compact_result.After()[0]; + // ChangelogNoRewrite only upgrade file level without rewrite + ASSERT_EQ(compact_file_meta->file_name, file0->file_name); + + auto expected_file_meta = std::make_shared( + "file.orc", 100l, /*row_count=*/3, + /*min_key=*/BinaryRowGenerator::GenerateRow({1}, pool_.get()), + /*max_key=*/BinaryRowGenerator::GenerateRow({5}, pool_.get()), + /*key_stats=*/ + BinaryRowGenerator::GenerateStats({1}, {5}, {0}, pool_.get()), + /*value_stats=*/ + BinaryRowGenerator::GenerateStats({1, 5}, {5, 33}, {0, 0}, pool_.get()), + /*min_sequence_number=*/0l, /*max_sequence_number=*/2l, /*schema_id=*/0, /*level=*/5, + std::vector>(), Timestamp(0l, 0), /*delete_row_count=*/0, + nullptr, FileSource::Append(), std::nullopt, std::nullopt, std::nullopt, std::nullopt); + ASSERT_TRUE(expected_file_meta->TEST_Equal(*compact_file_meta)) + << compact_file_meta->ToString(); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestWithFileFormatPerLevel) { + std::map options = {{Options::MERGE_ENGINE, "first-row"}, + {Options::FILE_FORMAT, "orc"}, + {Options::FILE_FORMAT_PER_LEVEL, "5:parquet"}}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 2 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/0, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 5]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/0, /*last_sequence_number=*/2, table_path, + core_options, "[[2, 22], [5, 55]]")); + std::vector> files = {file0, file1}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN(auto lookup_levels, CreateLookupLevels(table_path, table_schema, + processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN(auto rewriter, + CreateCompactRewriterForFirstRow(table_path, table_schema, core_options, + std::move(lookup_levels))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns(files)); + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Rewrite( + /*output_level=*/5, /*drop_delete=*/true, runs)); + ASSERT_EQ(2, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + const auto& compact_file_meta = compact_result.After()[0]; + ASSERT_TRUE(StringUtils::EndsWith(compact_file_meta->file_name, ".parquet")); + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = + arrow::struct_(SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[0, 0, 1, 11], +[3, 0, 2, 22], +[1, 0, 3, 33], +[2, 0, 5, 5] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "parquet", expected_array); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestRewriteWithAllHighLevel) { + // When all input files are high level, rewrite will call RewriteCompaction + std::map options = {{Options::MERGE_ENGINE, "first-row"}, + {Options::FILE_FORMAT, "orc"}, + {Options::FILE_FORMAT_PER_LEVEL, "5:parquet"}}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 2 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/1, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 5]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/1, /*last_sequence_number=*/2, table_path, + core_options, "[[8, 88], [9, 99]]")); + std::vector> files = {file0, file1}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN(auto lookup_levels, CreateLookupLevels(table_path, table_schema, + processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN(auto rewriter, + CreateCompactRewriterForFirstRow(table_path, table_schema, core_options, + std::move(lookup_levels))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns(files)); + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Rewrite( + /*output_level=*/5, /*drop_delete=*/true, runs)); + ASSERT_EQ(2, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + const auto& compact_file_meta = compact_result.After()[0]; + ASSERT_TRUE(StringUtils::EndsWith(compact_file_meta->file_name, ".parquet")); + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = + arrow::struct_(SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[0, 0, 1, 11], +[1, 0, 3, 33], +[2, 0, 5, 5], +[3, 0, 8, 88], +[4, 0, 9, 99] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "parquet", expected_array); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestRewriteWithForceLookupAndSumAgg) { + std::map options = {{Options::MERGE_ENGINE, "aggregation"}, + {Options::FILE_FORMAT, "orc"}, + {Options::FORCE_LOOKUP, "true"}, + {Options::FIELDS_DEFAULT_AGG_FUNC, "sum"}}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 3 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/5, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 55]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/0, /*last_sequence_number=*/2, table_path, + core_options, "[[2, 22], [4, 44], [5, 5]]")); + ASSERT_OK_AND_ASSIGN(auto file2, NewFiles(/*level=*/0, /*last_sequence_number=*/5, table_path, + core_options, "[[2, 222], [5, 555]]")); + std::vector> files = {file0, file1, file2}; + auto processor_factory = std::make_shared(arrow_schema_); + ASSERT_OK_AND_ASSIGN( + auto lookup_levels, + CreateLookupLevels(table_path, table_schema, processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN(auto rewriter, + CreateCompactRewriterForKeyValue(table_path, table_schema, core_options, + std::move(lookup_levels))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns({file1, file2})); + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Rewrite( + /*output_level=*/4, /*drop_delete=*/true, runs)); + ASSERT_EQ(2, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + const auto& compact_file_meta = compact_result.After()[0]; + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = + arrow::struct_(SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[6, 0, 2, 244], +[4, 0, 4, 44], +[7, 0, 5, 615] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "orc", expected_array); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestRewriteWithDvAndDeduplicate) { + std::map options = { + {Options::MERGE_ENGINE, "deduplicate"}, + {Options::FILE_FORMAT, "orc"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 3 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/5, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 55]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/0, /*last_sequence_number=*/2, table_path, + core_options, "[[2, 22], [4, 44], [5, 5]]")); + ASSERT_OK_AND_ASSIGN(auto file2, NewFiles(/*level=*/0, /*last_sequence_number=*/5, table_path, + core_options, "[[2, 222], [5, 555]]")); + std::vector> files = {file0, file1, file2}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN( + auto lookup_levels, + CreateLookupLevels(table_path, table_schema, processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN( + auto rewriter, CreateCompactRewriterForFilePosition(table_path, table_schema, core_options, + std::move(lookup_levels))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns({file1, file2})); + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Rewrite( + /*output_level=*/4, /*drop_delete=*/true, runs)); + ASSERT_EQ(2, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + const auto& compact_file_meta = compact_result.After()[0]; + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = + arrow::struct_(SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[6, 0, 2, 222], +[4, 0, 4, 44], +[7, 0, 5, 555] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "orc", expected_array); + + // test dv + auto dv_maintainer = rewriter->dv_maintainer_; + ASSERT_TRUE(dv_maintainer); + auto dv = dv_maintainer->DeletionVectorOf(file0->file_name); + ASSERT_TRUE(dv); + ASSERT_FALSE(dv.value()->IsDeleted(0).value()); + ASSERT_TRUE(dv.value()->IsDeleted(2).value()); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestRewriteWithDvAndAgg) { + std::map options = { + {Options::MERGE_ENGINE, "aggregation"}, + {Options::FIELDS_DEFAULT_AGG_FUNC, "sum"}, + {Options::FILE_FORMAT, "orc"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 3 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/5, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 55]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/0, /*last_sequence_number=*/2, table_path, + core_options, "[[2, 22], [4, 44], [5, 5]]")); + ASSERT_OK_AND_ASSIGN(auto file2, NewFiles(/*level=*/0, /*last_sequence_number=*/5, table_path, + core_options, "[[2, 222], [5, 555]]")); + std::vector> files = {file0, file1, file2}; + auto processor_factory = std::make_shared(arrow_schema_); + ASSERT_OK_AND_ASSIGN( + auto lookup_levels, + CreateLookupLevels(table_path, table_schema, processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN(auto rewriter, + CreateCompactRewriterForPositionedKeyValue( + table_path, table_schema, core_options, std::move(lookup_levels))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns({file1, file2})); + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Rewrite( + /*output_level=*/4, /*drop_delete=*/true, runs)); + ASSERT_EQ(2, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + const auto& compact_file_meta = compact_result.After()[0]; + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = + arrow::struct_(SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[6, 0, 2, 244], +[4, 0, 4, 44], +[7, 0, 5, 615] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "orc", expected_array); + + // test dv + auto dv_maintainer = rewriter->dv_maintainer_; + ASSERT_TRUE(dv_maintainer); + auto dv = dv_maintainer->DeletionVectorOf(file0->file_name); + ASSERT_TRUE(dv); + ASSERT_FALSE(dv.value()->IsDeleted(0).value()); + ASSERT_TRUE(dv.value()->IsDeleted(2).value()); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestDvUpgradeWithLookup) { + std::map options = { + {Options::MERGE_ENGINE, "deduplicate"}, + {Options::FILE_FORMAT, "orc"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 2 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/5, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 55]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/0, /*last_sequence_number=*/2, table_path, + core_options, "[[2, 22], [4, 44], [5, 5]]")); + std::vector> files = {file0, file1}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN( + auto lookup_levels, + CreateLookupLevels(table_path, table_schema, processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN( + auto rewriter, CreateCompactRewriterForFilePosition(table_path, table_schema, core_options, + std::move(lookup_levels))); + // goto ChangelogNoRewrite UpgradeStrategy + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Upgrade( + /*output_level=*/4, file1)); + ASSERT_EQ(1, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + const auto& compact_file_meta = compact_result.After()[0]; + ASSERT_EQ(compact_file_meta->file_name, file1->file_name); + ASSERT_EQ(compact_file_meta->level, 4); + + // test dv + auto dv_maintainer = rewriter->dv_maintainer_; + ASSERT_TRUE(dv_maintainer); + auto dv = dv_maintainer->DeletionVectorOf(file0->file_name); + ASSERT_TRUE(dv); + ASSERT_FALSE(dv.value()->IsDeleted(0).value()); + ASSERT_TRUE(dv.value()->IsDeleted(2).value()); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestDvUpgradeWithoutLookup) { + std::map options = { + {Options::MERGE_ENGINE, "deduplicate"}, + {Options::FILE_FORMAT, "orc"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 2 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/5, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 55]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/1, /*last_sequence_number=*/2, table_path, + core_options, "[[2, 22], [4, 44], [5, 5]]")); + std::vector> files = {file0, file1}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN( + auto lookup_levels, + CreateLookupLevels(table_path, table_schema, processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN( + auto rewriter, CreateCompactRewriterForFilePosition(table_path, table_schema, core_options, + std::move(lookup_levels))); + // goto NoChangelogNoRewrite UpgradeStrategy + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Upgrade( + /*output_level=*/4, file1)); + ASSERT_EQ(1, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + const auto& compact_file_meta = compact_result.After()[0]; + ASSERT_EQ(compact_file_meta->file_name, file1->file_name); + ASSERT_EQ(compact_file_meta->level, 4); + + // test dv + auto dv_maintainer = rewriter->dv_maintainer_; + ASSERT_TRUE(dv_maintainer); + auto dv = dv_maintainer->DeletionVectorOf(file0->file_name); + ASSERT_FALSE(dv); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestIOException) { + std::map options = { + {Options::MERGE_ENGINE, "aggregation"}, + {Options::FIELDS_DEFAULT_AGG_FUNC, "sum"}, + {Options::FILE_FORMAT, "orc"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + + bool run_complete = false; + auto io_hook = IOHook::GetInstance(); + for (size_t i = 0; i < 300; i++) { + tmp_dir_ = UniqueTestDirectory::Create("local"); + dir_ = UniqueTestDirectory::Create("local"); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 3 files + ASSERT_OK_AND_ASSIGN( + auto file0, NewFiles(/*level=*/5, /*last_sequence_number=*/-1, table_path, core_options, + "[[1, 11], [3, 33], [5, 55]]")); + ASSERT_OK_AND_ASSIGN( + auto file1, NewFiles(/*level=*/0, /*last_sequence_number=*/2, table_path, core_options, + "[[2, 22], [4, 44], [5, 5]]")); + ASSERT_OK_AND_ASSIGN( + auto file2, NewFiles(/*level=*/0, /*last_sequence_number=*/5, table_path, core_options, + "[[2, 222], [5, 555]]")); + std::vector> files = {file0, file1, file2}; + auto processor_factory = + std::make_shared(arrow_schema_); + ASSERT_OK_AND_ASSIGN(auto lookup_levels, + CreateLookupLevels(table_path, table_schema, + processor_factory, files)); + + // compact and rewrite + ASSERT_OK_AND_ASSIGN(auto rewriter, + CreateCompactRewriterForPositionedKeyValue( + table_path, table_schema, core_options, std::move(lookup_levels))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns({file1, file2})); + + // rewrite may trigger I/O exception + ScopeGuard guard([&io_hook]() { io_hook->Clear(); }); + io_hook->Reset(i, IOHook::Mode::RETURN_ERROR); + auto compact_result = rewriter->Rewrite( + /*output_level=*/4, /*drop_delete=*/true, runs); + CHECK_HOOK_STATUS_WITHOUT_MESSAGE_CHECK(compact_result.status()); + io_hook->Clear(); + + ASSERT_EQ(2, compact_result.value().Before().size()); + ASSERT_EQ(1, compact_result.value().After().size()); + + const auto& compact_file_meta = compact_result.value().After()[0]; + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = arrow::struct_( + SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[6, 0, 2, 244], +[4, 0, 4, 44], +[7, 0, 5, 615] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "orc", expected_array); + + // test dv + auto dv_maintainer = rewriter->dv_maintainer_; + ASSERT_TRUE(dv_maintainer); + auto dv = dv_maintainer->DeletionVectorOf(file0->file_name); + ASSERT_TRUE(dv); + ASSERT_FALSE(dv.value()->IsDeleted(0).value()); + ASSERT_TRUE(dv.value()->IsDeleted(2).value()); + run_complete = true; + break; + } + ASSERT_TRUE(run_complete); +} + +TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { + auto create_meta = [this](int32_t level, std::optional delete_row_count) { + return std::make_shared( + "file.orc", 100l, /*row_count=*/4, + /*min_key=*/BinaryRowGenerator::GenerateRow({1}, pool_.get()), + /*max_key=*/BinaryRowGenerator::GenerateRow({5}, pool_.get()), + /*key_stats=*/ + BinaryRowGenerator::GenerateStats({1}, {5}, {0}, pool_.get()), + /*value_stats=*/ + BinaryRowGenerator::GenerateStats({1, 5}, {5, 33}, {0, 0}, pool_.get()), + /*min_sequence_number=*/0l, /*max_sequence_number=*/3l, /*schema_id=*/0, level, + std::vector>(), Timestamp(0l, 0), delete_row_count, nullptr, + FileSource::Compact(), std::nullopt, std::nullopt, std::nullopt, std::nullopt); + }; + { + std::map options = {}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + + LookupMergeTreeCompactRewriter rewriter( + /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, + /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, + /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, + /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + auto file = create_meta(/*level=*/1, /*delete_row_count=*/std::nullopt); + ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::NoChangelogNoRewrite(), + rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); + } + { + std::map options = { + {Options::FILE_FORMAT, "orc"}, {Options::FILE_FORMAT_PER_LEVEL, "5:parquet"}}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + + LookupMergeTreeCompactRewriter rewriter( + /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, + /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, + /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, + /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); + ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), + rewriter.GenerateUpgradeStrategy(/*output_level=*/5, file)); + } + { + std::map options = {}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + + auto dv_maintainer = std::make_shared( + std::make_shared(/*fs=*/nullptr, /*path_factory=*/nullptr, + /*bitmap64=*/false, pool_), + std::map>()); + LookupMergeTreeCompactRewriter rewriter( + /*lookup_levels=*/nullptr, dv_maintainer, + /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, + /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, + /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + auto file = create_meta(/*level=*/0, /*delete_row_count=*/1); + ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), + rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); + } + { + std::map options = {}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + + LookupMergeTreeCompactRewriter rewriter( + /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, + /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, + /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, + /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); + ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogNoRewrite(), + rewriter.GenerateUpgradeStrategy(/*output_level=*/5, file)); + } + { + std::map options = {}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + + LookupMergeTreeCompactRewriter rewriter( + /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, + /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, + /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, + /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); + ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogNoRewrite(), + rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); + } + { + std::map options = {{Options::MERGE_ENGINE, "aggregation"}, + {Options::FIELDS_DEFAULT_AGG_FUNC, "sum"}}; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + + LookupMergeTreeCompactRewriter rewriter( + /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, + /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, + /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, + /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); + ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), + rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); + } +} +} // namespace paimon::test diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp index 3411d04f..ae239362 100644 --- a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp @@ -31,34 +31,37 @@ #include "paimon/read_context.h" namespace paimon { MergeTreeCompactRewriter::MergeTreeCompactRewriter( - const BinaryRow& partition, int64_t schema_id, + const BinaryRow& partition, int32_t bucket, int64_t schema_id, const std::vector& trimmed_primary_keys, const CoreOptions& options, const std::shared_ptr& data_schema, const std::shared_ptr& write_schema, - const std::shared_ptr& data_file_path_factory, + const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, + MergeFunctionWrapperFactory merge_function_wrapper_factory, const std::shared_ptr& pool) - : pool_(pool), + : options_(options), + merge_file_split_read_(std::move(merge_file_split_read)), + pool_(pool), partition_(partition), + bucket_(bucket), schema_id_(schema_id), trimmed_primary_keys_(trimmed_primary_keys), - options_(options), data_schema_(data_schema), write_schema_(write_schema), - data_file_path_factory_(data_file_path_factory), - merge_file_split_read_(std::move(merge_file_split_read)) {} + path_factory_cache_(path_factory_cache), + merge_function_wrapper_factory_(std::move(merge_function_wrapper_factory)) {} Result> MergeTreeCompactRewriter::Create( int32_t bucket, const BinaryRow& partition, const std::shared_ptr& table_schema, - const std::shared_ptr& path_factory, const CoreOptions& options, - const std::shared_ptr& pool) { + const std::shared_ptr& path_factory_cache, + const CoreOptions& options, const std::shared_ptr& pool) { PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, table_schema->TrimmedPrimaryKeys()); auto data_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); auto write_schema = SpecialFields::CompleteSequenceAndValueKindField(data_schema); // TODO(xinyu.lxy): set executor - ReadContextBuilder read_context_builder(path_factory->RootPath()); + ReadContextBuilder read_context_builder(path_factory_cache->RootPath()); read_context_builder.SetOptions(options.ToMap()).EnablePrefetch(true).WithMemoryPool(pool); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr read_context, read_context_builder.Finish()); @@ -66,20 +69,24 @@ Result> MergeTreeCompactRewriter::Crea PAIMON_ASSIGN_OR_RAISE( std::shared_ptr internal_context, InternalReadContext::Create(read_context, table_schema, options.ToMap())); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr path_factory, + path_factory_cache->GetOrCreatePathFactory( + options.GetWriteFileFormat(/*level=*/0)->Identifier())); PAIMON_ASSIGN_OR_RAISE( std::unique_ptr merge_file_split_read, MergeFileSplitRead::Create(path_factory, internal_context, pool, CreateDefaultExecutor())); - - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr data_file_path_factory, - path_factory->CreateDataFilePathFactory(partition, bucket)); - + auto merge_function_wrapper_factory = + [](int32_t output_level) -> Result>> { + return std::shared_ptr>(); + }; return std::unique_ptr(new MergeTreeCompactRewriter( - partition, table_schema->Id(), trimmed_primary_keys, options, data_schema, write_schema, - data_file_path_factory, std::move(merge_file_split_read), pool)); + partition, bucket, table_schema->Id(), trimmed_primary_keys, options, data_schema, + write_schema, path_factory_cache, std::move(merge_file_split_read), + merge_function_wrapper_factory, pool)); } -Result MergeTreeCompactRewriter::Upgrade( - int32_t output_level, const std::shared_ptr& file) const { +Result MergeTreeCompactRewriter::Upgrade(int32_t output_level, + const std::shared_ptr& file) { PAIMON_ASSIGN_OR_RAISE(auto upgraded_file, file->Upgrade(output_level)); return CompactResult({file}, {upgraded_file}); } @@ -102,13 +109,13 @@ std::vector> MergeTreeCompactRewriter::ExtractFile } std::unique_ptr -MergeTreeCompactRewriter::CreateRollingRowWriter(int32_t level) const { +MergeTreeCompactRewriter::CreateRollingRowWriter(int32_t level) { auto create_file_writer = [this, level]() -> Result>>> { ::ArrowSchema arrow_schema{}; ScopeGuard guard([&arrow_schema]() { ArrowSchemaRelease(&arrow_schema); }); PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*write_schema_, &arrow_schema)); - auto format = options_.GetWriteFileFormat(); + auto format = options_.GetWriteFileFormat(level); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr writer_builder, format->CreateWriterBuilder(&arrow_schema, options_.GetWriteBatchSize())); @@ -120,12 +127,15 @@ MergeTreeCompactRewriter::CreateRollingRowWriter(int32_t level) const { ArrowArrayMove(key_value_batch.batch.get(), array); return Status::OK(); }; + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr data_file_path_factory, + CreateDataFilePathFactory(format->Identifier())); + auto writer = std::make_unique( options_.GetFileCompression(), converter, schema_id_, level, FileSource::Compact(), trimmed_primary_keys_, stats_extractor, write_schema_, - data_file_path_factory_->IsExternalPath(), pool_); + data_file_path_factory->IsExternalPath(), pool_); PAIMON_RETURN_NOT_OK(writer->Init(options_.GetFileSystem(), - data_file_path_factory_->NewPath(), writer_builder)); + data_file_path_factory->NewPath(), writer_builder)); return writer; }; return std::make_unique( @@ -134,6 +144,10 @@ MergeTreeCompactRewriter::CreateRollingRowWriter(int32_t level) const { Result MergeTreeCompactRewriter::GenerateKeyValueConsumer() const { + if (!merge_file_split_read_) { + return Status::Invalid( + "merge_file_split_read in MergeTreeCompactRewriter cannot be nullptr"); + } PAIMON_ASSIGN_OR_RAISE(std::vector target_to_src_mapping, ArrowUtils::CreateProjection( /*src_schema=*/merge_file_split_read_->GetValueSchema(), @@ -147,25 +161,42 @@ MergeTreeCompactRewriter::GenerateKeyValueConsumer() const { }); } +Result> MergeTreeCompactRewriter::CreateDataFilePathFactory( + const std::string& format) { + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr path_factory, + path_factory_cache_->GetOrCreatePathFactory(format)); + return path_factory->CreateDataFilePathFactory(partition_, bucket_); +} + Status MergeTreeCompactRewriter::MergeReadAndWrite( - bool drop_delete, const std::vector& section, + int32_t output_level, bool drop_delete, const std::vector& section, const MergeTreeCompactRewriter::KeyValueConsumerCreator& create_consumer, MergeTreeCompactRewriter::KeyValueRollingFileWriter* rolling_writer, - std::vector>* + std::vector>* reader_holders_ptr) { + if (!merge_file_split_read_) { + return Status::Invalid( + "merge_file_split_read in MergeTreeCompactRewriter cannot be nullptr"); + } auto& reader_holders = *reader_holders_ptr; - // prepare loser tree sort merge reader + // prepare sort merge reader + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr data_file_path_factory, + CreateDataFilePathFactory(options_.GetWriteFileFormat(/*level=*/0)->Identifier())); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr> wrapper, + merge_function_wrapper_factory_(output_level)); + merge_file_split_read_->SetMergeFunctionWrapper(wrapper); PAIMON_ASSIGN_OR_RAISE(std::unique_ptr sort_merge_reader, merge_file_split_read_->CreateSortMergeReaderForSection( section, partition_, /*deletion_file_map=*/{}, - /*predicate=*/nullptr, data_file_path_factory_, drop_delete)); + /*predicate=*/nullptr, data_file_path_factory, drop_delete)); // consumer batch size is WriteBatchSize auto async_key_value_producer_consumer = - std::make_unique>( + std::make_shared>( std::move(sort_merge_reader), create_consumer, options_.GetWriteBatchSize(), /*projection_thread_num=*/1, pool_); - + reader_holders.push_back(async_key_value_producer_consumer); // read KeyValueBatch from SortMergeReader and write to RollingWriter while (true) { PAIMON_ASSIGN_OR_RAISE(KeyValueBatch key_value_batch, @@ -173,9 +204,10 @@ Status MergeTreeCompactRewriter::MergeReadAndWrite( if (key_value_batch.batch == nullptr) { break; } - PAIMON_RETURN_NOT_OK(rolling_writer->Write(std::move(key_value_batch))); + if (rolling_writer) { + PAIMON_RETURN_NOT_OK(rolling_writer->Write(std::move(key_value_batch))); + } } - reader_holders.push_back(std::move(async_key_value_producer_consumer)); return Status::OK(); } @@ -184,18 +216,19 @@ Result MergeTreeCompactRewriter::RewriteCompaction( PAIMON_ASSIGN_OR_RAISE(MergeTreeCompactRewriter::KeyValueConsumerCreator create_consumer, GenerateKeyValueConsumer()); - std::vector> reader_holders; + std::vector> reader_holders; auto rolling_writer = CreateRollingRowWriter(output_level); ScopeGuard write_guard([&]() -> void { rolling_writer->Abort(); + merge_file_split_read_.reset(); for (const auto& reader : reader_holders) { reader->Close(); } }); for (const auto& section : sections) { - PAIMON_RETURN_NOT_OK(MergeReadAndWrite(drop_delete, section, create_consumer, + PAIMON_RETURN_NOT_OK(MergeReadAndWrite(output_level, drop_delete, section, create_consumer, rolling_writer.get(), &reader_holders)); } diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.h b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.h index cd44c7c9..08bb8e15 100644 --- a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.h +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.h @@ -15,7 +15,6 @@ */ #pragma once - #include "arrow/api.h" #include "paimon/core/core_options.h" #include "paimon/core/io/async_key_value_producer_and_consumer.h" @@ -27,21 +26,25 @@ #include "paimon/core/operation/merge_file_split_read.h" #include "paimon/core/schema/table_schema.h" #include "paimon/core/utils/file_store_path_factory.h" +#include "paimon/core/utils/file_store_path_factory_cache.h" namespace paimon { /// Default `CompactRewriter` for merge trees. class MergeTreeCompactRewriter : public CompactRewriter { public: + using MergeFunctionWrapperFactory = + std::function>>(int32_t)>; + static Result> Create( int32_t bucket, const BinaryRow& partition, const std::shared_ptr& table_schema, - const std::shared_ptr& path_factory, const CoreOptions& options, - const std::shared_ptr& memory_pool); + const std::shared_ptr& path_factory_cache, + const CoreOptions& options, const std::shared_ptr& memory_pool); Result Rewrite(int32_t output_level, bool drop_delete, const std::vector>& sections) override; Result Upgrade(int32_t output_level, - const std::shared_ptr& file) const override; + const std::shared_ptr& file) override; Status Close() override { return Status::OK(); @@ -62,43 +65,52 @@ class MergeTreeCompactRewriter : public CompactRewriter { static std::vector> ExtractFilesFromSections( const std::vector>& sections); - private: - using KeyValueRollingFileWriter = - RollingFileWriter>; - using KeyValueMergeReader = AsyncKeyValueProducerAndConsumer; - using KeyValueConsumerCreator = - AsyncKeyValueProducerAndConsumer::ConsumerCreator; - - MergeTreeCompactRewriter(const BinaryRow& partition, int64_t schema_id, + MergeTreeCompactRewriter(const BinaryRow& partition, int32_t bucket, int64_t schema_id, const std::vector& trimmed_primary_keys, const CoreOptions& options, const std::shared_ptr& data_schema, const std::shared_ptr& write_schema, - const std::shared_ptr& data_file_path_factory, + const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, + MergeFunctionWrapperFactory merge_function_wrapper_factory, const std::shared_ptr& pool); - std::unique_ptr CreateRollingRowWriter(int32_t level) const; + using KeyValueRollingFileWriter = + RollingFileWriter>; + using KeyValueMergeReader = AsyncKeyValueProducerAndConsumer; + using KeyValueConsumerCreator = + AsyncKeyValueProducerAndConsumer::ConsumerCreator; + + std::unique_ptr CreateRollingRowWriter(int32_t level); Result GenerateKeyValueConsumer() const; - Status MergeReadAndWrite(bool drop_delete, const std::vector& section, + Status MergeReadAndWrite(int32_t output_level, bool drop_delete, + const std::vector& section, const KeyValueConsumerCreator& create_consumer, KeyValueRollingFileWriter* rolling_writer, - std::vector>* reader_holders_ptr); + std::vector>* reader_holders_ptr); + + private: + Result> CreateDataFilePathFactory( + const std::string& format); + + protected: + CoreOptions options_; + std::unique_ptr merge_file_split_read_; private: std::shared_ptr pool_; BinaryRow partition_; + int32_t bucket_; int64_t schema_id_; std::vector trimmed_primary_keys_; - CoreOptions options_; // all data fields in table schema std::shared_ptr data_schema_; // SequenceNumber + ValueKind + data_schema_ std::shared_ptr write_schema_; - std::shared_ptr data_file_path_factory_; - std::unique_ptr merge_file_split_read_; + std::shared_ptr path_factory_cache_; + MergeFunctionWrapperFactory merge_function_wrapper_factory_; }; } // namespace paimon diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter_test.cpp b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter_test.cpp index f4e439d5..082907e7 100644 --- a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter_test.cpp +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter_test.cpp @@ -40,20 +40,9 @@ class MergeTreeCompactRewriterTest : public testing::Test { int32_t bucket, const BinaryRow& partition) const { PAIMON_ASSIGN_OR_RAISE(auto options, CoreOptions::FromMap(table_schema->Options())); auto arrow_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); - - PAIMON_ASSIGN_OR_RAISE(std::vector external_paths, - options.CreateExternalPaths()); - PAIMON_ASSIGN_OR_RAISE(std::optional global_index_external_path, - options.CreateGlobalIndexExternalPath()); - PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr path_factory, - FileStorePathFactory::Create( - table_path, arrow_schema, table_schema->PartitionKeys(), - options.GetPartitionDefaultName(), options.GetWriteFileFormat()->Identifier(), - options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, options.IndexFileInDataFileDir(), pool_)); - - return MergeTreeCompactRewriter::Create(bucket, partition, table_schema, path_factory, + auto path_factory_cache = + std::make_shared(table_path, table_schema, options, pool_); + return MergeTreeCompactRewriter::Create(bucket, partition, table_schema, path_factory_cache, options, pool_); } diff --git a/src/paimon/core/mergetree/compact/reducer_merge_function_wrapper.h b/src/paimon/core/mergetree/compact/reducer_merge_function_wrapper.h index c7f60c99..ce48b4b3 100644 --- a/src/paimon/core/mergetree/compact/reducer_merge_function_wrapper.h +++ b/src/paimon/core/mergetree/compact/reducer_merge_function_wrapper.h @@ -62,7 +62,14 @@ class ReducerMergeFunctionWrapper : public MergeFunctionWrapper { /// Get current value of the `MergeFunction` helper. Result> GetResult() override { - return is_initialized_ ? merge_function_->GetResult() : std::move(initial_kv_); + std::optional result; + if (is_initialized_) { + PAIMON_ASSIGN_OR_RAISE(result, merge_function_->GetResult()); + } else { + result = std::move(initial_kv_); + } + Reset(); + return result; } private: diff --git a/src/paimon/core/mergetree/lookup_levels.h b/src/paimon/core/mergetree/lookup_levels.h index ed18432a..b59ad6c6 100644 --- a/src/paimon/core/mergetree/lookup_levels.h +++ b/src/paimon/core/mergetree/lookup_levels.h @@ -56,6 +56,12 @@ class LookupLevels { Result> Lookup(const std::shared_ptr& key, const SortedRun& level); + Status Close() { + // TODDO(xinyu.lxy): invalid cache + lookup_file_cache_.clear(); + return Status::OK(); + } + private: LookupLevels(const std::shared_ptr& fs, const BinaryRow& partition, int32_t bucket, const CoreOptions& options, const std::shared_ptr& schema_manager, diff --git a/src/paimon/core/mergetree/lookup_levels_test.cpp b/src/paimon/core/mergetree/lookup_levels_test.cpp index 1242b43f..45da7f8d 100644 --- a/src/paimon/core/mergetree/lookup_levels_test.cpp +++ b/src/paimon/core/mergetree/lookup_levels_test.cpp @@ -128,7 +128,7 @@ class LookupLevelsTest : public testing::Test { std::shared_ptr path_factory, FileStorePathFactory::Create( table_path, arrow_schema_, /*partition_keys=*/{}, options.GetPartitionDefaultName(), - options.GetWriteFileFormat()->Identifier(), options.DataFilePrefix(), + options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), pool_)); return path_factory; @@ -141,8 +141,6 @@ class LookupLevelsTest : public testing::Test { PAIMON_ASSIGN_OR_RAISE(CoreOptions options, CoreOptions::FromMap(table_schema->Options())); auto io_manager = IOManager::Create(tmp_dir_->Str()); - - auto arrow_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); auto processor_factory = std::make_shared(arrow_schema_); auto serializer_factory = std::make_shared(); @@ -217,6 +215,16 @@ TEST_F(LookupLevelsTest, TestMultiLevels) { ASSERT_EQ(lookup_levels->lookup_file_cache_.size(), 2); ASSERT_EQ(lookup_levels->schema_id_and_ser_version_to_processors_.size(), 1); + + // test lookup file in tmp dir + std::vector> file_status_list; + ASSERT_OK(fs_->ListDir(tmp_dir_->Str(), &file_status_list)); + ASSERT_EQ(file_status_list.size(), 2); + // test close will rm local lookup file + ASSERT_OK(lookup_levels->Close()); + file_status_list.clear(); + ASSERT_OK(fs_->ListDir(tmp_dir_->Str(), &file_status_list)); + ASSERT_TRUE(file_status_list.empty()); // TODO(lisizhuo.lsz): test lookuplevels close } diff --git a/src/paimon/core/mergetree/merge_tree_writer.cpp b/src/paimon/core/mergetree/merge_tree_writer.cpp index dba6f17e..dd14278e 100644 --- a/src/paimon/core/mergetree/merge_tree_writer.cpp +++ b/src/paimon/core/mergetree/merge_tree_writer.cpp @@ -181,7 +181,7 @@ MergeTreeWriter::CreateRollingRowWriter() const { ::ArrowSchema arrow_schema; ScopeGuard guard([&arrow_schema]() { ArrowSchemaRelease(&arrow_schema); }); PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*write_schema_, &arrow_schema)); - auto format = options_.GetWriteFileFormat(); + auto format = options_.GetWriteFileFormat(/*level=*/0); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr writer_builder, format->CreateWriterBuilder(&arrow_schema, options_.GetWriteBatchSize())); diff --git a/src/paimon/core/migrate/file_meta_utils.cpp b/src/paimon/core/migrate/file_meta_utils.cpp index 7a12c7e0..d9cfe9f5 100644 --- a/src/paimon/core/migrate/file_meta_utils.cpp +++ b/src/paimon/core/migrate/file_meta_utils.cpp @@ -124,7 +124,7 @@ Result> FileMetaUtils::GenerateCommitMessage( CoreOptions::FromMap(table_options, file_system)); std::shared_ptr fs = core_options.GetFileSystem(); - std::shared_ptr format = core_options.GetWriteFileFormat(); + std::shared_ptr format = core_options.GetWriteFileFormat(/*level=*/0); assert(fs); assert(format); PAIMON_RETURN_NOT_OK(ValidateNonObjectPath(src_data_files)); diff --git a/src/paimon/core/operation/append_only_file_store_write.cpp b/src/paimon/core/operation/append_only_file_store_write.cpp index 15c1395e..9f6f2606 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -223,7 +223,7 @@ AppendOnlyFileStoreWrite::GetDataFileWriterCreator( ::ArrowSchema arrow_schema; ScopeGuard guard([&arrow_schema]() { ArrowSchemaRelease(&arrow_schema); }); PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*schema, &arrow_schema)); - auto format = options_.GetWriteFileFormat(); + auto format = options_.GetWriteFileFormat(/*level=*/0); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr writer_builder, format->CreateWriterBuilder(&arrow_schema, options_.GetWriteBatchSize())); diff --git a/src/paimon/core/operation/expire_snapshots_test.cpp b/src/paimon/core/operation/expire_snapshots_test.cpp index 86744143..0e3fe3a0 100644 --- a/src/paimon/core/operation/expire_snapshots_test.cpp +++ b/src/paimon/core/operation/expire_snapshots_test.cpp @@ -121,7 +121,7 @@ class ExpireSnapshotsTest : public testing::Test { auto path_factory, FileStorePathFactory::Create( root, schema_, partition_keys_, options.GetPartitionDefaultName(), - options.GetWriteFileFormat()->Identifier(), options.DataFilePrefix(), + options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), mem_pool_)); return path_factory; diff --git a/src/paimon/core/operation/file_store_commit.cpp b/src/paimon/core/operation/file_store_commit.cpp index 5630776c..bd5939ba 100644 --- a/src/paimon/core/operation/file_store_commit.cpp +++ b/src/paimon/core/operation/file_store_commit.cpp @@ -79,7 +79,7 @@ Result> FileStoreCommit::Create( PAIMON_ASSIGN_OR_RAISE(CoreOptions options, CoreOptions::FromMap(opts, ctx->GetSpecificFileSystem())); assert(options.GetFileSystem()); - assert(options.GetWriteFileFormat()); + assert(options.GetWriteFileFormat(/*level=*/0)); PAIMON_ASSIGN_OR_RAISE(bool is_object_store, FileSystem::IsObjectStore(root_path)); if (is_object_store && opts.find("enable-object-store-commit-in-inte-test") == opts.end()) { return Status::NotImplemented( @@ -96,11 +96,12 @@ Result> FileStoreCommit::Create( PAIMON_ASSIGN_OR_RAISE( std::shared_ptr path_factory, - FileStorePathFactory::Create( - root_path, arrow_schema, table_schema.value()->PartitionKeys(), - options.GetPartitionDefaultName(), options.GetWriteFileFormat()->Identifier(), - options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); + FileStorePathFactory::Create(root_path, arrow_schema, table_schema.value()->PartitionKeys(), + options.GetPartitionDefaultName(), + options.GetWriteFileFormat(/*level=*/0)->Identifier(), + options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), + external_paths, global_index_external_path, + options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); auto snapshot_manager = std::make_shared(options.GetFileSystem(), root_path); PAIMON_ASSIGN_OR_RAISE( diff --git a/src/paimon/core/operation/file_store_write.cpp b/src/paimon/core/operation/file_store_write.cpp index 86c7a497..9d7a4d4d 100644 --- a/src/paimon/core/operation/file_store_write.cpp +++ b/src/paimon/core/operation/file_store_write.cpp @@ -92,11 +92,12 @@ Result> FileStoreWrite::Create(std::unique_ptr file_store_path_factory, - FileStorePathFactory::Create( - ctx->GetRootPath(), arrow_schema, schema->PartitionKeys(), - options.GetPartitionDefaultName(), options.GetWriteFileFormat()->Identifier(), - options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); + FileStorePathFactory::Create(ctx->GetRootPath(), arrow_schema, schema->PartitionKeys(), + options.GetPartitionDefaultName(), + options.GetWriteFileFormat(/*level=*/0)->Identifier(), + options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), + external_paths, global_index_external_path, + options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); auto snapshot_manager = std::make_shared(options.GetFileSystem(), ctx->GetRootPath(), branch); bool ignore_previous_files = ctx->IgnorePreviousFiles(); diff --git a/src/paimon/core/operation/key_value_file_store_scan_test.cpp b/src/paimon/core/operation/key_value_file_store_scan_test.cpp index 766c56e5..8a6a116f 100644 --- a/src/paimon/core/operation/key_value_file_store_scan_test.cpp +++ b/src/paimon/core/operation/key_value_file_store_scan_test.cpp @@ -83,12 +83,13 @@ class KeyValueFileStoreScanTest : public testing::Test { PAIMON_ASSIGN_OR_RAISE( std::shared_ptr path_factory, - FileStorePathFactory::Create( - table_path, arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat()->Identifier(), core_options.DataFilePrefix(), - core_options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); + FileStorePathFactory::Create(table_path, arrow_schema, table_schema->PartitionKeys(), + core_options.GetPartitionDefaultName(), + core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.DataFilePrefix(), + core_options.LegacyPartitionNameEnabled(), external_paths, + global_index_external_path, + core_options.IndexFileInDataFileDir(), pool_)); auto manifest_file_format = core_options.GetManifestFormat(); auto snapshot_manager = std::make_shared(fs, table_path); diff --git a/src/paimon/core/operation/manifest_file_merger_test.cpp b/src/paimon/core/operation/manifest_file_merger_test.cpp index c76ce70b..1504a4c6 100644 --- a/src/paimon/core/operation/manifest_file_merger_test.cpp +++ b/src/paimon/core/operation/manifest_file_merger_test.cpp @@ -125,7 +125,7 @@ class ManifestFileMergerTest : public testing::Test { static std::shared_ptr path_factory, FileStorePathFactory::Create( path_str, schema, /*partition_keys=*/{"f0"}, options.GetPartitionDefaultName(), - options.GetWriteFileFormat()->Identifier(), options.DataFilePrefix(), + options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), pool_)); ASSERT_OK_AND_ASSIGN(std::shared_ptr partition_schema, diff --git a/src/paimon/core/operation/merge_file_split_read.cpp b/src/paimon/core/operation/merge_file_split_read.cpp index 7154a2b0..1bf431eb 100644 --- a/src/paimon/core/operation/merge_file_split_read.cpp +++ b/src/paimon/core/operation/merge_file_split_read.cpp @@ -144,6 +144,11 @@ Result> MergeFileSplitRead::CreateReader( return std::make_unique(std::move(batch_reader), pool_); } +void MergeFileSplitRead::SetMergeFunctionWrapper( + const std::shared_ptr>& merge_function_wrapper) { + merge_function_wrapper_ = merge_function_wrapper; +} + Result>> MergeFileSplitRead::GetMergeFunctionWrapper() { if (!merge_function_wrapper_) { diff --git a/src/paimon/core/operation/merge_file_split_read.h b/src/paimon/core/operation/merge_file_split_read.h index c2a79fc0..5b380c16 100644 --- a/src/paimon/core/operation/merge_file_split_read.h +++ b/src/paimon/core/operation/merge_file_split_read.h @@ -105,6 +105,9 @@ class MergeFileSplitRead : public AbstractSplitRead { return value_schema_; } + void SetMergeFunctionWrapper( + const std::shared_ptr>& merge_function_wrapper); + private: Result> CreateMergeReader( const std::shared_ptr& data_split, @@ -127,6 +130,8 @@ class MergeFileSplitRead : public AbstractSplitRead { Result> CreateSortMergeReader( std::vector>&& record_readers); + Result>> GetMergeFunctionWrapper(); + MergeFileSplitRead(const std::shared_ptr& path_factory, const std::shared_ptr& context, std::unique_ptr&& schema_manager, int32_t key_arity, @@ -140,9 +145,6 @@ class MergeFileSplitRead : public AbstractSplitRead { const std::shared_ptr& memory_pool, const std::shared_ptr& executor); - private: - Result>> GetMergeFunctionWrapper(); - static Result>> CreateMergeFunctionWrapper( const CoreOptions& core_options, const std::shared_ptr& table_schema, const std::shared_ptr& value_schema); diff --git a/src/paimon/core/operation/merge_file_split_read_test.cpp b/src/paimon/core/operation/merge_file_split_read_test.cpp index 09ed81a7..fe0ba4df 100644 --- a/src/paimon/core/operation/merge_file_split_read_test.cpp +++ b/src/paimon/core/operation/merge_file_split_read_test.cpp @@ -341,9 +341,10 @@ class MergeFileSplitReadTest : public ::testing::Test, FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat()->Identifier(), core_options.DataFilePrefix(), - core_options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); + core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), + external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), + pool_)); PAIMON_ASSIGN_OR_RAISE(auto split_read, MergeFileSplitRead::Create(path_factory, std::move(internal_context), pool_, executor_)); diff --git a/src/paimon/core/operation/orphan_files_cleaner.cpp b/src/paimon/core/operation/orphan_files_cleaner.cpp index e6d61a79..2131ff5c 100644 --- a/src/paimon/core/operation/orphan_files_cleaner.cpp +++ b/src/paimon/core/operation/orphan_files_cleaner.cpp @@ -185,11 +185,12 @@ Result> OrphanFilesCleaner::Create( PAIMON_ASSIGN_OR_RAISE( std::shared_ptr path_factory, - FileStorePathFactory::Create( - ctx->GetRootPath(), arrow_schema, schema->PartitionKeys(), - options.GetPartitionDefaultName(), options.GetWriteFileFormat()->Identifier(), - options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); + FileStorePathFactory::Create(ctx->GetRootPath(), arrow_schema, schema->PartitionKeys(), + options.GetPartitionDefaultName(), + options.GetWriteFileFormat(/*level=*/0)->Identifier(), + options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), + external_paths, global_index_external_path, + options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); auto snapshot_manager = std::make_shared(options.GetFileSystem(), ctx->GetRootPath()); PAIMON_ASSIGN_OR_RAISE( diff --git a/src/paimon/core/operation/raw_file_split_read_test.cpp b/src/paimon/core/operation/raw_file_split_read_test.cpp index 6131347f..b4a97f0b 100644 --- a/src/paimon/core/operation/raw_file_split_read_test.cpp +++ b/src/paimon/core/operation/raw_file_split_read_test.cpp @@ -151,9 +151,10 @@ class RawFileSplitReadTest : public ::testing::Test { FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat()->Identifier(), core_options.DataFilePrefix(), - core_options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); + core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), + external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), + pool_)); auto split_read = std::make_unique(path_factory, std::move(internal_context), pool_, CreateDefaultExecutor(/*thread_count=*/2)); @@ -394,7 +395,8 @@ TEST_F(RawFileSplitReadTest, TestEmptyPlan) { std::shared_ptr path_factory, FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), core_options.GetWriteFileFormat()->Identifier(), + core_options.GetPartitionDefaultName(), + core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); @@ -495,9 +497,7 @@ TEST_F(RawFileSplitReadTest, TestMatch) { split_read->Match(data_split, /*force_keep_delete=*/false)); ASSERT_FALSE(match_result); } - { - ASSERT_NOK(split_read->Match(nullptr, /*force_keep_delete=*/false)); - } + { ASSERT_NOK(split_read->Match(nullptr, /*force_keep_delete=*/false)); } } } // namespace paimon::test diff --git a/src/paimon/core/options/lookup_strategy.h b/src/paimon/core/options/lookup_strategy.h new file mode 100644 index 00000000..c13bd4a1 --- /dev/null +++ b/src/paimon/core/options/lookup_strategy.h @@ -0,0 +1,42 @@ +/* + * Copyright 2026-present Alibaba 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 paimon { +/// Strategy for lookup. +struct LookupStrategy { + LookupStrategy(bool _is_first_row, bool _produce_changelog, bool _deletion_vector, + bool _force_lookup) + : is_first_row(_is_first_row), + produce_changelog(_produce_changelog), + deletion_vector(_deletion_vector), + need_lookup(_produce_changelog || _deletion_vector || _is_first_row || _force_lookup) {} + + bool operator==(const LookupStrategy& other) const { + if (this == &other) { + return true; + } + return is_first_row == other.is_first_row && produce_changelog == other.produce_changelog && + deletion_vector == other.deletion_vector && need_lookup == other.need_lookup; + } + + bool is_first_row; + bool produce_changelog; + bool deletion_vector; + bool need_lookup; +}; +} // namespace paimon diff --git a/src/paimon/core/postpone/postpone_bucket_file_store_write.h b/src/paimon/core/postpone/postpone_bucket_file_store_write.h index c070b472..f3c12ece 100644 --- a/src/paimon/core/postpone/postpone_bucket_file_store_write.h +++ b/src/paimon/core/postpone/postpone_bucket_file_store_write.h @@ -71,12 +71,13 @@ class PostponeBucketFileStoreWrite : public AbstractFileStoreWrite { PAIMON_ASSIGN_OR_RAISE( std::shared_ptr file_store_path_factory, - FileStorePathFactory::Create( - root_path, schema, table_schema->PartitionKeys(), - new_options.GetPartitionDefaultName(), - new_options.GetWriteFileFormat()->Identifier(), new_options.DataFilePrefix(), - new_options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, new_options.IndexFileInDataFileDir(), pool)); + FileStorePathFactory::Create(root_path, schema, table_schema->PartitionKeys(), + new_options.GetPartitionDefaultName(), + new_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + new_options.DataFilePrefix(), + new_options.LegacyPartitionNameEnabled(), external_paths, + global_index_external_path, + new_options.IndexFileInDataFileDir(), pool)); // Ignoring previous files saves scanning time. // For postpone bucket tables, we only append new files to bucket = -2 directories. diff --git a/src/paimon/core/postpone/postpone_bucket_writer.cpp b/src/paimon/core/postpone/postpone_bucket_writer.cpp index 24b5a78f..5894ec91 100644 --- a/src/paimon/core/postpone/postpone_bucket_writer.cpp +++ b/src/paimon/core/postpone/postpone_bucket_writer.cpp @@ -243,7 +243,7 @@ PostponeBucketWriter::CreateRollingRowWriter() const { ::ArrowSchema arrow_schema; ScopeGuard guard([&arrow_schema]() { ArrowSchemaRelease(&arrow_schema); }); PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*write_schema_, &arrow_schema)); - auto format = options_.GetWriteFileFormat(); + auto format = options_.GetWriteFileFormat(/*level=*/0); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr writer_builder, format->CreateWriterBuilder(&arrow_schema, options_.GetWriteBatchSize())); diff --git a/src/paimon/core/table/source/table_read.cpp b/src/paimon/core/table/source/table_read.cpp index 9d784478..7a1504be 100644 --- a/src/paimon/core/table/source/table_read.cpp +++ b/src/paimon/core/table/source/table_read.cpp @@ -95,7 +95,8 @@ Result> CreateTableRead( std::shared_ptr path_factory, FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), core_options.GetWriteFileFormat()->Identifier(), + core_options.GetPartitionDefaultName(), + core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), memory_pool)); diff --git a/src/paimon/core/table/source/table_scan.cpp b/src/paimon/core/table/source/table_scan.cpp index de870395..5fd42074 100644 --- a/src/paimon/core/table/source/table_scan.cpp +++ b/src/paimon/core/table/source/table_scan.cpp @@ -215,7 +215,8 @@ Result> TableScan::Create(std::unique_ptr path_factory, FileStorePathFactory::Create( context->GetPath(), arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), core_options.GetWriteFileFormat()->Identifier(), + core_options.GetPartitionDefaultName(), + core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), context->GetMemoryPool())); diff --git a/src/paimon/core/utils/file_store_path_factory_cache.h b/src/paimon/core/utils/file_store_path_factory_cache.h new file mode 100644 index 00000000..1b7d2d7f --- /dev/null +++ b/src/paimon/core/utils/file_store_path_factory_cache.h @@ -0,0 +1,67 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/core_options.h" +#include "paimon/core/schema/table_schema.h" +#include "paimon/core/utils/file_store_path_factory.h" +#include "paimon/memory/memory_pool.h" +#include "paimon/result.h" + +namespace paimon { +class FileStorePathFactoryCache { + public: + FileStorePathFactoryCache(const std::string& root, + const std::shared_ptr& table_schema, + const CoreOptions& options, const std::shared_ptr& pool) + : root_(root), table_schema_(table_schema), options_(options), pool_(pool) {} + + Result> GetOrCreatePathFactory( + const std::string& format) { + auto iter = format_to_path_factory_.find(format); + if (iter != format_to_path_factory_.end()) { + return iter->second; + } + auto arrow_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema_->Fields()); + PAIMON_ASSIGN_OR_RAISE(std::vector external_paths, + options_.CreateExternalPaths()); + PAIMON_ASSIGN_OR_RAISE(std::optional global_index_external_path, + options_.CreateGlobalIndexExternalPath()); + + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr path_factory, + FileStorePathFactory::Create( + root_, arrow_schema, table_schema_->PartitionKeys(), + options_.GetPartitionDefaultName(), format, options_.DataFilePrefix(), + options_.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, + options_.IndexFileInDataFileDir(), pool_)); + format_to_path_factory_[format] = path_factory; + return path_factory; + } + + const std::string& RootPath() const { + return root_; + } + + private: + std::string root_; + std::shared_ptr table_schema_; + CoreOptions options_; + std::shared_ptr pool_; + std::map> format_to_path_factory_; +}; +} // namespace paimon diff --git a/src/paimon/core/utils/file_store_path_factory_cache_test.cpp b/src/paimon/core/utils/file_store_path_factory_cache_test.cpp new file mode 100644 index 00000000..40a40a76 --- /dev/null +++ b/src/paimon/core/utils/file_store_path_factory_cache_test.cpp @@ -0,0 +1,53 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/utils/file_store_path_factory_cache.h" + +#include "gtest/gtest.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::test { +TEST(FileStorePathFactoryCacheTest, TestSimple) { + std::vector fields = {DataField(0, arrow::field("f0", arrow::int32())), + DataField(1, arrow::field("f1", arrow::utf8()))}; + ASSERT_OK_AND_ASSIGN( + std::shared_ptr table_schema, + TableSchema::InitSchema(/*schema_id=*/0, fields, /*highest_field_id=*/fields.size(), + /*partition_keys=*/{}, + /*primary_keys=*/{}, /*options=*/{}, /*time_millis=*/0)); + + ASSERT_OK_AND_ASSIGN(auto options, CoreOptions::FromMap({})); + FileStorePathFactoryCache cache("/test_root/", table_schema, options, GetDefaultPool()); + + ASSERT_EQ(cache.format_to_path_factory_.size(), 0); + + ASSERT_OK_AND_ASSIGN(auto factory, cache.GetOrCreatePathFactory("orc")); + ASSERT_TRUE(factory); + ASSERT_EQ(factory->format_identifier_, "orc"); + ASSERT_EQ(cache.format_to_path_factory_.size(), 1); + + ASSERT_OK_AND_ASSIGN(factory, cache.GetOrCreatePathFactory("parquet")); + ASSERT_TRUE(factory); + ASSERT_EQ(factory->format_identifier_, "parquet"); + ASSERT_EQ(cache.format_to_path_factory_.size(), 2); + + ASSERT_OK_AND_ASSIGN(factory, cache.GetOrCreatePathFactory("orc")); + ASSERT_TRUE(factory); + ASSERT_EQ(factory->format_identifier_, "orc"); + ASSERT_EQ(cache.format_to_path_factory_.size(), 2); +} + +} // namespace paimon::test diff --git a/src/paimon/core/utils/file_store_path_factory_test.cpp b/src/paimon/core/utils/file_store_path_factory_test.cpp index bc8825bd..bfd20657 100644 --- a/src/paimon/core/utils/file_store_path_factory_test.cpp +++ b/src/paimon/core/utils/file_store_path_factory_test.cpp @@ -68,7 +68,7 @@ class FileStorePathFactoryTest : public ::testing::Test { EXPECT_OK_AND_ASSIGN(auto path_factory, FileStorePathFactory::Create( root, schema, {"f0", "f3"}, options.GetPartitionDefaultName(), - options.GetWriteFileFormat()->Identifier(), + options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, /*global_index_external_path=*/std::nullopt, options.IndexFileInDataFileDir(), mem_pool_)); From 22fb14cd20a948aebcf317abc213f546b354651a Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Thu, 19 Mar 2026 06:54:34 +0000 Subject: [PATCH 02/10] fix0254 From c9db098435a31a09f6085acb652524024852988c Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Thu, 19 Mar 2026 07:02:28 +0000 Subject: [PATCH 03/10] fix0302 --- src/paimon/core/operation/raw_file_split_read_test.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/paimon/core/operation/raw_file_split_read_test.cpp b/src/paimon/core/operation/raw_file_split_read_test.cpp index b4a97f0b..7ba8dc3e 100644 --- a/src/paimon/core/operation/raw_file_split_read_test.cpp +++ b/src/paimon/core/operation/raw_file_split_read_test.cpp @@ -497,7 +497,9 @@ TEST_F(RawFileSplitReadTest, TestMatch) { split_read->Match(data_split, /*force_keep_delete=*/false)); ASSERT_FALSE(match_result); } - { ASSERT_NOK(split_read->Match(nullptr, /*force_keep_delete=*/false)); } + { + ASSERT_NOK(split_read->Match(nullptr, /*force_keep_delete=*/false)); + } } } // namespace paimon::test From 75906ccb571cba1a163dc72ef106b9208ca40c2c Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Thu, 19 Mar 2026 16:17:35 +0800 Subject: [PATCH 04/10] fix --- src/paimon/CMakeLists.txt | 2 + src/paimon/core/append/append_only_writer.cpp | 2 +- src/paimon/core/core_options.cpp | 5 + src/paimon/core/core_options.h | 1 + src/paimon/core/core_options_test.cpp | 5 +- .../global_index/global_index_scan_impl.cpp | 2 +- .../global_index/global_index_write_task.cpp | 2 +- .../compact/changelog_merge_tree_rewriter.cpp | 124 +++++++++++++ .../compact/changelog_merge_tree_rewriter.h | 97 +--------- .../lookup_merge_tree_compact_rewriter.cpp | 171 ++++++++++++++++++ .../lookup_merge_tree_compact_rewriter.h | 113 +----------- ...ookup_merge_tree_compact_rewriter_test.cpp | 2 +- .../compact/merge_tree_compact_rewriter.cpp | 4 +- .../core/mergetree/lookup_levels_test.cpp | 2 +- src/paimon/core/migrate/file_meta_utils.cpp | 2 +- .../append_only_file_store_write.cpp | 2 +- .../core/operation/expire_snapshots_test.cpp | 2 +- .../core/operation/file_store_commit.cpp | 4 +- .../key_value_file_store_scan_test.cpp | 2 +- .../operation/manifest_file_merger_test.cpp | 2 +- .../operation/merge_file_split_read_test.cpp | 2 +- .../core/operation/orphan_files_cleaner.cpp | 2 +- .../operation/raw_file_split_read_test.cpp | 4 +- src/paimon/core/table/source/table_read.cpp | 2 +- src/paimon/core/table/source/table_scan.cpp | 2 +- .../utils/file_store_path_factory_test.cpp | 2 +- 26 files changed, 339 insertions(+), 221 deletions(-) create mode 100644 src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp create mode 100644 src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index c2bfe0fb..dc972227 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -233,6 +233,8 @@ set(PAIMON_CORE_SRCS core/mergetree/compact/partial_update_merge_function.cpp core/mergetree/compact/sort_merge_reader_with_loser_tree.cpp core/mergetree/compact/sort_merge_reader_with_min_heap.cpp + core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp + core/mergetree/compact/changelog_merge_tree_rewriter.cpp core/mergetree/merge_tree_writer.cpp core/mergetree/levels.cpp core/mergetree/lookup_levels.cpp diff --git a/src/paimon/core/append/append_only_writer.cpp b/src/paimon/core/append/append_only_writer.cpp index 32f5a12f..a45cf8c0 100644 --- a/src/paimon/core/append/append_only_writer.cpp +++ b/src/paimon/core/append/append_only_writer.cpp @@ -173,7 +173,7 @@ AppendOnlyWriter::SingleFileWriterCreator AppendOnlyWriter::GetDataFileWriterCre ::ArrowSchema arrow_schema; ScopeGuard guard([&arrow_schema]() { ArrowSchemaRelease(&arrow_schema); }); PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*schema, &arrow_schema)); - auto format = options_.GetWriteFileFormat(/*level=*/0); + auto format = options_.GetFileFormat(); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr writer_builder, format->CreateWriterBuilder(&arrow_schema, options_.GetWriteBatchSize())); diff --git a/src/paimon/core/core_options.cpp b/src/paimon/core/core_options.cpp index 8d43bccb..8bb6d90a 100644 --- a/src/paimon/core/core_options.cpp +++ b/src/paimon/core/core_options.cpp @@ -653,6 +653,10 @@ std::shared_ptr CoreOptions::GetWriteFileFormat(int32_t level) const return impl_->file_format; } +std::shared_ptr CoreOptions::GetFileFormat() const { + return impl_->file_format; +} + std::shared_ptr CoreOptions::GetFileSystem() const { return impl_->file_system; } @@ -1002,4 +1006,5 @@ const CompressOptions& CoreOptions::GetLookupCompressOptions() const { int32_t CoreOptions::GetCachePageSize() const { return static_cast(impl_->cache_page_size); } + } // namespace paimon diff --git a/src/paimon/core/core_options.h b/src/paimon/core/core_options.h index 59e28390..21352d69 100644 --- a/src/paimon/core/core_options.h +++ b/src/paimon/core/core_options.h @@ -54,6 +54,7 @@ class PAIMON_EXPORT CoreOptions { ~CoreOptions(); int32_t GetBucket() const; + std::shared_ptr GetFileFormat() const; std::shared_ptr GetWriteFileFormat(int32_t level) const; std::shared_ptr GetFileSystem() const; const std::string& GetFileCompression() const; diff --git a/src/paimon/core/core_options_test.cpp b/src/paimon/core/core_options_test.cpp index a90e82a6..779398d0 100644 --- a/src/paimon/core/core_options_test.cpp +++ b/src/paimon/core/core_options_test.cpp @@ -33,6 +33,7 @@ namespace paimon::test { TEST(CoreOptionsTest, TestDefaultValue) { ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap({})); ASSERT_EQ(core_options.GetManifestFormat()->Identifier(), "avro"); + ASSERT_EQ(core_options.GetFileFormat()->Identifier(), "parquet"); ASSERT_EQ(core_options.GetWriteFileFormat(0)->Identifier(), "parquet"); ASSERT_EQ(core_options.GetWriteFileFormat(3)->Identifier(), "parquet"); ASSERT_TRUE(core_options.GetFileSystem()); @@ -197,8 +198,8 @@ TEST(CoreOptionsTest, TestFromMap) { auto fs = core_options.GetFileSystem(); ASSERT_TRUE(fs); - auto format = core_options.GetWriteFileFormat(0); - ASSERT_EQ(format->Identifier(), "avro"); + ASSERT_EQ(core_options.GetFileFormat()->Identifier(), "orc"); + ASSERT_EQ(core_options.GetWriteFileFormat(0)->Identifier(), "avro"); ASSERT_EQ(core_options.GetWriteFileFormat(1)->Identifier(), "orc"); ASSERT_EQ(core_options.GetWriteFileFormat(3)->Identifier(), "parquet"); diff --git a/src/paimon/core/global_index/global_index_scan_impl.cpp b/src/paimon/core/global_index/global_index_scan_impl.cpp index b4034cbf..53237e4a 100644 --- a/src/paimon/core/global_index/global_index_scan_impl.cpp +++ b/src/paimon/core/global_index/global_index_scan_impl.cpp @@ -111,7 +111,7 @@ Status GlobalIndexScanImpl::Scan() { FileStorePathFactory::Create( root_path_, arrow_schema, table_schema_->PartitionKeys(), options_.GetPartitionDefaultName(), - options_.GetWriteFileFormat(/*level=*/0)->Identifier(), options_.DataFilePrefix(), + options_.GetFileFormat()->Identifier(), options_.DataFilePrefix(), options_.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options_.IndexFileInDataFileDir(), pool_)); diff --git a/src/paimon/core/global_index/global_index_write_task.cpp b/src/paimon/core/global_index/global_index_write_task.cpp index 2beb8e02..3654cead 100644 --- a/src/paimon/core/global_index/global_index_write_task.cpp +++ b/src/paimon/core/global_index/global_index_write_task.cpp @@ -45,7 +45,7 @@ Result> CreateGlobalIndexFileManager( std::shared_ptr path_factory, FileStorePathFactory::Create(table_path, all_arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, diff --git a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp new file mode 100644 index 00000000..a06b52da --- /dev/null +++ b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp @@ -0,0 +1,124 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h" +namespace paimon { +ChangelogMergeTreeRewriter::ChangelogMergeTreeRewriter( + int32_t max_level, bool force_drop_delete, const BinaryRow& partition, int32_t bucket, + int64_t schema_id, const std::vector& trimmed_primary_keys, + const CoreOptions& options, const std::shared_ptr& data_schema, + const std::shared_ptr& write_schema, + const std::shared_ptr& path_factory_cache, + std::unique_ptr&& merge_file_split_read, + MergeFunctionWrapperFactory merge_function_wrapper_factory, + const std::shared_ptr& pool) + : MergeTreeCompactRewriter(partition, bucket, schema_id, trimmed_primary_keys, options, + data_schema, write_schema, path_factory_cache, + std::move(merge_file_split_read), + std::move(merge_function_wrapper_factory), pool), + max_level_(max_level), + force_drop_delete_(force_drop_delete) {} + +Result ChangelogMergeTreeRewriter::Rewrite( + int32_t output_level, bool drop_delete, + const std::vector>& sections) { + if (RewriteChangelog(output_level, drop_delete, sections)) { + return RewriteOrProduceChangelog(output_level, sections, drop_delete, + /*rewrite_compact_file=*/true); + } else { + return RewriteCompaction(output_level, drop_delete, sections); + } +} + +Result ChangelogMergeTreeRewriter::Upgrade( + int32_t output_level, const std::shared_ptr& file) { + UpgradeStrategy upgrade_strategy = GenerateUpgradeStrategy(output_level, file); + if (upgrade_strategy.changelog) { + return RewriteOrProduceChangelog(output_level, {{SortedRun::FromSingle(file)}}, + force_drop_delete_, upgrade_strategy.rewrite); + } else { + return MergeTreeCompactRewriter::Upgrade(output_level, file); + } +} + +bool ChangelogMergeTreeRewriter::RewriteLookupChangelog( + int32_t output_level, const std::vector>& sections) const { + if (output_level == 0) { + return false; + } + for (const auto& runs : sections) { + for (const auto& run : runs) { + for (const auto& file : run.Files()) { + if (file->level == 0) { + return true; + } + } + } + } + return false; +} + +Result ChangelogMergeTreeRewriter::RewriteOrProduceChangelog( + int32_t output_level, const std::vector>& sections, bool drop_delete, + bool rewrite_compact_file) { + PAIMON_ASSIGN_OR_RAISE(MergeTreeCompactRewriter::KeyValueConsumerCreator create_consumer, + GenerateKeyValueConsumer()); + std::vector> reader_holders; + + std::unique_ptr compact_file_writer; + if (rewrite_compact_file) { + compact_file_writer = CreateRollingRowWriter(output_level); + } + // TODO(xinyu.lxy): produce changelog + ScopeGuard write_guard([&]() -> void { + if (compact_file_writer) { + compact_file_writer->Abort(); + } + merge_file_split_read_.reset(); + for (const auto& reader : reader_holders) { + reader->Close(); + } + }); + + for (const auto& section : sections) { + PAIMON_RETURN_NOT_OK(MergeReadAndWrite(output_level, drop_delete, section, create_consumer, + compact_file_writer.get(), &reader_holders)); + } + if (compact_file_writer) { + PAIMON_RETURN_NOT_OK(compact_file_writer->Close()); + } + auto before = ExtractFilesFromSections(sections); + std::vector> after; + if (compact_file_writer) { + PAIMON_ASSIGN_OR_RAISE(after, compact_file_writer->GetResult()); + } else { + after.reserve(before.size()); + for (const auto& file : before) { + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr new_file, + file->Upgrade(output_level)); + after.emplace_back(std::move(new_file)); + } + } + if (rewrite_compact_file) { + NotifyRewriteCompactBefore(before); + } + write_guard.Release(); + + after = NotifyRewriteCompactAfter(after); + return CompactResult(before, after); +} + +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h index 6a3f3668..b0bd5546 100644 --- a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h +++ b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h @@ -19,37 +19,17 @@ #include "arrow/api.h" #include "paimon/core/core_options.h" #include "paimon/core/io/data_file_meta.h" -#include "paimon/core/io/rolling_file_writer.h" #include "paimon/core/key_value.h" #include "paimon/core/mergetree/compact/merge_tree_compact_rewriter.h" -#include "paimon/core/mergetree/merge_tree_writer.h" -#include "paimon/core/operation/merge_file_split_read.h" -#include "paimon/core/schema/table_schema.h" -#include "paimon/core/utils/file_store_path_factory.h" namespace paimon { /// A `MergeTreeCompactRewriter` which produces changelog files while performing compaction. class ChangelogMergeTreeRewriter : public MergeTreeCompactRewriter { public: Result Rewrite(int32_t output_level, bool drop_delete, - const std::vector>& sections) override { - if (RewriteChangelog(output_level, drop_delete, sections)) { - return RewriteOrProduceChangelog(output_level, sections, drop_delete, - /*rewrite_compact_file=*/true); - } else { - return RewriteCompaction(output_level, drop_delete, sections); - } - } + const std::vector>& sections) override; Result Upgrade(int32_t output_level, - const std::shared_ptr& file) override { - UpgradeStrategy upgrade_strategy = GenerateUpgradeStrategy(output_level, file); - if (upgrade_strategy.changelog) { - return RewriteOrProduceChangelog(output_level, {{SortedRun::FromSingle(file)}}, - force_drop_delete_, upgrade_strategy.rewrite); - } else { - return MergeTreeCompactRewriter::Upgrade(output_level, file); - } - } + const std::shared_ptr& file) override; protected: ChangelogMergeTreeRewriter(int32_t max_level, bool force_drop_delete, @@ -61,13 +41,7 @@ class ChangelogMergeTreeRewriter : public MergeTreeCompactRewriter { const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, MergeFunctionWrapperFactory merge_function_wrapper_factory, - const std::shared_ptr& pool) - : MergeTreeCompactRewriter(partition, bucket, schema_id, trimmed_primary_keys, options, - data_schema, write_schema, path_factory_cache, - std::move(merge_file_split_read), - std::move(merge_function_wrapper_factory), pool), - max_level_(max_level), - force_drop_delete_(force_drop_delete) {} + const std::shared_ptr& pool); struct UpgradeStrategy { static UpgradeStrategy NoChangelogNoRewrite() { @@ -100,73 +74,12 @@ class ChangelogMergeTreeRewriter : public MergeTreeCompactRewriter { const std::vector>& sections) const = 0; bool RewriteLookupChangelog(int32_t output_level, - const std::vector>& sections) const { - if (output_level == 0) { - return false; - } - for (const auto& runs : sections) { - for (const auto& run : runs) { - for (const auto& file : run.Files()) { - if (file->level == 0) { - return true; - } - } - } - } - return false; - } + const std::vector>& sections) const; private: Result RewriteOrProduceChangelog( int32_t output_level, const std::vector>& sections, bool drop_delete, - bool rewrite_compact_file) { - PAIMON_ASSIGN_OR_RAISE(MergeTreeCompactRewriter::KeyValueConsumerCreator create_consumer, - GenerateKeyValueConsumer()); - std::vector> reader_holders; - - std::unique_ptr compact_file_writer; - if (rewrite_compact_file) { - compact_file_writer = CreateRollingRowWriter(output_level); - } - // TODO(xinyu.lxy): produce changelog - ScopeGuard write_guard([&]() -> void { - if (compact_file_writer) { - compact_file_writer->Abort(); - } - merge_file_split_read_.reset(); - for (const auto& reader : reader_holders) { - reader->Close(); - } - }); - - for (const auto& section : sections) { - PAIMON_RETURN_NOT_OK(MergeReadAndWrite(output_level, drop_delete, section, - create_consumer, compact_file_writer.get(), - &reader_holders)); - } - if (compact_file_writer) { - PAIMON_RETURN_NOT_OK(compact_file_writer->Close()); - } - auto before = ExtractFilesFromSections(sections); - std::vector> after; - if (compact_file_writer) { - PAIMON_ASSIGN_OR_RAISE(after, compact_file_writer->GetResult()); - } else { - after.reserve(before.size()); - for (const auto& file : before) { - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr new_file, - file->Upgrade(output_level)); - after.emplace_back(std::move(new_file)); - } - } - if (rewrite_compact_file) { - NotifyRewriteCompactBefore(before); - } - write_guard.Release(); - - after = NotifyRewriteCompactAfter(after); - return CompactResult(before, after); - } + bool rewrite_compact_file); protected: int32_t max_level_; diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp new file mode 100644 index 00000000..13e087c2 --- /dev/null +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp @@ -0,0 +1,171 @@ +/* + * Copyright 2026-present Alibaba 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 "paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h" + +#include "paimon/common/table/special_fields.h" +#include "paimon/core/mergetree/compact/first_row_merge_function_wrapper.h" +#include "paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h" +#include "paimon/core/mergetree/lookup/file_position.h" +#include "paimon/core/mergetree/lookup/positioned_key_value.h" + +namespace paimon { +template +LookupMergeTreeCompactRewriter::LookupMergeTreeCompactRewriter( + std::unique_ptr>&& lookup_levels, + const std::shared_ptr& dv_maintainer, int32_t max_level, + const BinaryRow& partition, int32_t bucket, int64_t schema_id, + const std::vector& trimmed_primary_keys, const CoreOptions& options, + const std::shared_ptr& data_schema, + const std::shared_ptr& write_schema, + const std::shared_ptr& path_factory_cache, + std::unique_ptr&& merge_file_split_read, + MergeFunctionWrapperFactory merge_function_wrapper_factory, + const std::shared_ptr& pool) + : ChangelogMergeTreeRewriter( + max_level, /*force_drop_delete=*/dv_maintainer != nullptr, partition, bucket, schema_id, + trimmed_primary_keys, options, data_schema, write_schema, path_factory_cache, + std::move(merge_file_split_read), std::move(merge_function_wrapper_factory), pool), + lookup_levels_(std::move(lookup_levels)), + dv_maintainer_(dv_maintainer) {} + +template +Result>> +LookupMergeTreeCompactRewriter::Create( + int32_t max_level, std::unique_ptr>&& lookup_levels, + const std::shared_ptr& dv_maintainer, + MergeFunctionWrapperFactory merge_function_wrapper_factory, int32_t bucket, + const BinaryRow& partition, const std::shared_ptr& table_schema, + const std::shared_ptr& path_factory_cache, + const CoreOptions& options, const std::shared_ptr& pool) { + PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, + table_schema->TrimmedPrimaryKeys()); + auto data_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); + auto write_schema = SpecialFields::CompleteSequenceAndValueKindField(data_schema); + + // TODO(xinyu.lxy): set executor + ReadContextBuilder read_context_builder(path_factory_cache->RootPath()); + read_context_builder.SetOptions(options.ToMap()).EnablePrefetch(true).WithMemoryPool(pool); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr read_context, + read_context_builder.Finish()); + + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr internal_context, + InternalReadContext::Create(read_context, table_schema, options.ToMap())); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr path_factory, + path_factory_cache->GetOrCreatePathFactory( + options.GetFileFormat()->Identifier())); + PAIMON_ASSIGN_OR_RAISE( + std::unique_ptr merge_file_split_read, + MergeFileSplitRead::Create(path_factory, internal_context, pool, CreateDefaultExecutor())); + return std::unique_ptr(new LookupMergeTreeCompactRewriter( + std::move(lookup_levels), dv_maintainer, max_level, partition, bucket, table_schema->Id(), + trimmed_primary_keys, options, data_schema, write_schema, path_factory_cache, + std::move(merge_file_split_read), std::move(merge_function_wrapper_factory), pool)); +} + +template +std::shared_ptr> +LookupMergeTreeCompactRewriter::CreateFirstRowMergeFunctionWrapper( + std::unique_ptr&& merge_func, int32_t output_level, + LookupLevels* lookup_levels) { + auto contains = [output_level, + lookup_levels](const std::shared_ptr& key) -> Result { + PAIMON_ASSIGN_OR_RAISE(std::optional contain, + lookup_levels->Lookup(key, output_level + 1)); + return contain != std::nullopt; + }; + return std::make_shared(std::move(merge_func), + std::move(contains)); +} + +template +Result>> +LookupMergeTreeCompactRewriter::CreateLookupMergeFunctionWrapper( + std::unique_ptr&& merge_func, int32_t output_level, + const std::shared_ptr& deletion_vectors_maintainer, + const LookupStrategy& lookup_strategy, + const std::shared_ptr& user_defined_seq_comparator, + LookupLevels* lookup_levels) { + auto lookup = [output_level, lookup_levels]( + const std::shared_ptr& key) -> Result> { + return lookup_levels->Lookup(key, output_level + 1); + }; + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr> wrapper, + LookupChangelogMergeFunctionWrapper::Create( + std::move(merge_func), std::move(lookup), lookup_strategy, + deletion_vectors_maintainer, user_defined_seq_comparator)); + return wrapper; +} + +template +ChangelogMergeTreeRewriter::UpgradeStrategy +LookupMergeTreeCompactRewriter::GenerateUpgradeStrategy( + int32_t output_level, const std::shared_ptr& file) const { + if (file->level != 0) { + return ChangelogMergeTreeRewriter::UpgradeStrategy::NoChangelogNoRewrite(); + } + // forcing rewriting when upgrading from level 0 to level x with different file formats + if (options_.GetWriteFileFormat(file->level)->Identifier() != + options_.GetWriteFileFormat(output_level)->Identifier()) { + return ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(); + } + + // In deletionVector mode, since drop delete is required, when delete row count > 0 rewrite + // is required. + if (dv_maintainer_ && (!file->delete_row_count || file->delete_row_count.value() > 0)) { + return ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(); + } + + if (output_level == max_level_) { + return ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogNoRewrite(); + } + + // DEDUPLICATE retains the latest records as the final result, so merging has no impact on + // it at all. + if (options_.GetMergeEngine() == MergeEngine::DEDUPLICATE && + options_.GetSequenceField().empty()) { + return ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogNoRewrite(); + } + // other merge engines must rewrite file, because some records that are already at higher + // level may be merged + // See LookupMergeFunction, it just returns newly records. + return ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(); +} + +template +void LookupMergeTreeCompactRewriter::NotifyRewriteCompactBefore( + const std::vector>& files) { + if (dv_maintainer_) { + for (const auto& file : files) { + dv_maintainer_->RemoveDeletionVectorOf(file->file_name); + } + } +} + +template +std::vector> +LookupMergeTreeCompactRewriter::NotifyRewriteCompactAfter( + const std::vector>& files) { + // TODO(xinyu.lxy): support remoteLookupFileManager + return files; +} + +template class LookupMergeTreeCompactRewriter; +template class LookupMergeTreeCompactRewriter; +template class LookupMergeTreeCompactRewriter; +template class LookupMergeTreeCompactRewriter; +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h index e1ba178f..f5c51f30 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h @@ -16,21 +16,13 @@ #pragma once #include "arrow/api.h" -#include "paimon/common/table/special_fields.h" #include "paimon/core/core_options.h" #include "paimon/core/io/data_file_meta.h" -#include "paimon/core/io/rolling_file_writer.h" -#include "paimon/core/key_value.h" #include "paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h" #include "paimon/core/mergetree/compact/first_row_merge_function.h" -#include "paimon/core/mergetree/compact/first_row_merge_function_wrapper.h" -#include "paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h" #include "paimon/core/mergetree/compact/lookup_merge_function.h" #include "paimon/core/mergetree/lookup_levels.h" -#include "paimon/core/mergetree/merge_tree_writer.h" -#include "paimon/core/operation/merge_file_split_read.h" #include "paimon/core/schema/table_schema.h" -#include "paimon/core/utils/file_store_path_factory.h" namespace paimon { /// A `MergeTreeCompactRewriter` which produces changelog files by lookup for the compaction /// involving level 0 files. @@ -43,33 +35,7 @@ class LookupMergeTreeCompactRewriter : public ChangelogMergeTreeRewriter { MergeFunctionWrapperFactory merge_function_wrapper_factory, int32_t bucket, const BinaryRow& partition, const std::shared_ptr& table_schema, const std::shared_ptr& path_factory_cache, - const CoreOptions& options, const std::shared_ptr& pool) { - PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, - table_schema->TrimmedPrimaryKeys()); - auto data_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); - auto write_schema = SpecialFields::CompleteSequenceAndValueKindField(data_schema); - - // TODO(xinyu.lxy): set executor - ReadContextBuilder read_context_builder(path_factory_cache->RootPath()); - read_context_builder.SetOptions(options.ToMap()).EnablePrefetch(true).WithMemoryPool(pool); - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr read_context, - read_context_builder.Finish()); - - PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr internal_context, - InternalReadContext::Create(read_context, table_schema, options.ToMap())); - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr path_factory, - path_factory_cache->GetOrCreatePathFactory( - options.GetWriteFileFormat(/*level=*/0)->Identifier())); - PAIMON_ASSIGN_OR_RAISE(std::unique_ptr merge_file_split_read, - MergeFileSplitRead::Create(path_factory, internal_context, pool, - CreateDefaultExecutor())); - return std::unique_ptr(new LookupMergeTreeCompactRewriter( - std::move(lookup_levels), dv_maintainer, max_level, partition, bucket, - table_schema->Id(), trimmed_primary_keys, options, data_schema, write_schema, - path_factory_cache, std::move(merge_file_split_read), - std::move(merge_function_wrapper_factory), pool)); - } + const CoreOptions& options, const std::shared_ptr& pool); Status Close() override { return lookup_levels_->Close(); @@ -77,33 +43,14 @@ class LookupMergeTreeCompactRewriter : public ChangelogMergeTreeRewriter { static std::shared_ptr> CreateFirstRowMergeFunctionWrapper( std::unique_ptr&& merge_func, int32_t output_level, - LookupLevels* lookup_levels) { - auto contains = [output_level, - lookup_levels](const std::shared_ptr& key) -> Result { - PAIMON_ASSIGN_OR_RAISE(std::optional contain, - lookup_levels->Lookup(key, output_level + 1)); - return contain != std::nullopt; - }; - return std::make_shared(std::move(merge_func), - std::move(contains)); - } + LookupLevels* lookup_levels); static Result>> CreateLookupMergeFunctionWrapper( std::unique_ptr&& merge_func, int32_t output_level, const std::shared_ptr& deletion_vectors_maintainer, const LookupStrategy& lookup_strategy, const std::shared_ptr& user_defined_seq_comparator, - LookupLevels* lookup_levels) { - auto lookup = [output_level, lookup_levels]( - const std::shared_ptr& key) -> Result> { - return lookup_levels->Lookup(key, output_level + 1); - }; - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr> wrapper, - LookupChangelogMergeFunctionWrapper::Create( - std::move(merge_func), std::move(lookup), lookup_strategy, - deletion_vectors_maintainer, user_defined_seq_comparator)); - return wrapper; - } + LookupLevels* lookup_levels); private: LookupMergeTreeCompactRewriter( @@ -116,14 +63,7 @@ class LookupMergeTreeCompactRewriter : public ChangelogMergeTreeRewriter { const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, MergeFunctionWrapperFactory merge_function_wrapper_factory, - const std::shared_ptr& pool) - : ChangelogMergeTreeRewriter(max_level, /*force_drop_delete=*/dv_maintainer_ != nullptr, - partition, bucket, schema_id, trimmed_primary_keys, options, - data_schema, write_schema, path_factory_cache, - std::move(merge_file_split_read), - std::move(merge_function_wrapper_factory), pool), - lookup_levels_(std::move(lookup_levels)), - dv_maintainer_(dv_maintainer) {} + const std::shared_ptr& pool); bool RewriteChangelog(int32_t output_level, bool drop_delete, const std::vector>& sections) const override { @@ -131,52 +71,13 @@ class LookupMergeTreeCompactRewriter : public ChangelogMergeTreeRewriter { } UpgradeStrategy GenerateUpgradeStrategy( - int32_t output_level, const std::shared_ptr& file) const override { - if (file->level != 0) { - return UpgradeStrategy::NoChangelogNoRewrite(); - } - // forcing rewriting when upgrading from level 0 to level x with different file formats - if (options_.GetWriteFileFormat(file->level)->Identifier() != - options_.GetWriteFileFormat(output_level)->Identifier()) { - return UpgradeStrategy::ChangelogWithRewrite(); - } - - // In deletionVector mode, since drop delete is required, when delete row count > 0 rewrite - // is required. - if (dv_maintainer_ && (!file->delete_row_count || file->delete_row_count.value() > 0)) { - return UpgradeStrategy::ChangelogWithRewrite(); - } - - if (output_level == max_level_) { - return UpgradeStrategy::ChangelogNoRewrite(); - } - - // DEDUPLICATE retains the latest records as the final result, so merging has no impact on - // it at all. - if (options_.GetMergeEngine() == MergeEngine::DEDUPLICATE && - options_.GetSequenceField().empty()) { - return UpgradeStrategy::ChangelogNoRewrite(); - } - // other merge engines must rewrite file, because some records that are already at higher - // level may be merged - // See LookupMergeFunction, it just returns newly records. - return UpgradeStrategy::ChangelogWithRewrite(); - } + int32_t output_level, const std::shared_ptr& file) const override; void NotifyRewriteCompactBefore( - const std::vector>& files) override { - if (dv_maintainer_) { - for (const auto& file : files) { - dv_maintainer_->RemoveDeletionVectorOf(file->file_name); - } - } - } + const std::vector>& files) override; std::vector> NotifyRewriteCompactAfter( - const std::vector>& files) override { - // TODO(xinyu.lxy): support remoteLookupFileManager - return files; - } + const std::vector>& files) override; private: std::unique_ptr> lookup_levels_; diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp index cdd3af2b..6be87ff9 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp @@ -292,7 +292,7 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { std::shared_ptr path_factory, FileStorePathFactory::Create( table_path, arrow_schema_, /*partition_keys=*/{}, options.GetPartitionDefaultName(), - options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), + options.GetFileFormat()->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), pool_)); return path_factory; diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp index ae239362..b4dd9345 100644 --- a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp @@ -71,7 +71,7 @@ Result> MergeTreeCompactRewriter::Crea InternalReadContext::Create(read_context, table_schema, options.ToMap())); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr path_factory, path_factory_cache->GetOrCreatePathFactory( - options.GetWriteFileFormat(/*level=*/0)->Identifier())); + options.GetFileFormat()->Identifier())); PAIMON_ASSIGN_OR_RAISE( std::unique_ptr merge_file_split_read, MergeFileSplitRead::Create(path_factory, internal_context, pool, CreateDefaultExecutor())); @@ -182,7 +182,7 @@ Status MergeTreeCompactRewriter::MergeReadAndWrite( // prepare sort merge reader PAIMON_ASSIGN_OR_RAISE( std::shared_ptr data_file_path_factory, - CreateDataFilePathFactory(options_.GetWriteFileFormat(/*level=*/0)->Identifier())); + CreateDataFilePathFactory(options_.GetFileFormat()->Identifier())); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr> wrapper, merge_function_wrapper_factory_(output_level)); merge_file_split_read_->SetMergeFunctionWrapper(wrapper); diff --git a/src/paimon/core/mergetree/lookup_levels_test.cpp b/src/paimon/core/mergetree/lookup_levels_test.cpp index 45da7f8d..c059e19b 100644 --- a/src/paimon/core/mergetree/lookup_levels_test.cpp +++ b/src/paimon/core/mergetree/lookup_levels_test.cpp @@ -128,7 +128,7 @@ class LookupLevelsTest : public testing::Test { std::shared_ptr path_factory, FileStorePathFactory::Create( table_path, arrow_schema_, /*partition_keys=*/{}, options.GetPartitionDefaultName(), - options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), + options.GetFileFormat()->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), pool_)); return path_factory; diff --git a/src/paimon/core/migrate/file_meta_utils.cpp b/src/paimon/core/migrate/file_meta_utils.cpp index d9cfe9f5..bb8b1748 100644 --- a/src/paimon/core/migrate/file_meta_utils.cpp +++ b/src/paimon/core/migrate/file_meta_utils.cpp @@ -124,7 +124,7 @@ Result> FileMetaUtils::GenerateCommitMessage( CoreOptions::FromMap(table_options, file_system)); std::shared_ptr fs = core_options.GetFileSystem(); - std::shared_ptr format = core_options.GetWriteFileFormat(/*level=*/0); + std::shared_ptr format = core_options.GetFileFormat(); assert(fs); assert(format); PAIMON_RETURN_NOT_OK(ValidateNonObjectPath(src_data_files)); diff --git a/src/paimon/core/operation/append_only_file_store_write.cpp b/src/paimon/core/operation/append_only_file_store_write.cpp index 9f6f2606..6fc66305 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -223,7 +223,7 @@ AppendOnlyFileStoreWrite::GetDataFileWriterCreator( ::ArrowSchema arrow_schema; ScopeGuard guard([&arrow_schema]() { ArrowSchemaRelease(&arrow_schema); }); PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*schema, &arrow_schema)); - auto format = options_.GetWriteFileFormat(/*level=*/0); + auto format = options_.GetFileFormat(); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr writer_builder, format->CreateWriterBuilder(&arrow_schema, options_.GetWriteBatchSize())); diff --git a/src/paimon/core/operation/expire_snapshots_test.cpp b/src/paimon/core/operation/expire_snapshots_test.cpp index 0e3fe3a0..67102854 100644 --- a/src/paimon/core/operation/expire_snapshots_test.cpp +++ b/src/paimon/core/operation/expire_snapshots_test.cpp @@ -121,7 +121,7 @@ class ExpireSnapshotsTest : public testing::Test { auto path_factory, FileStorePathFactory::Create( root, schema_, partition_keys_, options.GetPartitionDefaultName(), - options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), + options.GetFileFormat()->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), mem_pool_)); return path_factory; diff --git a/src/paimon/core/operation/file_store_commit.cpp b/src/paimon/core/operation/file_store_commit.cpp index bd5939ba..a0176086 100644 --- a/src/paimon/core/operation/file_store_commit.cpp +++ b/src/paimon/core/operation/file_store_commit.cpp @@ -79,7 +79,7 @@ Result> FileStoreCommit::Create( PAIMON_ASSIGN_OR_RAISE(CoreOptions options, CoreOptions::FromMap(opts, ctx->GetSpecificFileSystem())); assert(options.GetFileSystem()); - assert(options.GetWriteFileFormat(/*level=*/0)); + assert(options.GetFileFormat()); PAIMON_ASSIGN_OR_RAISE(bool is_object_store, FileSystem::IsObjectStore(root_path)); if (is_object_store && opts.find("enable-object-store-commit-in-inte-test") == opts.end()) { return Status::NotImplemented( @@ -98,7 +98,7 @@ Result> FileStoreCommit::Create( std::shared_ptr path_factory, FileStorePathFactory::Create(root_path, arrow_schema, table_schema.value()->PartitionKeys(), options.GetPartitionDefaultName(), - options.GetWriteFileFormat(/*level=*/0)->Identifier(), + options.GetFileFormat()->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); diff --git a/src/paimon/core/operation/key_value_file_store_scan_test.cpp b/src/paimon/core/operation/key_value_file_store_scan_test.cpp index 8a6a116f..cc0f7d1d 100644 --- a/src/paimon/core/operation/key_value_file_store_scan_test.cpp +++ b/src/paimon/core/operation/key_value_file_store_scan_test.cpp @@ -85,7 +85,7 @@ class KeyValueFileStoreScanTest : public testing::Test { std::shared_ptr path_factory, FileStorePathFactory::Create(table_path, arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, diff --git a/src/paimon/core/operation/manifest_file_merger_test.cpp b/src/paimon/core/operation/manifest_file_merger_test.cpp index 1504a4c6..0e36af35 100644 --- a/src/paimon/core/operation/manifest_file_merger_test.cpp +++ b/src/paimon/core/operation/manifest_file_merger_test.cpp @@ -125,7 +125,7 @@ class ManifestFileMergerTest : public testing::Test { static std::shared_ptr path_factory, FileStorePathFactory::Create( path_str, schema, /*partition_keys=*/{"f0"}, options.GetPartitionDefaultName(), - options.GetWriteFileFormat(/*level=*/0)->Identifier(), options.DataFilePrefix(), + options.GetFileFormat()->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), pool_)); ASSERT_OK_AND_ASSIGN(std::shared_ptr partition_schema, diff --git a/src/paimon/core/operation/merge_file_split_read_test.cpp b/src/paimon/core/operation/merge_file_split_read_test.cpp index fe0ba4df..a20265d7 100644 --- a/src/paimon/core/operation/merge_file_split_read_test.cpp +++ b/src/paimon/core/operation/merge_file_split_read_test.cpp @@ -341,7 +341,7 @@ class MergeFileSplitReadTest : public ::testing::Test, FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); diff --git a/src/paimon/core/operation/orphan_files_cleaner.cpp b/src/paimon/core/operation/orphan_files_cleaner.cpp index 2131ff5c..c581a14b 100644 --- a/src/paimon/core/operation/orphan_files_cleaner.cpp +++ b/src/paimon/core/operation/orphan_files_cleaner.cpp @@ -187,7 +187,7 @@ Result> OrphanFilesCleaner::Create( std::shared_ptr path_factory, FileStorePathFactory::Create(ctx->GetRootPath(), arrow_schema, schema->PartitionKeys(), options.GetPartitionDefaultName(), - options.GetWriteFileFormat(/*level=*/0)->Identifier(), + options.GetFileFormat()->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); diff --git a/src/paimon/core/operation/raw_file_split_read_test.cpp b/src/paimon/core/operation/raw_file_split_read_test.cpp index 7ba8dc3e..be775589 100644 --- a/src/paimon/core/operation/raw_file_split_read_test.cpp +++ b/src/paimon/core/operation/raw_file_split_read_test.cpp @@ -151,7 +151,7 @@ class RawFileSplitReadTest : public ::testing::Test { FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); @@ -396,7 +396,7 @@ TEST_F(RawFileSplitReadTest, TestEmptyPlan) { FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); diff --git a/src/paimon/core/table/source/table_read.cpp b/src/paimon/core/table/source/table_read.cpp index 7a1504be..e2de90f5 100644 --- a/src/paimon/core/table/source/table_read.cpp +++ b/src/paimon/core/table/source/table_read.cpp @@ -96,7 +96,7 @@ Result> CreateTableRead( FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), memory_pool)); diff --git a/src/paimon/core/table/source/table_scan.cpp b/src/paimon/core/table/source/table_scan.cpp index 5fd42074..d7810b07 100644 --- a/src/paimon/core/table/source/table_scan.cpp +++ b/src/paimon/core/table/source/table_scan.cpp @@ -216,7 +216,7 @@ Result> TableScan::Create(std::unique_ptrGetPath(), arrow_schema, table_schema->PartitionKeys(), core_options.GetPartitionDefaultName(), - core_options.GetWriteFileFormat(/*level=*/0)->Identifier(), + core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), context->GetMemoryPool())); diff --git a/src/paimon/core/utils/file_store_path_factory_test.cpp b/src/paimon/core/utils/file_store_path_factory_test.cpp index bfd20657..c0bb7856 100644 --- a/src/paimon/core/utils/file_store_path_factory_test.cpp +++ b/src/paimon/core/utils/file_store_path_factory_test.cpp @@ -68,7 +68,7 @@ class FileStorePathFactoryTest : public ::testing::Test { EXPECT_OK_AND_ASSIGN(auto path_factory, FileStorePathFactory::Create( root, schema, {"f0", "f3"}, options.GetPartitionDefaultName(), - options.GetWriteFileFormat(/*level=*/0)->Identifier(), + options.GetFileFormat()->Identifier(), options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, /*global_index_external_path=*/std::nullopt, options.IndexFileInDataFileDir(), mem_pool_)); From 06633204edecd11318697e903a2f2ef4904c0bc6 Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Thu, 19 Mar 2026 18:06:42 +0800 Subject: [PATCH 05/10] fix --- src/paimon/common/sst/sst_file_reader.cpp | 4 +- src/paimon/core/core_options_test.cpp | 2 +- .../compact/changelog_merge_tree_rewriter.cpp | 9 +- .../compact/changelog_merge_tree_rewriter.h | 1 + .../lookup_merge_tree_compact_rewriter.cpp | 23 ++-- .../lookup_merge_tree_compact_rewriter.h | 3 +- ...ookup_merge_tree_compact_rewriter_test.cpp | 114 ++++++++++++++++-- .../compact/merge_tree_compact_rewriter.cpp | 19 +-- .../compact/merge_tree_compact_rewriter.h | 4 +- .../merge_tree_compact_rewriter_test.cpp | 8 +- 10 files changed, 144 insertions(+), 43 deletions(-) diff --git a/src/paimon/common/sst/sst_file_reader.cpp b/src/paimon/common/sst/sst_file_reader.cpp index 9d4c0db2..f6eb2add 100644 --- a/src/paimon/common/sst/sst_file_reader.cpp +++ b/src/paimon/common/sst/sst_file_reader.cpp @@ -131,13 +131,13 @@ Result> SstFileReader::ReadBlock( auto trailer_data = block_cache_->GetBlock(handle->Offset() + handle->Size(), BlockTrailer::ENCODED_LENGTH, true); if (!trailer_data) { - return Status::Invalid("read block failed"); + return Status::Invalid("Read trailer failed"); } auto trailer_input = MemorySlice::Wrap(trailer_data)->ToInput(); auto trailer = BlockTrailer::ReadBlockTrailer(trailer_input); auto block_data = block_cache_->GetBlock(handle->Offset(), handle->Size(), index); if (!block_data) { - return Status::Invalid("read block failed"); + return Status::Invalid("Read block failed"); } PAIMON_ASSIGN_OR_RAISE(std::shared_ptr uncompressed_data, DecompressBlock(block_data, trailer, pool_)); diff --git a/src/paimon/core/core_options_test.cpp b/src/paimon/core/core_options_test.cpp index 779398d0..9c589929 100644 --- a/src/paimon/core/core_options_test.cpp +++ b/src/paimon/core/core_options_test.cpp @@ -246,7 +246,7 @@ TEST(CoreOptionsTest, TestFromMap) { ASSERT_TRUE(core_options.NeedLookup()); LookupStrategy expected_lookup_strategy = {/*is_first_row=*/false, /*produce_changelog=*/false, - /*deletion_vector=*/true, /*need_lookup=*/true}; + /*deletion_vector=*/true, /*force_lookup=*/true}; ASSERT_EQ(expected_lookup_strategy, core_options.GetLookupStrategy()); std::map seq_grp; diff --git a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp index a06b52da..e77d34ad 100644 --- a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp +++ b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp @@ -20,21 +20,20 @@ ChangelogMergeTreeRewriter::ChangelogMergeTreeRewriter( int32_t max_level, bool force_drop_delete, const BinaryRow& partition, int32_t bucket, int64_t schema_id, const std::vector& trimmed_primary_keys, const CoreOptions& options, const std::shared_ptr& data_schema, - const std::shared_ptr& write_schema, + const std::shared_ptr& write_schema, DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, MergeFunctionWrapperFactory merge_function_wrapper_factory, const std::shared_ptr& pool) : MergeTreeCompactRewriter(partition, bucket, schema_id, trimmed_primary_keys, options, - data_schema, write_schema, path_factory_cache, + data_schema, write_schema, std::move(dv_factory), path_factory_cache, std::move(merge_file_split_read), std::move(merge_function_wrapper_factory), pool), max_level_(max_level), force_drop_delete_(force_drop_delete) {} Result ChangelogMergeTreeRewriter::Rewrite( - int32_t output_level, bool drop_delete, - const std::vector>& sections) { + int32_t output_level, bool drop_delete, const std::vector>& sections) { if (RewriteChangelog(output_level, drop_delete, sections)) { return RewriteOrProduceChangelog(output_level, sections, drop_delete, /*rewrite_compact_file=*/true); @@ -44,7 +43,7 @@ Result ChangelogMergeTreeRewriter::Rewrite( } Result ChangelogMergeTreeRewriter::Upgrade( - int32_t output_level, const std::shared_ptr& file) { + int32_t output_level, const std::shared_ptr& file) { UpgradeStrategy upgrade_strategy = GenerateUpgradeStrategy(output_level, file); if (upgrade_strategy.changelog) { return RewriteOrProduceChangelog(output_level, {{SortedRun::FromSingle(file)}}, diff --git a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h index b0bd5546..56a22a49 100644 --- a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h +++ b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h @@ -38,6 +38,7 @@ class ChangelogMergeTreeRewriter : public MergeTreeCompactRewriter { const CoreOptions& options, const std::shared_ptr& data_schema, const std::shared_ptr& write_schema, + DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, MergeFunctionWrapperFactory merge_function_wrapper_factory, diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp index 13e087c2..63a2c4e1 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.cpp @@ -30,15 +30,16 @@ LookupMergeTreeCompactRewriter::LookupMergeTreeCompactRewriter( const BinaryRow& partition, int32_t bucket, int64_t schema_id, const std::vector& trimmed_primary_keys, const CoreOptions& options, const std::shared_ptr& data_schema, - const std::shared_ptr& write_schema, + const std::shared_ptr& write_schema, DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, MergeFunctionWrapperFactory merge_function_wrapper_factory, const std::shared_ptr& pool) - : ChangelogMergeTreeRewriter( - max_level, /*force_drop_delete=*/dv_maintainer != nullptr, partition, bucket, schema_id, - trimmed_primary_keys, options, data_schema, write_schema, path_factory_cache, - std::move(merge_file_split_read), std::move(merge_function_wrapper_factory), pool), + : ChangelogMergeTreeRewriter(max_level, /*force_drop_delete=*/dv_maintainer != nullptr, + partition, bucket, schema_id, trimmed_primary_keys, options, + data_schema, write_schema, std::move(dv_factory), + path_factory_cache, std::move(merge_file_split_read), + std::move(merge_function_wrapper_factory), pool), lookup_levels_(std::move(lookup_levels)), dv_maintainer_(dv_maintainer) {} @@ -49,6 +50,7 @@ LookupMergeTreeCompactRewriter::Create( const std::shared_ptr& dv_maintainer, MergeFunctionWrapperFactory merge_function_wrapper_factory, int32_t bucket, const BinaryRow& partition, const std::shared_ptr& table_schema, + DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, const CoreOptions& options, const std::shared_ptr& pool) { PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, @@ -65,16 +67,17 @@ LookupMergeTreeCompactRewriter::Create( PAIMON_ASSIGN_OR_RAISE( std::shared_ptr internal_context, InternalReadContext::Create(read_context, table_schema, options.ToMap())); - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr path_factory, - path_factory_cache->GetOrCreatePathFactory( - options.GetFileFormat()->Identifier())); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr path_factory, + path_factory_cache->GetOrCreatePathFactory(options.GetFileFormat()->Identifier())); PAIMON_ASSIGN_OR_RAISE( std::unique_ptr merge_file_split_read, MergeFileSplitRead::Create(path_factory, internal_context, pool, CreateDefaultExecutor())); return std::unique_ptr(new LookupMergeTreeCompactRewriter( std::move(lookup_levels), dv_maintainer, max_level, partition, bucket, table_schema->Id(), - trimmed_primary_keys, options, data_schema, write_schema, path_factory_cache, - std::move(merge_file_split_read), std::move(merge_function_wrapper_factory), pool)); + trimmed_primary_keys, options, data_schema, write_schema, std::move(dv_factory), + path_factory_cache, std::move(merge_file_split_read), + std::move(merge_function_wrapper_factory), pool)); } template diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h index f5c51f30..eb090e51 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter.h @@ -34,6 +34,7 @@ class LookupMergeTreeCompactRewriter : public ChangelogMergeTreeRewriter { const std::shared_ptr& dv_maintainer, MergeFunctionWrapperFactory merge_function_wrapper_factory, int32_t bucket, const BinaryRow& partition, const std::shared_ptr& table_schema, + DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, const CoreOptions& options, const std::shared_ptr& pool); @@ -59,7 +60,7 @@ class LookupMergeTreeCompactRewriter : public ChangelogMergeTreeRewriter { const BinaryRow& partition, int32_t bucket, int64_t schema_id, const std::vector& trimmed_primary_keys, const CoreOptions& options, const std::shared_ptr& data_schema, - const std::shared_ptr& write_schema, + const std::shared_ptr& write_schema, DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, MergeFunctionWrapperFactory merge_function_wrapper_factory, diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp index 6be87ff9..19f5c058 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp @@ -138,6 +138,9 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { Result>> CreateCompactRewriterForFirstRow( const std::string& table_path, const std::shared_ptr& table_schema, const CoreOptions& options, std::unique_ptr>&& lookup_levels) const { + auto dv_factory = [](const std::string&) -> Result> { + return std::shared_ptr(); + }; auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); auto merge_function_wrapper_factory = @@ -154,13 +157,17 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { /*max_level=*/5, std::move(lookup_levels), /*dv_maintainer=*/nullptr, std::move(merge_function_wrapper_factory), /*bucket=*/0, - /*partition=*/BinaryRow::EmptyRow(), table_schema, path_factory_cache, options, pool_); + /*partition=*/BinaryRow::EmptyRow(), table_schema, std::move(dv_factory), + path_factory_cache, options, pool_); } Result>> CreateCompactRewriterForKeyValue( const std::string& table_path, const std::shared_ptr& table_schema, const CoreOptions& options, std::unique_ptr>&& lookup_levels) const { + auto dv_factory = [](const std::string&) -> Result> { + return std::shared_ptr(); + }; auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); auto merge_function_wrapper_factory = @@ -183,14 +190,20 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { return LookupMergeTreeCompactRewriter::Create( /*max_level=*/5, std::move(lookup_levels), /*dv_maintainer=*/nullptr, std::move(merge_function_wrapper_factory), /*bucket=*/0, - /*partition=*/BinaryRow::EmptyRow(), table_schema, path_factory_cache, options, pool_); + /*partition=*/BinaryRow::EmptyRow(), table_schema, std::move(dv_factory), + path_factory_cache, options, pool_); } Result>> CreateCompactRewriterForFilePosition( const std::string& table_path, const std::shared_ptr& table_schema, - const CoreOptions& options, - std::unique_ptr>&& lookup_levels) const { + const CoreOptions& options, std::unique_ptr>&& lookup_levels, + DeletionVector::Factory dv_factory = nullptr) const { + if (!dv_factory) { + dv_factory = [](const std::string&) -> Result> { + return std::shared_ptr(); + }; + } auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); @@ -218,7 +231,8 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { return LookupMergeTreeCompactRewriter::Create( /*max_level=*/5, std::move(lookup_levels), dv_maintainer, std::move(merge_function_wrapper_factory), /*bucket=*/0, - /*partition=*/BinaryRow::EmptyRow(), table_schema, path_factory_cache, options, pool_); + /*partition=*/BinaryRow::EmptyRow(), table_schema, std::move(dv_factory), + path_factory_cache, options, pool_); } Result>> @@ -226,6 +240,10 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { const std::string& table_path, const std::shared_ptr& table_schema, const CoreOptions& options, std::unique_ptr>&& lookup_levels) const { + auto dv_factory = [](const std::string&) -> Result> { + return std::shared_ptr(); + }; + auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); @@ -257,7 +275,8 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { return LookupMergeTreeCompactRewriter::Create( /*max_level=*/5, std::move(lookup_levels), dv_maintainer, std::move(merge_function_wrapper_factory), /*bucket=*/0, - /*partition=*/BinaryRow::EmptyRow(), table_schema, path_factory_cache, options, pool_); + /*partition=*/BinaryRow::EmptyRow(), table_schema, std::move(dv_factory), + path_factory_cache, options, pool_); } void CheckResult(const std::string& compact_file_name, @@ -824,6 +843,77 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestDvUpgradeWithoutLookup) { ASSERT_FALSE(dv); } +TEST_F(LookupMergeTreeCompactRewriterTest, TestRewriteWithDvFactory) { + std::map options = { + {Options::MERGE_ENGINE, "deduplicate"}, + {Options::FILE_FORMAT, "orc"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 3 files + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/5, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33], [5, 55]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/1, /*last_sequence_number=*/2, table_path, + core_options, "[[2, 22], [4, 44], [5, 5]]")); + ASSERT_OK_AND_ASSIGN(auto file2, NewFiles(/*level=*/0, /*last_sequence_number=*/5, table_path, + core_options, "[[2, 222], [5, 555]]")); + std::vector> files = {file0, file1, file2}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN( + auto lookup_levels, + CreateLookupLevels(table_path, table_schema, processor_factory, files)); + auto dv_factory = [file_name = file1->file_name]( + const std::string& name) -> Result> { + if (name == file_name) { + // rm the second row for file1 + auto dv = std::make_shared(RoaringBitmap32()); + PAIMON_RETURN_NOT_OK(dv->Delete(1)); + return dv; + } + return std::shared_ptr(); + }; + + // compact and rewrite + ASSERT_OK_AND_ASSIGN(auto rewriter, CreateCompactRewriterForFilePosition( + table_path, table_schema, core_options, + std::move(lookup_levels), std::move(dv_factory))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns({file1, file2})); + + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Rewrite( + /*output_level=*/4, /*drop_delete=*/true, runs)); + ASSERT_EQ(2, compact_result.Before().size()); + ASSERT_EQ(1, compact_result.After().size()); + + const auto& compact_file_meta = compact_result.After()[0]; + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = + arrow::struct_(SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[6, 0, 2, 222], +[7, 0, 5, 555] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "orc", expected_array); + + // test dv + auto dv_maintainer = rewriter->dv_maintainer_; + ASSERT_TRUE(dv_maintainer); + auto dv = dv_maintainer->DeletionVectorOf(file0->file_name); + ASSERT_FALSE(dv); +} + TEST_F(LookupMergeTreeCompactRewriterTest, TestIOException) { std::map options = { {Options::MERGE_ENGINE, "aggregation"}, @@ -931,7 +1021,7 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, - /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); auto file = create_meta(/*level=*/1, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::NoChangelogNoRewrite(), @@ -946,7 +1036,7 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, - /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), @@ -964,7 +1054,7 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*lookup_levels=*/nullptr, dv_maintainer, /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, - /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); auto file = create_meta(/*level=*/0, /*delete_row_count=*/1); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), @@ -978,7 +1068,7 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, - /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogNoRewrite(), @@ -992,7 +1082,7 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, - /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogNoRewrite(), @@ -1007,7 +1097,7 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*lookup_levels=*/nullptr, /*dv_maintainer=*/nullptr, /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, - /*write_schema=*/nullptr, /*path_factory_cache=*/nullptr, + /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp index b4dd9345..3adfeeb8 100644 --- a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp @@ -34,7 +34,7 @@ MergeTreeCompactRewriter::MergeTreeCompactRewriter( const BinaryRow& partition, int32_t bucket, int64_t schema_id, const std::vector& trimmed_primary_keys, const CoreOptions& options, const std::shared_ptr& data_schema, - const std::shared_ptr& write_schema, + const std::shared_ptr& write_schema, DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, MergeFunctionWrapperFactory merge_function_wrapper_factory, @@ -48,11 +48,13 @@ MergeTreeCompactRewriter::MergeTreeCompactRewriter( trimmed_primary_keys_(trimmed_primary_keys), data_schema_(data_schema), write_schema_(write_schema), + dv_factory_(std::move(dv_factory)), path_factory_cache_(path_factory_cache), merge_function_wrapper_factory_(std::move(merge_function_wrapper_factory)) {} Result> MergeTreeCompactRewriter::Create( int32_t bucket, const BinaryRow& partition, const std::shared_ptr& table_schema, + DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, const CoreOptions& options, const std::shared_ptr& pool) { PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, @@ -69,9 +71,9 @@ Result> MergeTreeCompactRewriter::Crea PAIMON_ASSIGN_OR_RAISE( std::shared_ptr internal_context, InternalReadContext::Create(read_context, table_schema, options.ToMap())); - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr path_factory, - path_factory_cache->GetOrCreatePathFactory( - options.GetFileFormat()->Identifier())); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr path_factory, + path_factory_cache->GetOrCreatePathFactory(options.GetFileFormat()->Identifier())); PAIMON_ASSIGN_OR_RAISE( std::unique_ptr merge_file_split_read, MergeFileSplitRead::Create(path_factory, internal_context, pool, CreateDefaultExecutor())); @@ -81,7 +83,7 @@ Result> MergeTreeCompactRewriter::Crea }; return std::unique_ptr(new MergeTreeCompactRewriter( partition, bucket, table_schema->Id(), trimmed_primary_keys, options, data_schema, - write_schema, path_factory_cache, std::move(merge_file_split_read), + write_schema, std::move(dv_factory), path_factory_cache, std::move(merge_file_split_read), merge_function_wrapper_factory, pool)); } @@ -180,15 +182,14 @@ Status MergeTreeCompactRewriter::MergeReadAndWrite( } auto& reader_holders = *reader_holders_ptr; // prepare sort merge reader - PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr data_file_path_factory, - CreateDataFilePathFactory(options_.GetFileFormat()->Identifier())); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr data_file_path_factory, + CreateDataFilePathFactory(options_.GetFileFormat()->Identifier())); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr> wrapper, merge_function_wrapper_factory_(output_level)); merge_file_split_read_->SetMergeFunctionWrapper(wrapper); PAIMON_ASSIGN_OR_RAISE(std::unique_ptr sort_merge_reader, merge_file_split_read_->CreateSortMergeReaderForSection( - section, partition_, /*deletion_file_map=*/{}, + section, partition_, dv_factory_, /*predicate=*/nullptr, data_file_path_factory, drop_delete)); // consumer batch size is WriteBatchSize diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.h b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.h index 08bb8e15..4229d365 100644 --- a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.h +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.h @@ -36,7 +36,7 @@ class MergeTreeCompactRewriter : public CompactRewriter { static Result> Create( int32_t bucket, const BinaryRow& partition, - const std::shared_ptr& table_schema, + const std::shared_ptr& table_schema, DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, const CoreOptions& options, const std::shared_ptr& memory_pool); @@ -70,6 +70,7 @@ class MergeTreeCompactRewriter : public CompactRewriter { const CoreOptions& options, const std::shared_ptr& data_schema, const std::shared_ptr& write_schema, + DeletionVector::Factory dv_factory, const std::shared_ptr& path_factory_cache, std::unique_ptr&& merge_file_split_read, MergeFunctionWrapperFactory merge_function_wrapper_factory, @@ -109,6 +110,7 @@ class MergeTreeCompactRewriter : public CompactRewriter { std::shared_ptr data_schema_; // SequenceNumber + ValueKind + data_schema_ std::shared_ptr write_schema_; + DeletionVector::Factory dv_factory_; std::shared_ptr path_factory_cache_; MergeFunctionWrapperFactory merge_function_wrapper_factory_; }; diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter_test.cpp b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter_test.cpp index 082907e7..3c4f40db 100644 --- a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter_test.cpp +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter_test.cpp @@ -40,10 +40,14 @@ class MergeTreeCompactRewriterTest : public testing::Test { int32_t bucket, const BinaryRow& partition) const { PAIMON_ASSIGN_OR_RAISE(auto options, CoreOptions::FromMap(table_schema->Options())); auto arrow_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); + auto dv_factory = [](const std::string&) -> Result> { + return std::shared_ptr(); + }; + auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); - return MergeTreeCompactRewriter::Create(bucket, partition, table_schema, path_factory_cache, - options, pool_); + return MergeTreeCompactRewriter::Create(bucket, partition, table_schema, dv_factory, + path_factory_cache, options, pool_); } Result>> GenerateSortedRuns( From 5826802c83b80a5acf202e94dd1314f5aece2bb2 Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Thu, 19 Mar 2026 18:37:49 +0800 Subject: [PATCH 06/10] fix --- .../core/global_index/global_index_scan_impl.cpp | 7 +++---- .../core/global_index/global_index_write_task.cpp | 13 ++++++------- src/paimon/core/operation/file_store_commit.cpp | 11 +++++------ .../operation/key_value_file_store_scan_test.cpp | 13 ++++++------- .../core/operation/merge_file_split_read_test.cpp | 3 +-- src/paimon/core/operation/orphan_files_cleaner.cpp | 11 +++++------ .../core/operation/raw_file_split_read_test.cpp | 10 +++------- src/paimon/core/table/source/table_read.cpp | 3 +-- src/paimon/core/table/source/table_scan.cpp | 3 +-- .../core/utils/file_store_path_factory_test.cpp | 6 +++--- 10 files changed, 34 insertions(+), 46 deletions(-) diff --git a/src/paimon/core/global_index/global_index_scan_impl.cpp b/src/paimon/core/global_index/global_index_scan_impl.cpp index 53237e4a..0e61cff3 100644 --- a/src/paimon/core/global_index/global_index_scan_impl.cpp +++ b/src/paimon/core/global_index/global_index_scan_impl.cpp @@ -110,10 +110,9 @@ Status GlobalIndexScanImpl::Scan() { path_factory_, FileStorePathFactory::Create( root_path_, arrow_schema, table_schema_->PartitionKeys(), - options_.GetPartitionDefaultName(), - options_.GetFileFormat()->Identifier(), options_.DataFilePrefix(), - options_.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, - options_.IndexFileInDataFileDir(), pool_)); + options_.GetPartitionDefaultName(), options_.GetFileFormat()->Identifier(), + options_.DataFilePrefix(), options_.LegacyPartitionNameEnabled(), external_paths, + global_index_external_path, options_.IndexFileInDataFileDir(), pool_)); PAIMON_ASSIGN_OR_RAISE( std::unique_ptr index_manifest_file, diff --git a/src/paimon/core/global_index/global_index_write_task.cpp b/src/paimon/core/global_index/global_index_write_task.cpp index 3654cead..6ba17a74 100644 --- a/src/paimon/core/global_index/global_index_write_task.cpp +++ b/src/paimon/core/global_index/global_index_write_task.cpp @@ -43,13 +43,12 @@ Result> CreateGlobalIndexFileManager( core_options.CreateGlobalIndexExternalPath()); PAIMON_ASSIGN_OR_RAISE( std::shared_ptr path_factory, - FileStorePathFactory::Create(table_path, all_arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), - core_options.GetFileFormat()->Identifier(), - core_options.DataFilePrefix(), - core_options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, - core_options.IndexFileInDataFileDir(), pool)); + FileStorePathFactory::Create( + table_path, all_arrow_schema, table_schema->PartitionKeys(), + core_options.GetPartitionDefaultName(), core_options.GetFileFormat()->Identifier(), + core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), + external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), + pool)); std::shared_ptr index_path_factory = path_factory->CreateGlobalIndexFileFactory(); return std::make_shared(core_options.GetFileSystem(), diff --git a/src/paimon/core/operation/file_store_commit.cpp b/src/paimon/core/operation/file_store_commit.cpp index a0176086..b2f87bf3 100644 --- a/src/paimon/core/operation/file_store_commit.cpp +++ b/src/paimon/core/operation/file_store_commit.cpp @@ -96,12 +96,11 @@ Result> FileStoreCommit::Create( PAIMON_ASSIGN_OR_RAISE( std::shared_ptr path_factory, - FileStorePathFactory::Create(root_path, arrow_schema, table_schema.value()->PartitionKeys(), - options.GetPartitionDefaultName(), - options.GetFileFormat()->Identifier(), - options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), - external_paths, global_index_external_path, - options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); + FileStorePathFactory::Create( + root_path, arrow_schema, table_schema.value()->PartitionKeys(), + options.GetPartitionDefaultName(), options.GetFileFormat()->Identifier(), + options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, + global_index_external_path, options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); auto snapshot_manager = std::make_shared(options.GetFileSystem(), root_path); PAIMON_ASSIGN_OR_RAISE( diff --git a/src/paimon/core/operation/key_value_file_store_scan_test.cpp b/src/paimon/core/operation/key_value_file_store_scan_test.cpp index cc0f7d1d..569558ee 100644 --- a/src/paimon/core/operation/key_value_file_store_scan_test.cpp +++ b/src/paimon/core/operation/key_value_file_store_scan_test.cpp @@ -83,13 +83,12 @@ class KeyValueFileStoreScanTest : public testing::Test { PAIMON_ASSIGN_OR_RAISE( std::shared_ptr path_factory, - FileStorePathFactory::Create(table_path, arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), - core_options.GetFileFormat()->Identifier(), - core_options.DataFilePrefix(), - core_options.LegacyPartitionNameEnabled(), external_paths, - global_index_external_path, - core_options.IndexFileInDataFileDir(), pool_)); + FileStorePathFactory::Create( + table_path, arrow_schema, table_schema->PartitionKeys(), + core_options.GetPartitionDefaultName(), core_options.GetFileFormat()->Identifier(), + core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), + external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), + pool_)); auto manifest_file_format = core_options.GetManifestFormat(); auto snapshot_manager = std::make_shared(fs, table_path); diff --git a/src/paimon/core/operation/merge_file_split_read_test.cpp b/src/paimon/core/operation/merge_file_split_read_test.cpp index a20265d7..e5baf5e3 100644 --- a/src/paimon/core/operation/merge_file_split_read_test.cpp +++ b/src/paimon/core/operation/merge_file_split_read_test.cpp @@ -340,8 +340,7 @@ class MergeFileSplitReadTest : public ::testing::Test, std::shared_ptr path_factory, FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), - core_options.GetFileFormat()->Identifier(), + core_options.GetPartitionDefaultName(), core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); diff --git a/src/paimon/core/operation/orphan_files_cleaner.cpp b/src/paimon/core/operation/orphan_files_cleaner.cpp index c581a14b..c3a001cc 100644 --- a/src/paimon/core/operation/orphan_files_cleaner.cpp +++ b/src/paimon/core/operation/orphan_files_cleaner.cpp @@ -185,12 +185,11 @@ Result> OrphanFilesCleaner::Create( PAIMON_ASSIGN_OR_RAISE( std::shared_ptr path_factory, - FileStorePathFactory::Create(ctx->GetRootPath(), arrow_schema, schema->PartitionKeys(), - options.GetPartitionDefaultName(), - options.GetFileFormat()->Identifier(), - options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), - external_paths, global_index_external_path, - options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); + FileStorePathFactory::Create( + ctx->GetRootPath(), arrow_schema, schema->PartitionKeys(), + options.GetPartitionDefaultName(), options.GetFileFormat()->Identifier(), + options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), external_paths, + global_index_external_path, options.IndexFileInDataFileDir(), ctx->GetMemoryPool())); auto snapshot_manager = std::make_shared(options.GetFileSystem(), ctx->GetRootPath()); PAIMON_ASSIGN_OR_RAISE( diff --git a/src/paimon/core/operation/raw_file_split_read_test.cpp b/src/paimon/core/operation/raw_file_split_read_test.cpp index be775589..d4879242 100644 --- a/src/paimon/core/operation/raw_file_split_read_test.cpp +++ b/src/paimon/core/operation/raw_file_split_read_test.cpp @@ -150,8 +150,7 @@ class RawFileSplitReadTest : public ::testing::Test { std::shared_ptr path_factory, FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), - core_options.GetFileFormat()->Identifier(), + core_options.GetPartitionDefaultName(), core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); @@ -395,8 +394,7 @@ TEST_F(RawFileSplitReadTest, TestEmptyPlan) { std::shared_ptr path_factory, FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), - core_options.GetFileFormat()->Identifier(), + core_options.GetPartitionDefaultName(), core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), pool_)); @@ -497,9 +495,7 @@ TEST_F(RawFileSplitReadTest, TestMatch) { split_read->Match(data_split, /*force_keep_delete=*/false)); ASSERT_FALSE(match_result); } - { - ASSERT_NOK(split_read->Match(nullptr, /*force_keep_delete=*/false)); - } + { ASSERT_NOK(split_read->Match(nullptr, /*force_keep_delete=*/false)); } } } // namespace paimon::test diff --git a/src/paimon/core/table/source/table_read.cpp b/src/paimon/core/table/source/table_read.cpp index e2de90f5..cc7e646f 100644 --- a/src/paimon/core/table/source/table_read.cpp +++ b/src/paimon/core/table/source/table_read.cpp @@ -95,8 +95,7 @@ Result> CreateTableRead( std::shared_ptr path_factory, FileStorePathFactory::Create( internal_context->GetPath(), arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), - core_options.GetFileFormat()->Identifier(), + core_options.GetPartitionDefaultName(), core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), memory_pool)); diff --git a/src/paimon/core/table/source/table_scan.cpp b/src/paimon/core/table/source/table_scan.cpp index d7810b07..ba1b5d8c 100644 --- a/src/paimon/core/table/source/table_scan.cpp +++ b/src/paimon/core/table/source/table_scan.cpp @@ -215,8 +215,7 @@ Result> TableScan::Create(std::unique_ptr path_factory, FileStorePathFactory::Create( context->GetPath(), arrow_schema, table_schema->PartitionKeys(), - core_options.GetPartitionDefaultName(), - core_options.GetFileFormat()->Identifier(), + core_options.GetPartitionDefaultName(), core_options.GetFileFormat()->Identifier(), core_options.DataFilePrefix(), core_options.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, core_options.IndexFileInDataFileDir(), context->GetMemoryPool())); diff --git a/src/paimon/core/utils/file_store_path_factory_test.cpp b/src/paimon/core/utils/file_store_path_factory_test.cpp index c0bb7856..7c47cb8c 100644 --- a/src/paimon/core/utils/file_store_path_factory_test.cpp +++ b/src/paimon/core/utils/file_store_path_factory_test.cpp @@ -68,9 +68,9 @@ class FileStorePathFactoryTest : public ::testing::Test { EXPECT_OK_AND_ASSIGN(auto path_factory, FileStorePathFactory::Create( root, schema, {"f0", "f3"}, options.GetPartitionDefaultName(), - options.GetFileFormat()->Identifier(), - options.DataFilePrefix(), options.LegacyPartitionNameEnabled(), - external_paths, /*global_index_external_path=*/std::nullopt, + options.GetFileFormat()->Identifier(), options.DataFilePrefix(), + options.LegacyPartitionNameEnabled(), external_paths, + /*global_index_external_path=*/std::nullopt, options.IndexFileInDataFileDir(), mem_pool_)); return path_factory; } From 2ef3d3a15579ac1c4d66e73b360ce54cc53ffaac Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Thu, 19 Mar 2026 10:57:22 +0000 Subject: [PATCH 07/10] fix0657 --- src/paimon/core/operation/raw_file_split_read_test.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/paimon/core/operation/raw_file_split_read_test.cpp b/src/paimon/core/operation/raw_file_split_read_test.cpp index d4879242..e1b0b995 100644 --- a/src/paimon/core/operation/raw_file_split_read_test.cpp +++ b/src/paimon/core/operation/raw_file_split_read_test.cpp @@ -495,7 +495,9 @@ TEST_F(RawFileSplitReadTest, TestMatch) { split_read->Match(data_split, /*force_keep_delete=*/false)); ASSERT_FALSE(match_result); } - { ASSERT_NOK(split_read->Match(nullptr, /*force_keep_delete=*/false)); } + { + ASSERT_NOK(split_read->Match(nullptr, /*force_keep_delete=*/false)); + } } } // namespace paimon::test From f47c35452af0f847577f2dd038e24a0a6ef24d27 Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Thu, 19 Mar 2026 21:27:49 +0800 Subject: [PATCH 08/10] remove copy constructor for KeyValue --- src/paimon/common/data/generic_row.h | 6 +++--- src/paimon/core/key_value.h | 20 ++----------------- .../lookup_changelog_merge_function_wrapper.h | 13 ++++++------ .../mergetree/compact/lookup_merge_function.h | 19 +++++------------- .../compact/lookup_merge_function_test.cpp | 10 +++------- 5 files changed, 20 insertions(+), 48 deletions(-) diff --git a/src/paimon/common/data/generic_row.h b/src/paimon/common/data/generic_row.h index 6c9070a7..30409005 100644 --- a/src/paimon/common/data/generic_row.h +++ b/src/paimon/common/data/generic_row.h @@ -103,8 +103,8 @@ class GenericRow : public InternalRow { return DataDefine::IsVariantNull(fields_[pos]); } - void AddDataHolder(const std::shared_ptr& holder) { - holders_.push_back(holder); + void AddDataHolder(std::unique_ptr&& holder) { + holders_.push_back(std::move(holder)); } bool GetBoolean(int32_t pos) const override { @@ -219,7 +219,7 @@ class GenericRow : public InternalRow { std::vector fields_; /// As GenericRow only holds string view for string data to avoid deep copy, original data must /// be held in holders_ - std::vector> holders_; + std::vector> holders_; /// The kind of change that a row describes in a changelog. const RowKind* kind_; }; diff --git a/src/paimon/core/key_value.h b/src/paimon/core/key_value.h index e020d88d..65369e00 100644 --- a/src/paimon/core/key_value.h +++ b/src/paimon/core/key_value.h @@ -38,29 +38,13 @@ struct KeyValue { KeyValue() = default; KeyValue(const RowKind* _value_kind, int64_t _sequence_number, int32_t _level, - std::shared_ptr _key, std::shared_ptr _value) + std::shared_ptr _key, std::unique_ptr _value) : value_kind(_value_kind), sequence_number(_sequence_number), level(_level), key(std::move(_key)), value(std::move(_value)) {} - KeyValue(const KeyValue& other) noexcept { - *this = other; - } - - KeyValue& operator=(const KeyValue& other) noexcept { - if (&other == this) { - return *this; - } - value_kind = other.value_kind; - sequence_number = other.sequence_number; - level = other.level; - key = other.key; - value = other.value; - return *this; - } - KeyValue(KeyValue&& other) noexcept { *this = std::move(other); } @@ -83,7 +67,7 @@ struct KeyValue { // determined after read from file int32_t level = -1; std::shared_ptr key; - std::shared_ptr value; + std::unique_ptr value; }; struct KeyValueBatch { diff --git a/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h b/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h index 5a8aa1c0..f217108e 100644 --- a/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h +++ b/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper.h @@ -70,24 +70,25 @@ class LookupChangelogMergeFunctionWrapper : public MergeFunctionWrapper> GetResult() override { // 1. Find the latest high level record and compute containLevel0 - std::optional high_level = merge_function_->PickHighLevel(); + std::optional high_level_idx = merge_function_->PickHighLevelIdx(); // 2. Lookup if latest high level record is absent - if (high_level == std::nullopt) { + if (high_level_idx == std::nullopt) { + std::optional lookup_high_level; PAIMON_ASSIGN_OR_RAISE(std::optional lookup_result, lookup_(merge_function_->GetKey())); if (lookup_result) { std::string file_name; int64_t row_position = -1; if constexpr (std::is_same_v) { - high_level = lookup_result->key_value; + lookup_high_level = std::move(lookup_result->key_value); file_name = lookup_result->file_name; row_position = lookup_result->row_position; } else if constexpr (std::is_same_v) { file_name = lookup_result->file_name; row_position = lookup_result->row_position; } else if constexpr (std::is_same_v) { - high_level = lookup_result; + lookup_high_level = std::move(lookup_result); } else { return Status::Invalid( "deletion vector mode must have PositionedKeyValue or FilePosition " @@ -98,8 +99,8 @@ class LookupChangelogMergeFunctionWrapper : public MergeFunctionWrapperNotifyNewDeletion(file_name, row_position)); } } - if (high_level) { - merge_function_->InsertInto(std::move(high_level), comparator_); + if (lookup_high_level) { + merge_function_->InsertInto(std::move(lookup_high_level), comparator_); } } diff --git a/src/paimon/core/mergetree/compact/lookup_merge_function.h b/src/paimon/core/mergetree/compact/lookup_merge_function.h index 381b1e9d..87f81b4c 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_function.h +++ b/src/paimon/core/mergetree/compact/lookup_merge_function.h @@ -58,21 +58,13 @@ class LookupMergeFunction : public MergeFunction { return current_key_; } - std::optional PickHighLevel() const { - int32_t idx = PickHighLevelIdx(); - if (idx == -1) { - return std::optional(); - } - return candidates_[idx]; - } - Result> GetResult() override { merge_function_->Reset(); - int32_t high_level_idx = PickHighLevelIdx(); + std::optional high_level_idx = PickHighLevelIdx(); for (int32_t i = 0; i < static_cast(candidates_.size()); ++i) { // records that has not been stored on the disk yet, such as the data in the write // buffer being at level -1 - if (candidates_[i].level <= 0 || i == high_level_idx) { + if (candidates_[i].level <= 0 || i == high_level_idx.value()) { PAIMON_RETURN_NOT_OK(merge_function_->Add(std::move(candidates_[i]))); } } @@ -88,9 +80,8 @@ class LookupMergeFunction : public MergeFunction { std::sort(candidates_.begin(), candidates_.end(), cmp_function); } - private: - int32_t PickHighLevelIdx() const { - int32_t high_level_idx = -1; + std::optional PickHighLevelIdx() const { + std::optional high_level_idx; for (int32_t i = 0; i < static_cast(candidates_.size()); i++) { const auto& kv = candidates_[i]; // records that has not been stored on the disk yet, such as the data in the write @@ -100,7 +91,7 @@ class LookupMergeFunction : public MergeFunction { } // For high-level comparison logic (not involving Level 0), only the value of the // minimum Level should be selected - if (high_level_idx == -1 || kv.level < candidates_[high_level_idx].level) { + if (!high_level_idx || kv.level < candidates_[high_level_idx.value()].level) { high_level_idx = i; } } diff --git a/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp b/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp index 9336a0d2..5eb27421 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp +++ b/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp @@ -127,7 +127,7 @@ TEST(LookupMergeFunctionTest, TestPickHighLevel) { auto pool = GetDefaultPool(); merge_func->Reset(); - ASSERT_FALSE(merge_func->PickHighLevel()); + ASSERT_FALSE(merge_func->PickHighLevelIdx()); ASSERT_FALSE(merge_func->ContainLevel0()); merge_func->Reset(); @@ -145,12 +145,8 @@ TEST(LookupMergeFunctionTest, TestPickHighLevel) { ASSERT_OK(merge_func->Add(std::move(kv2))); ASSERT_OK(merge_func->Add(std::move(kv3))); ASSERT_TRUE(merge_func->ContainLevel0()); - ASSERT_EQ(merge_func->GetKey()->GetInt(0), 10); - auto result_kv = std::move(merge_func->PickHighLevel().value()); - KeyValue expected(RowKind::Insert(), /*sequence_number=*/2, /*level=*/1, /*key=*/ - BinaryRowGenerator::GenerateRowPtr({10}, pool.get()), - /*value=*/BinaryRowGenerator::GenerateRowPtr({10, 200}, pool.get())); - KeyValueChecker::CheckResult(expected, result_kv, /*key_arity=*/1, /*value_arity=*/2); + ASSERT_EQ(merge_func->GetKey()->GetInt(0), 10); + ASSERT_EQ(merge_func->PickHighLevelIdx(), 1); } TEST(LookupMergeFunctionTest, TestInsertInto) { From 48a2190247c8f9f47559d60a4e89e884fbef404f Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Fri, 20 Mar 2026 09:08:39 +0800 Subject: [PATCH 09/10] fix --- .../core/mergetree/compact/first_row_merge_function_wrapper.h | 2 ++ .../core/mergetree/compact/lookup_merge_function_test.cpp | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper.h b/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper.h index 1f357024..80ae7ad6 100644 --- a/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper.h +++ b/src/paimon/core/mergetree/compact/first_row_merge_function_wrapper.h @@ -46,12 +46,14 @@ class FirstRowMergeFunctionWrapper : public MergeFunctionWrapper { Result> GetResult() override { PAIMON_ASSIGN_OR_RAISE(std::optional result, merge_function_->GetResult()); if (merge_function_->ContainsHighLevel()) { + Reset(); return result; } assert(result); PAIMON_ASSIGN_OR_RAISE(bool contains, contains_(result.value().key)); if (contains) { // empty + Reset(); return std::optional(); } // new record, output changelog diff --git a/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp b/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp index 5eb27421..54098de2 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp +++ b/src/paimon/core/mergetree/compact/lookup_merge_function_test.cpp @@ -145,7 +145,7 @@ TEST(LookupMergeFunctionTest, TestPickHighLevel) { ASSERT_OK(merge_func->Add(std::move(kv2))); ASSERT_OK(merge_func->Add(std::move(kv3))); ASSERT_TRUE(merge_func->ContainLevel0()); - ASSERT_EQ(merge_func->GetKey()->GetInt(0), 10); + ASSERT_EQ(merge_func->GetKey()->GetInt(0), 10); ASSERT_EQ(merge_func->PickHighLevelIdx(), 1); } From f4d5624782022c03882bcb98e4e0e13dc3373cfe Mon Sep 17 00:00:00 2001 From: "lisizhuo.lsz" Date: Fri, 20 Mar 2026 06:56:16 +0000 Subject: [PATCH 10/10] fix0256 --- .../row_compacted_serializer_test.cpp | 32 ++++++++++ ..._changelog_merge_function_wrapper_test.cpp | 18 ++++++ ...ookup_merge_tree_compact_rewriter_test.cpp | 59 +++++++++++++++++++ src/paimon/core/mergetree/levels_test.cpp | 21 +++++++ 4 files changed, 130 insertions(+) diff --git a/src/paimon/common/data/serializer/row_compacted_serializer_test.cpp b/src/paimon/common/data/serializer/row_compacted_serializer_test.cpp index 3189b291..6d183d6e 100644 --- a/src/paimon/common/data/serializer/row_compacted_serializer_test.cpp +++ b/src/paimon/common/data/serializer/row_compacted_serializer_test.cpp @@ -506,6 +506,38 @@ TEST(RowCompactedSerializerTest, TestNestedNullWithTimestampAndDecimal2) { } } +TEST(RowCompactedSerializerTest, TestListType) { + auto pool = GetDefaultPool(); + // prepare data + auto inner_child1 = arrow::field("inner1", arrow::list(arrow::int32())); + auto arrow_type = arrow::struct_({inner_child1}); + // each inner child per row + std::shared_ptr array = arrow::ipc::internal::json::ArrayFromJSON(arrow_type, + R"([ +[[5, 6, 7]], +[[1, 2, 3]], +[[4]] + ])") + .ValueOrDie(); + auto struct_array = std::dynamic_pointer_cast(array); + ASSERT_TRUE(struct_array); + auto columnar_row = + std::make_shared(/*struct_array=*/nullptr, struct_array->fields(), pool, + /*row_id=*/0); + + // serialize and deserialize + ASSERT_OK_AND_ASSIGN(auto serializer, + RowCompactedSerializer::Create(arrow::schema(arrow_type->fields()), pool)); + ASSERT_OK_AND_ASSIGN(auto bytes, serializer->SerializeToBytes(*columnar_row)); + ASSERT_OK_AND_ASSIGN(auto row, serializer->Deserialize(bytes)); + + // check result + ASSERT_EQ(row->GetFieldCount(), 1); + + // for inner_child1 + ASSERT_EQ(row->GetArray(0)->ToIntArray().value(), std::vector({5, 6, 7})); +} + TEST(RowCompactedSerializerTest, TestSliceComparator) { auto pool = GetDefaultPool(); arrow::FieldVector fields = { diff --git a/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp b/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp index a4ca1d18..f9c4741e 100644 --- a/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp +++ b/src/paimon/core/mergetree/compact/lookup_changelog_merge_function_wrapper_test.cpp @@ -30,6 +30,24 @@ #include "paimon/testing/utils/testharness.h" namespace paimon::test { +TEST(LookupChangelogMergeFunctionWrapperTest, TestCreateInvalid) { + auto pool = GetDefaultPool(); + auto mfunc = std::make_unique(/*ignore_delete=*/true); + auto lookup_mfunc = std::make_unique(std::move(mfunc)); + auto lookup = [&](const std::shared_ptr& key) -> Result> { + return std::optional( + KeyValue(RowKind::Insert(), /*sequence_number=*/1000, /*level=*/3, key, + BinaryRowGenerator::GenerateRowPtr({1001}, pool.get()))); + }; + LookupStrategy lookup_strategy(/*is_first_row=*/false, /*produce_changelog=*/false, + /*deletion_vector=*/true, /*force_lookup=*/true); + ASSERT_NOK_WITH_MSG(LookupChangelogMergeFunctionWrapper::Create( + std::move(lookup_mfunc), lookup, lookup_strategy, + /*deletion_vectors_maintainer=*/nullptr, + /*comparator=*/nullptr), + "deletionVectorsMaintainer should not be null, there is a bug."); +} + TEST(LookupChangelogMergeFunctionWrapperTest, TestSimple) { auto pool = GetDefaultPool(); KeyValue kv1(RowKind::Insert(), /*sequence_number=*/0, /*level=*/2, /*key=*/ diff --git a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp index 19f5c058..5c93d319 100644 --- a/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp +++ b/src/paimon/core/mergetree/compact/lookup_merge_tree_compact_rewriter_test.cpp @@ -1104,4 +1104,63 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); } } + +TEST_F(LookupMergeTreeCompactRewriterTest, TestRewriteLookupChangelogWithOutputLevelZero) { + std::map options = { + {Options::MERGE_ENGINE, "deduplicate"}, + {Options::FILE_FORMAT, "orc"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_OK_AND_ASSIGN(auto table_path, CreateTable(options)); + auto schema_manager = std::make_shared(fs_, table_path); + ASSERT_OK_AND_ASSIGN(auto table_schema, schema_manager->ReadSchema(0)); + + // write 2 files with level 0 + ASSERT_OK_AND_ASSIGN(auto file0, NewFiles(/*level=*/0, /*last_sequence_number=*/-1, table_path, + core_options, "[[1, 11], [3, 33]]")); + ASSERT_OK_AND_ASSIGN(auto file1, NewFiles(/*level=*/0, /*last_sequence_number=*/1, table_path, + core_options, "[[2, 22], [4, 44]]")); + std::vector> files = {file0, file1}; + auto processor_factory = std::make_shared(); + ASSERT_OK_AND_ASSIGN( + auto lookup_levels, + CreateLookupLevels(table_path, table_schema, processor_factory, files)); + + // compact and rewrite with output_level=0 + ASSERT_OK_AND_ASSIGN( + auto rewriter, CreateCompactRewriterForFilePosition(table_path, table_schema, core_options, + std::move(lookup_levels))); + ASSERT_OK_AND_ASSIGN(auto runs, GenerateSortedRuns({file0, file1})); + + // When output_level is 0, RewriteLookupChangelog should return false + // This tests the condition at line 59 in changelog_merge_tree_rewriter.cpp + ASSERT_OK_AND_ASSIGN(auto compact_result, rewriter->Rewrite( + /*output_level=*/0, /*drop_delete=*/true, runs)); + ASSERT_EQ(2, compact_result.Before().size()); + // When output_level is 0, rewrite should still produce valid result + ASSERT_GE(compact_result.After().size(), 1); + + const auto& compact_file_meta = compact_result.After()[0]; + // check compact file exist + std::string compact_file_name = table_path + "/bucket-0/" + compact_file_meta->file_name; + ASSERT_OK_AND_ASSIGN(bool exist, fs_->Exists(compact_file_name)); + ASSERT_TRUE(exist); + + // check file content + auto type_with_special_fields = + arrow::struct_(SpecialFields::CompleteSequenceAndValueKindField(arrow_schema_)->fields()); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(type_with_special_fields, {R"([ +[0, 0, 1, 11], +[2, 0, 2, 22], +[1, 0, 3, 33], +[3, 0, 4, 44] +])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()); + CheckResult(compact_file_name, table_schema, "orc", expected_array); +} + } // namespace paimon::test diff --git a/src/paimon/core/mergetree/levels_test.cpp b/src/paimon/core/mergetree/levels_test.cpp index e7af5d1c..da2d91e1 100644 --- a/src/paimon/core/mergetree/levels_test.cpp +++ b/src/paimon/core/mergetree/levels_test.cpp @@ -66,6 +66,22 @@ TEST_F(LevelsTest, TestNonEmptyHighestLevelNo) { ASSERT_EQ(levels->NonEmptyHighestLevel(), -1); } +TEST_F(LevelsTest, TestInvalidNumberLevels) { + std::vector> input_files; + ASSERT_NOK_WITH_MSG(Levels::Create(CreateComparator(), input_files, /*num_levels=*/1), + "Number of levels must be at least 2."); +} + +TEST_F(LevelsTest, TestAddLevel0FileInvalid) { + std::vector> input_files = {CreateDataFileMeta(0, 0, 1, 0), + CreateDataFileMeta(0, 2, 3, 0)}; + ASSERT_OK_AND_ASSIGN(auto levels, + Levels::Create(CreateComparator(), input_files, /*num_levels=*/3)); + std::vector> new_files = {CreateDataFileMeta(1, 0, 1, 0)}; + ASSERT_NOK_WITH_MSG(levels->AddLevel0File(new_files[0]), + "must add level0 file in AddLevel0File"); +} + TEST_F(LevelsTest, TestNonEmptyHighestLevel0) { std::vector> input_files = {CreateDataFileMeta(0, 0, 1, 0), CreateDataFileMeta(0, 2, 3, 0)}; @@ -105,6 +121,11 @@ TEST_F(LevelsTest, TestNonEmptyHighestLevel2) { ASSERT_EQ(levels->LevelSortedRuns(), expected_sorted_run); ASSERT_EQ(levels->NumberOfSortedRuns(), 6); + + std::vector> expected_all_files = { + input_files[2], input_files[3], input_files[1], + input_files[0], input_files[4], input_files[5]}; + ASSERT_EQ(levels->AllFiles(), expected_all_files); } TEST_F(LevelsTest, TestAddLevel0File) {