diff --git a/include/paimon/defs.h b/include/paimon/defs.h index 4c5158d2..14b5427d 100644 --- a/include/paimon/defs.h +++ b/include/paimon/defs.h @@ -203,6 +203,43 @@ struct PAIMON_EXPORT Options { /// "commit.max-retries" - Maximum number of retries when commit failed. Default value is 10. static const char COMMIT_MAX_RETRIES[]; + /// "compaction.max-size-amplification-percent" - The size amplification is defined as the + /// amount (in percentage) of additional storage needed to store a single byte of data in the + /// merge tree for changelog mode table. Default value is 200. + static const char COMPACTION_MAX_SIZE_AMPLIFICATION_PERCENT[]; + + /// "compaction.size-ratio" - Percentage flexibility while comparing sorted run size for + /// changelog mode table. If the candidate sorted run(s) size is 1% smaller than the next + /// sorted run's size, then include next sorted run into this candidate set. Default value is 1. + static const char COMPACTION_SIZE_RATIO[]; + + /// "num-sorted-run.compaction-trigger" - The sorted run number to trigger compaction. Includes + /// level0 files (one file one sorted run) and high-level runs (one level one sorted run). + /// Default value is 5. + static const char NUM_SORTED_RUNS_COMPACTION_TRIGGER[]; + + /// "num-sorted-run.stop-trigger" - The number of sorted runs that trigger the stopping of + /// writes, the default value is 'num-sorted-run.compaction-trigger' + 3. + static const char NUM_SORTED_RUNS_STOP_TRIGGER[]; + + /// "num-levels" - Total level number, for example, there are 3 levels, including 0,1,2 levels. + /// No default value. + static const char NUM_LEVELS[]; + + /// "lookup-compact" - Lookup compact mode used for lookup compaction. Default value is + /// LookupCompactMode::RADICAL. + static const char LOOKUP_COMPACT[]; + + /// "compaction.force-up-level-0" - If set to true, compaction strategy will always include all + /// level 0 files in candidates. Default value is false. + static const char COMPACTION_FORCE_UP_LEVEL_0[]; + + /// "lookup-compact.max-interval" - The max interval for a gentle mode lookup compaction to be + /// triggered. For every interval, a forced lookup compaction will be performed to flush L0 + /// files to higher level. This option is only valid when lookup-compact mode is gentle. No + /// default value. + static const char LOOKUP_COMPACT_MAX_INTERVAL[]; + /// "sequence.field" - The field that generates the sequence number for primary key table, the /// sequence number determines which data is the most recent. Value use "," as delimiter. static const char SEQUENCE_FIELD[]; diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index dc972227..842eff92 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -229,7 +229,9 @@ set(PAIMON_CORE_SRCS core/mergetree/compact/aggregate/field_sum_agg.cpp core/mergetree/compact/interval_partition.cpp core/mergetree/compact/loser_tree.cpp + core/mergetree/compact/merge_tree_compact_manager.cpp core/mergetree/compact/merge_tree_compact_rewriter.cpp + core/mergetree/compact/merge_tree_compact_task.cpp 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 @@ -508,6 +510,7 @@ if(PAIMON_BUILD_TESTS) core/catalog/identifier_test.cpp core/compact/compact_deletion_file_test.cpp core/core_options_test.cpp + core/options/lookup_strategy_test.cpp core/deletionvectors/apply_deletion_vector_batch_reader_test.cpp core/deletionvectors/bitmap_deletion_vector_test.cpp core/deletionvectors/bucketed_dv_maintainer_test.cpp @@ -575,6 +578,7 @@ if(PAIMON_BUILD_TESTS) core/mergetree/compact/universal_compaction_test.cpp core/mergetree/compact/force_up_level0_compaction_test.cpp core/mergetree/compact/compact_strategy_test.cpp + core/mergetree/compact/merge_tree_compact_manager_test.cpp core/mergetree/compact/merge_tree_compact_rewriter_test.cpp core/mergetree/lookup/persist_processor_test.cpp core/mergetree/drop_delete_reader_test.cpp diff --git a/src/paimon/common/defs.cpp b/src/paimon/common/defs.cpp index 349f4822..91b34350 100644 --- a/src/paimon/common/defs.cpp +++ b/src/paimon/common/defs.cpp @@ -102,4 +102,14 @@ const char Options::LOOKUP_CACHE_SPILL_COMPRESSION[] = "lookup.cache-spill-compr 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"; +const char Options::COMPACTION_MAX_SIZE_AMPLIFICATION_PERCENT[] = + "compaction.max-size-amplification-percent"; +const char Options::COMPACTION_SIZE_RATIO[] = "compaction.size-ratio"; +const char Options::NUM_SORTED_RUNS_COMPACTION_TRIGGER[] = "num-sorted-run.compaction-trigger"; +const char Options::NUM_SORTED_RUNS_STOP_TRIGGER[] = "num-sorted-run.stop-trigger"; +const char Options::NUM_LEVELS[] = "num-levels"; +const char Options::COMPACTION_FORCE_UP_LEVEL_0[] = "compaction.force-up-level-0"; +const char Options::LOOKUP_COMPACT[] = "lookup-compact"; +const char Options::LOOKUP_COMPACT_MAX_INTERVAL[] = "lookup-compact.max-interval"; + } // namespace paimon diff --git a/src/paimon/core/append/append_only_writer.cpp b/src/paimon/core/append/append_only_writer.cpp index a45cf8c0..136af6bd 100644 --- a/src/paimon/core/append/append_only_writer.cpp +++ b/src/paimon/core/append/append_only_writer.cpp @@ -140,7 +140,7 @@ Status AppendOnlyWriter::Flush(bool wait_for_latest_compaction, bool forced_full } // add new generated files for (const auto& flushed_file : flushed_files) { - compact_manager_->AddNewFile(flushed_file); + PAIMON_RETURN_NOT_OK(compact_manager_->AddNewFile(flushed_file)); } PAIMON_RETURN_NOT_OK(TrySyncLatestCompaction(wait_for_latest_compaction)); PAIMON_RETURN_NOT_OK(compact_manager_->TriggerCompaction(forced_full_compaction)); diff --git a/src/paimon/core/append/bucketed_append_compact_manager.cpp b/src/paimon/core/append/bucketed_append_compact_manager.cpp index 45798207..0431c416 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager.cpp @@ -16,6 +16,8 @@ #include "paimon/core/append/bucketed_append_compact_manager.h" +#include + #include "paimon/common/executor/future.h" namespace paimon { @@ -26,7 +28,7 @@ BucketedAppendCompactManager::BucketedAppendCompactManager( const std::shared_ptr& dv_maintainer, int32_t min_file_num, int64_t target_file_size, int64_t compaction_file_size, bool force_rewrite_all_files, CompactRewriter rewriter, const std::shared_ptr& reporter, - const std::shared_ptr& cancel_flag) + const std::shared_ptr& cancellation_controller) : executor_(executor), dv_maintainer_(dv_maintainer), min_file_num_(min_file_num), @@ -39,15 +41,16 @@ BucketedAppendCompactManager::BucketedAppendCompactManager( [](const std::shared_ptr& lhs, const std::shared_ptr& rhs) { return lhs->min_sequence_number > rhs->min_sequence_number; }), - cancel_flag_(cancel_flag ? cancel_flag : std::make_shared(false)), + cancellation_controller_(cancellation_controller), logger_(Logger::GetLogger("BucketedAppendCompactManager")) { + assert(cancellation_controller_ != nullptr); for (const auto& file : restored) { to_compact_.push(file); } } void BucketedAppendCompactManager::CancelCompaction() { - cancel_flag_->store(true, std::memory_order_relaxed); + cancellation_controller_->Cancel(); CompactFutureManager::CancelCompaction(); } @@ -78,7 +81,7 @@ Status BucketedAppendCompactManager::TriggerFullCompaction() { compacting.push_back(to_compact_.top()); to_compact_.pop(); } - cancel_flag_->store(false, std::memory_order_relaxed); + cancellation_controller_->Reset(); auto compact_task = std::make_shared(reporter_, dv_maintainer_, compacting, compaction_file_size_, force_rewrite_all_files_, rewriter_); @@ -95,7 +98,7 @@ void BucketedAppendCompactManager::TriggerCompactionWithBestEffort() { } std::optional>> picked = PickCompactBefore(); if (picked) { - cancel_flag_->store(false, std::memory_order_relaxed); + cancellation_controller_->Reset(); compacting_ = picked.value(); auto compact_task = std::make_shared(reporter_, dv_maintainer_, compacting_.value(), rewriter_); diff --git a/src/paimon/core/append/bucketed_append_compact_manager.h b/src/paimon/core/append/bucketed_append_compact_manager.h index 81613933..facfd4cf 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.h +++ b/src/paimon/core/append/bucketed_append_compact_manager.h @@ -16,7 +16,6 @@ #pragma once -#include #include #include #include @@ -25,6 +24,7 @@ #include #include "paimon/common/executor/future.h" +#include "paimon/core/compact/cancellation_controller.h" #include "paimon/core/compact/compact_deletion_file.h" #include "paimon/core/compact/compact_future_manager.h" #include "paimon/core/compact/compact_task.h" @@ -67,14 +67,13 @@ class BucketedAppendCompactManager : public CompactFutureManager { }; } - BucketedAppendCompactManager(const std::shared_ptr& executor, - const std::vector>& restored, - const std::shared_ptr& dv_maintainer, - int32_t min_file_num, int64_t target_file_size, - int64_t compaction_file_size, bool force_rewrite_all_files, - CompactRewriter rewriter, - const std::shared_ptr& reporter, - const std::shared_ptr& cancel_flag); + BucketedAppendCompactManager( + const std::shared_ptr& executor, + const std::vector>& restored, + const std::shared_ptr& dv_maintainer, int32_t min_file_num, + int64_t target_file_size, int64_t compaction_file_size, bool force_rewrite_all_files, + CompactRewriter rewriter, const std::shared_ptr& reporter, + const std::shared_ptr& cancellation_controller); ~BucketedAppendCompactManager() override = default; void CancelCompaction() override; @@ -88,8 +87,9 @@ class BucketedAppendCompactManager : public CompactFutureManager { return false; } - void AddNewFile(const std::shared_ptr& file) override { + Status AddNewFile(const std::shared_ptr& file) override { to_compact_.push(file); + return Status::OK(); } std::vector> AllFiles() const override; @@ -199,7 +199,7 @@ class BucketedAppendCompactManager : public CompactFutureManager { std::shared_ptr reporter_; std::optional>> compacting_; DataFileMetaPriorityQueue to_compact_; - std::shared_ptr cancel_flag_; + std::shared_ptr cancellation_controller_; std::unique_ptr logger_; }; diff --git a/src/paimon/core/append/bucketed_append_compact_manager_test.cpp b/src/paimon/core/append/bucketed_append_compact_manager_test.cpp index 9784c122..887521a8 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager_test.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager_test.cpp @@ -16,7 +16,6 @@ #include "paimon/core/append/bucketed_append_compact_manager.h" -#include #include #include #include @@ -76,7 +75,7 @@ class BucketedAppendCompactManagerTest : public testing::Test { executor_, to_compact_before_pick, /*dv_maintainer=*/nullptr, min_file_num, target_file_size, threshold, /*force_rewrite_all_files=*/false, /*rewriter=*/nullptr, /*reporter=*/nullptr, - /*cancel_flag=*/std::make_shared(false)); + /*cancellation_controller=*/std::make_shared()); auto actual = manager.PickCompactBefore(); if (expected_present) { ASSERT_TRUE(actual.has_value()); @@ -267,14 +266,14 @@ TEST_F(BucketedAppendCompactManagerTest, TestPick) { } TEST_F(BucketedAppendCompactManagerTest, TestCancelCompactionPropagatesToRewriteLoop) { - auto cancel_flag = std::make_shared(false); + auto cancellation_controller = std::make_shared(); auto exit_signal = std::make_shared>(); auto exit_future = exit_signal->get_future(); - auto rewriter = [cancel_flag, + auto rewriter = [cancellation_controller, exit_signal](const std::vector>& to_compact) -> Result>> { - while (!cancel_flag->load(std::memory_order_relaxed)) { + while (!cancellation_controller->IsCancelled()) { std::this_thread::sleep_for(std::chrono::milliseconds(1)); } exit_signal->set_value(); @@ -288,7 +287,7 @@ TEST_F(BucketedAppendCompactManagerTest, TestCancelCompactionPropagatesToRewrite /*target_file_size=*/1024, /*compaction_file_size=*/700, /*force_rewrite_all_files=*/false, rewriter, - /*reporter=*/nullptr, cancel_flag); + /*reporter=*/nullptr, cancellation_controller); ASSERT_OK(manager.TriggerCompaction(/*full_compaction=*/true)); manager.CancelCompaction(); @@ -297,7 +296,8 @@ TEST_F(BucketedAppendCompactManagerTest, TestCancelCompactionPropagatesToRewrite } TEST_F(BucketedAppendCompactManagerTest, TestTriggerCompactionResetsCancelFlag) { - auto cancel_flag = std::make_shared(true); + auto cancellation_controller = std::make_shared(); + cancellation_controller->Cancel(); auto rewriter = [](const std::vector>& to_compact) -> Result>> { return to_compact; }; @@ -308,10 +308,10 @@ TEST_F(BucketedAppendCompactManagerTest, TestTriggerCompactionResetsCancelFlag) /*target_file_size=*/1024, /*compaction_file_size=*/700, /*force_rewrite_all_files=*/false, rewriter, - /*reporter=*/nullptr, cancel_flag); + /*reporter=*/nullptr, cancellation_controller); ASSERT_OK(manager.TriggerCompaction(/*full_compaction=*/true)); - EXPECT_FALSE(cancel_flag->load(std::memory_order_relaxed)); + EXPECT_FALSE(cancellation_controller->IsCancelled()); } } // namespace paimon::test diff --git a/src/paimon/core/compact/cancellation_controller.h b/src/paimon/core/compact/cancellation_controller.h new file mode 100644 index 00000000..07ded108 --- /dev/null +++ b/src/paimon/core/compact/cancellation_controller.h @@ -0,0 +1,41 @@ +/* + * 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 + +namespace paimon { + +class CancellationController { + public: + void Cancel() { + cancelled_.store(true, std::memory_order_relaxed); + } + + void Reset() { + cancelled_.store(false, std::memory_order_relaxed); + } + + bool IsCancelled() const { + return cancelled_.load(std::memory_order_relaxed); + } + + private: + std::atomic_bool cancelled_{false}; +}; + +} // namespace paimon diff --git a/src/paimon/core/compact/compact_deletion_file.h b/src/paimon/core/compact/compact_deletion_file.h index 2223c3d4..8717ac67 100644 --- a/src/paimon/core/compact/compact_deletion_file.h +++ b/src/paimon/core/compact/compact_deletion_file.h @@ -37,7 +37,11 @@ class CompactDeletionFile { static Result> GenerateFiles( const std::shared_ptr& maintainer); - virtual std::optional> GetOrCompute() = 0; + /// For sync compaction, only create deletion files when prepare commit. + static Result> LazyGeneration( + const std::shared_ptr& maintainer); + + virtual Result>> GetOrCompute() = 0; virtual Result> MergeOldFile( const std::shared_ptr& old) = 0; @@ -53,7 +57,7 @@ class GeneratedDeletionFile : public CompactDeletionFile, const std::shared_ptr& dv_index_file) : deletion_file_(deletion_file), dv_index_file_(dv_index_file) {} - std::optional> GetOrCompute() override { + Result>> GetOrCompute() override { get_invoked_ = true; return deletion_file_ ? std::optional>(deletion_file_) : std::nullopt; @@ -87,6 +91,39 @@ class GeneratedDeletionFile : public CompactDeletionFile, bool get_invoked_ = false; }; +/// A lazy generation implementation of `CompactDeletionFile`. +class LazyCompactDeletionFile : public CompactDeletionFile, + public std::enable_shared_from_this { + public: + explicit LazyCompactDeletionFile(const std::shared_ptr& maintainer) + : maintainer_(maintainer) {} + + Result>> GetOrCompute() override { + generated_ = true; + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr generated, + CompactDeletionFile::GenerateFiles(maintainer_)); + return generated->GetOrCompute(); + } + + Result> MergeOldFile( + const std::shared_ptr& old) override { + auto derived = dynamic_cast(old.get()); + if (derived == nullptr) { + return Status::Invalid("old should be a LazyCompactDeletionFile, but it is not"); + } + if (derived->generated_) { + return Status::Invalid("old should not be generated, this is a bug."); + } + return shared_from_this(); + } + + void Clean() override {} + + private: + std::shared_ptr maintainer_; + bool generated_ = false; +}; + inline Result> CompactDeletionFile::GenerateFiles( const std::shared_ptr& maintainer) { PAIMON_ASSIGN_OR_RAISE(std::optional> file, @@ -95,4 +132,9 @@ inline Result> CompactDeletionFile::Generat maintainer->DvIndexFile()); } +inline Result> CompactDeletionFile::LazyGeneration( + const std::shared_ptr& maintainer) { + return std::make_shared(maintainer); +} + } // namespace paimon diff --git a/src/paimon/core/compact/compact_deletion_file_test.cpp b/src/paimon/core/compact/compact_deletion_file_test.cpp index bb36426d..2e855c06 100644 --- a/src/paimon/core/compact/compact_deletion_file_test.cpp +++ b/src/paimon/core/compact/compact_deletion_file_test.cpp @@ -59,8 +59,8 @@ class TestGeneratedDeletionFile : public GeneratedDeletionFile { class NonGeneratedCompactDeletionFile : public CompactDeletionFile { public: - std::optional> GetOrCompute() override { - return std::nullopt; + Result>> GetOrCompute() override { + return std::optional>(); } Result> MergeOldFile( @@ -90,7 +90,7 @@ TEST(CompactDeletionFileTest, GenerateFilesShouldReturnFileWhenModified) { ASSERT_OK_AND_ASSIGN(std::shared_ptr generated, CompactDeletionFile::GenerateFiles(maintainer)); - auto file = generated->GetOrCompute(); + ASSERT_OK_AND_ASSIGN(auto file, generated->GetOrCompute()); ASSERT_TRUE(file.has_value()); ASSERT_NE(file.value(), nullptr); ASSERT_EQ(file.value()->IndexType(), DeletionVectorsIndexFile::DELETION_VECTORS_INDEX); @@ -108,7 +108,7 @@ TEST(CompactDeletionFileTest, GenerateFilesShouldReturnNulloptWhenNotModified) { ASSERT_OK_AND_ASSIGN(std::shared_ptr generated, CompactDeletionFile::GenerateFiles(maintainer)); - auto file = generated->GetOrCompute(); + ASSERT_OK_AND_ASSIGN(auto file, generated->GetOrCompute()); ASSERT_FALSE(file.has_value()); } @@ -149,7 +149,8 @@ TEST(CompactDeletionFileTest, MergeOldFileShouldRejectInvokedOldFile) { auto current = std::make_shared(current_meta, dv_index_file); auto old = std::make_shared(old_meta, dv_index_file); - ASSERT_TRUE(old->GetOrCompute().has_value()); + ASSERT_OK_AND_ASSIGN(auto old_file, old->GetOrCompute()); + ASSERT_TRUE(old_file.has_value()); ASSERT_NOK_WITH_MSG(current->MergeOldFile(old), "old should not be get"); } @@ -227,4 +228,74 @@ TEST(CompactDeletionFileTest, CleanShouldDeleteIndexFile) { ASSERT_FALSE(exists_after); } +TEST(CompactDeletionFileTest, LazyGenerationShouldComputeWhenInvoked) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + RoaringBitmap32 roaring; + roaring.Add(1); + std::map> deletion_vectors; + deletion_vectors["data-a"] = std::make_shared(roaring); + + auto maintainer = CreateMaintainer(fs, path_factory, pool, deletion_vectors); + maintainer->RemoveDeletionVectorOf("data-a"); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr lazy, + CompactDeletionFile::LazyGeneration(maintainer)); + ASSERT_OK_AND_ASSIGN(auto file, lazy->GetOrCompute()); + ASSERT_TRUE(file.has_value()); + ASSERT_NE(file.value(), nullptr); + ASSERT_EQ(file.value()->IndexType(), DeletionVectorsIndexFile::DELETION_VECTORS_INDEX); +} + +TEST(CompactDeletionFileTest, LazyMergeOldFileShouldRejectNonLazyType) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + std::map> deletion_vectors; + auto maintainer = CreateMaintainer(fs, path_factory, pool, deletion_vectors); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr lazy, + CompactDeletionFile::LazyGeneration(maintainer)); + auto old = std::make_shared(); + + auto result = lazy->MergeOldFile(old); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("LazyCompactDeletionFile") != std::string::npos); +} + +TEST(CompactDeletionFileTest, LazyMergeOldFileShouldRejectGeneratedOldLazy) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + RoaringBitmap32 roaring; + roaring.Add(1); + std::map> deletion_vectors; + deletion_vectors["data-a"] = std::make_shared(roaring); + auto maintainer = CreateMaintainer(fs, path_factory, pool, deletion_vectors); + maintainer->RemoveDeletionVectorOf("data-a"); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr current, + CompactDeletionFile::LazyGeneration(maintainer)); + ASSERT_OK_AND_ASSIGN(std::shared_ptr old, + CompactDeletionFile::LazyGeneration(maintainer)); + + ASSERT_OK_AND_ASSIGN(auto old_file, old->GetOrCompute()); + ASSERT_TRUE(old_file.has_value()); + + auto result = current->MergeOldFile(old); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("old should not be generated") != + std::string::npos); +} + } // namespace paimon::test diff --git a/src/paimon/core/compact/compact_manager.h b/src/paimon/core/compact/compact_manager.h index ed180db9..405938c5 100644 --- a/src/paimon/core/compact/compact_manager.h +++ b/src/paimon/core/compact/compact_manager.h @@ -31,7 +31,7 @@ class CompactManager { public: virtual ~CompactManager() = default; /// Add a new file. - virtual void AddNewFile(const std::shared_ptr& file) = 0; + virtual Status AddNewFile(const std::shared_ptr& file) = 0; virtual std::vector> AllFiles() const = 0; diff --git a/src/paimon/core/compact/noop_compact_manager.h b/src/paimon/core/compact/noop_compact_manager.h index 46819128..fe3716df 100644 --- a/src/paimon/core/compact/noop_compact_manager.h +++ b/src/paimon/core/compact/noop_compact_manager.h @@ -32,7 +32,10 @@ class NoopCompactManager : public CompactManager { NoopCompactManager() = default; ~NoopCompactManager() override = default; - void AddNewFile(const std::shared_ptr& file) override {} + Status AddNewFile(const std::shared_ptr& file) override { + (void)file; + return Status::OK(); + } std::vector> AllFiles() const override { static std::vector> empty; diff --git a/src/paimon/core/core_options.cpp b/src/paimon/core/core_options.cpp index 67191eab..ba9725e7 100644 --- a/src/paimon/core/core_options.cpp +++ b/src/paimon/core/core_options.cpp @@ -29,6 +29,7 @@ #include "paimon/common/utils/path_util.h" #include "paimon/common/utils/string_utils.h" #include "paimon/core/options/expire_config.h" +#include "paimon/core/options/lookup_strategy.h" #include "paimon/core/options/sort_order.h" #include "paimon/core/utils/branch_manager.h" #include "paimon/defs.h" @@ -164,6 +165,22 @@ class ConfigParser { return Status::OK(); } + // Parse LookupCompactMode + Status ParseLookupCompactMode(LookupCompactMode* mode) { + auto iter = config_map_.find(Options::LOOKUP_COMPACT); + if (iter != config_map_.end()) { + std::string str = StringUtils::ToLowerCase(iter->second); + if (str == "radical") { + *mode = LookupCompactMode::RADICAL; + } else if (str == "gentle") { + *mode = LookupCompactMode::GENTLE; + } else { + return Status::Invalid(fmt::format("invalid lookup mode: {}", str)); + } + } + return Status::OK(); + } + // Parse SortEngine Status ParseSortEngine(SortEngine* sort_engine) const { auto iter = config_map_.find(Options::SORT_ENGINE); @@ -331,12 +348,19 @@ struct CoreOptions::Impl { int32_t write_batch_size = 1024; int32_t commit_max_retries = 10; int32_t compaction_min_file_num = 5; + int32_t compaction_max_size_amplification_percent = 200; + int32_t compaction_size_ratio = 1; + int32_t num_sorted_runs_compaction_trigger = 5; + std::optional num_sorted_runs_stop_trigger; + std::optional num_levels; SortOrder sequence_field_sort_order = SortOrder::ASCENDING; MergeEngine merge_engine = MergeEngine::DEDUPLICATE; SortEngine sort_engine = SortEngine::LOSER_TREE; ChangelogProducer changelog_producer = ChangelogProducer::NONE; ExternalPathStrategy external_path_strategy = ExternalPathStrategy::NONE; + LookupCompactMode lookup_compact_mode = LookupCompactMode::RADICAL; + std::optional lookup_compact_max_interval; int32_t file_compression_zstd_level = 1; @@ -355,6 +379,7 @@ struct CoreOptions::Impl { bool global_index_enabled = true; bool commit_force_compact = false; bool compaction_force_rewrite_all_files = false; + bool compaction_force_up_level_0 = false; std::optional global_index_external_path; std::optional scan_tag_name; @@ -571,6 +596,10 @@ Result CoreOptions::FromMap( PAIMON_RETURN_NOT_OK(parser.Parse(Options::COMPACTION_FORCE_REWRITE_ALL_FILES, &impl->compaction_force_rewrite_all_files)); + // Parse compaction.force-up-level-0 + PAIMON_RETURN_NOT_OK(parser.Parse(Options::COMPACTION_FORCE_UP_LEVEL_0, + &impl->compaction_force_up_level_0)); + // Parse compaction.optimization-interval std::string optimized_compaction_interval_str; PAIMON_RETURN_NOT_OK(parser.ParseString(Options::COMPACTION_OPTIMIZATION_INTERVAL, @@ -631,6 +660,21 @@ Result CoreOptions::FromMap( // parse file.format.per.level PAIMON_RETURN_NOT_OK(parser.ParseFileFormatPerLevel(&impl->file_format_per_level)); + + PAIMON_RETURN_NOT_OK(parser.Parse(Options::COMPACTION_MAX_SIZE_AMPLIFICATION_PERCENT, + &impl->compaction_max_size_amplification_percent)); + PAIMON_RETURN_NOT_OK( + parser.Parse(Options::COMPACTION_SIZE_RATIO, &impl->compaction_size_ratio)); + + PAIMON_RETURN_NOT_OK(parser.Parse(Options::NUM_SORTED_RUNS_COMPACTION_TRIGGER, + &impl->num_sorted_runs_compaction_trigger)); + PAIMON_RETURN_NOT_OK(parser.Parse(Options::NUM_SORTED_RUNS_STOP_TRIGGER, + &impl->num_sorted_runs_stop_trigger)); + PAIMON_RETURN_NOT_OK(parser.Parse(Options::NUM_LEVELS, &impl->num_levels)); + + PAIMON_RETURN_NOT_OK(parser.ParseLookupCompactMode(&impl->lookup_compact_mode)); + PAIMON_RETURN_NOT_OK( + parser.Parse(Options::LOOKUP_COMPACT_MAX_INTERVAL, &impl->lookup_compact_max_interval)); return options; } @@ -772,6 +816,70 @@ int32_t CoreOptions::GetCompactionMinFileNum() const { return impl_->compaction_min_file_num; } +int32_t CoreOptions::GetCompactionMaxSizeAmplificationPercent() const { + return impl_->compaction_max_size_amplification_percent; +} + +int32_t CoreOptions::GetCompactionSizeRatio() const { + return impl_->compaction_size_ratio; +} + +int32_t CoreOptions::GetNumSortedRunsCompactionTrigger() const { + return impl_->num_sorted_runs_compaction_trigger; +} + +int32_t CoreOptions::GetNumSortedRunsStopTrigger() const { + int32_t compact_trigger = GetNumSortedRunsCompactionTrigger(); + int32_t stop_trigger = 0; + if (impl_->num_sorted_runs_stop_trigger.has_value()) { + stop_trigger = impl_->num_sorted_runs_stop_trigger.value(); + } else { + int64_t computed = static_cast(compact_trigger) + 3; + if (computed > std::numeric_limits::max()) { + computed = std::numeric_limits::max(); + } + stop_trigger = static_cast(computed); + } + return std::max(compact_trigger, stop_trigger); +} + +int32_t CoreOptions::GetNumLevels() const { + // By default, this ensures that the compaction does not fall to level 0, but at least to + // level 1 + if (impl_->num_levels.has_value()) { + return impl_->num_levels.value(); + } + + int64_t incremented = static_cast(GetNumSortedRunsCompactionTrigger()) + 1; + if (incremented > std::numeric_limits::max()) { + incremented = std::numeric_limits::max(); + } + return static_cast(incremented); +} + +LookupCompactMode CoreOptions::GetLookupCompactMode() const { + return impl_->lookup_compact_mode; +} + +int32_t CoreOptions::GetLookupCompactMaxInterval() const { + int32_t compact_trigger = GetNumSortedRunsCompactionTrigger(); + int32_t max_interval; + if (impl_->lookup_compact_max_interval.has_value()) { + max_interval = impl_->lookup_compact_max_interval.value(); + } else { + int64_t doubled = static_cast(compact_trigger) * 2; + if (doubled > std::numeric_limits::max()) { + doubled = std::numeric_limits::max(); + } + max_interval = static_cast(doubled); + } + + if (max_interval < compact_trigger) { + max_interval = compact_trigger; + } + return max_interval; +} + const ExpireConfig& CoreOptions::GetExpireConfig() const { return impl_->expire_config; } @@ -845,6 +953,14 @@ ChangelogProducer CoreOptions::GetChangelogProducer() const { return impl_->changelog_producer; } +LookupStrategy CoreOptions::GetLookupStrategy() const { + return LookupStrategy::From( + /*is_first_row=*/GetMergeEngine() == MergeEngine::FIRST_ROW, + /*produce_changelog=*/GetChangelogProducer() == ChangelogProducer::LOOKUP, + /*deletion_vector=*/DeletionVectorsEnabled(), + /*force_lookup=*/impl_->force_lookup); +} + const std::map& CoreOptions::ToMap() const { return impl_->raw_options; } @@ -853,16 +969,14 @@ bool CoreOptions::NeedLookup() const { return GetLookupStrategy().need_lookup; } -LookupStrategy CoreOptions::GetLookupStrategy() const { - return {GetMergeEngine() == MergeEngine::FIRST_ROW, - GetChangelogProducer() == ChangelogProducer::LOOKUP, DeletionVectorsEnabled(), - impl_->force_lookup}; -} - bool CoreOptions::CompactionForceRewriteAllFiles() const { return impl_->compaction_force_rewrite_all_files; } +bool CoreOptions::CompactionForceUpLevel0() const { + return impl_->compaction_force_up_level_0; +} + std::map CoreOptions::GetFieldsSequenceGroups() const { auto raw_options = impl_->raw_options; std::map sequence_groups; diff --git a/src/paimon/core/core_options.h b/src/paimon/core/core_options.h index 19623a26..3687effd 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_compact_mode.h" #include "paimon/core/options/lookup_strategy.h" #include "paimon/core/options/merge_engine.h" #include "paimon/core/options/sort_engine.h" @@ -84,9 +85,17 @@ class PAIMON_EXPORT CoreOptions { bool CommitForceCompact() const; bool CompactionForceRewriteAllFiles() const; + bool CompactionForceUpLevel0() const; int64_t GetCommitTimeout() const; int32_t GetCommitMaxRetries() const; int32_t GetCompactionMinFileNum() const; + int32_t GetCompactionMaxSizeAmplificationPercent() const; + int32_t GetCompactionSizeRatio() const; + int32_t GetNumSortedRunsCompactionTrigger() const; + int32_t GetNumSortedRunsStopTrigger() const; + int32_t GetNumLevels() const; + LookupCompactMode GetLookupCompactMode() const; + int32_t GetLookupCompactMaxInterval() const; const std::vector& GetSequenceField() const; bool SequenceFieldSortOrderIsAscending() const; @@ -102,8 +111,9 @@ class PAIMON_EXPORT CoreOptions { bool DeletionVectorsBitmap64() const; int64_t DeletionVectorTargetFileSize() const; ChangelogProducer GetChangelogProducer() const; - bool NeedLookup() const; LookupStrategy GetLookupStrategy() const; + + bool NeedLookup() 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 9c589929..77c2429f 100644 --- a/src/paimon/core/core_options_test.cpp +++ b/src/paimon/core/core_options_test.cpp @@ -76,6 +76,7 @@ TEST(CoreOptionsTest, TestDefaultValue) { ASSERT_FALSE(core_options.WriteOnly()); ASSERT_EQ(5, core_options.GetCompactionMinFileNum()); ASSERT_FALSE(core_options.CompactionForceRewriteAllFiles()); + ASSERT_FALSE(core_options.CompactionForceUpLevel0()); ASSERT_EQ(std::nullopt, core_options.GetFieldsDefaultFunc()); ASSERT_EQ(std::nullopt, core_options.GetFieldAggFunc("f0").value()); ASSERT_FALSE(core_options.FieldAggIgnoreRetract("f1").value()); @@ -116,6 +117,12 @@ TEST(CoreOptionsTest, TestDefaultValue) { ASSERT_EQ("zstd", core_options.GetLookupCompressOptions().compress); ASSERT_EQ(1, core_options.GetLookupCompressOptions().zstd_level); ASSERT_EQ(64 * 1024, core_options.GetCachePageSize()); + ASSERT_EQ(200, core_options.GetCompactionMaxSizeAmplificationPercent()); + ASSERT_EQ(1, core_options.GetCompactionSizeRatio()); + ASSERT_EQ(5, core_options.GetNumSortedRunsCompactionTrigger()); + ASSERT_EQ(8, core_options.GetNumSortedRunsStopTrigger()); + ASSERT_EQ(LookupCompactMode::RADICAL, core_options.GetLookupCompactMode()); + ASSERT_EQ(10, core_options.GetLookupCompactMaxInterval()); } TEST(CoreOptionsTest, TestFromMap) { @@ -181,6 +188,13 @@ TEST(CoreOptionsTest, TestFromMap) { {Options::WRITE_ONLY, "true"}, {Options::COMPACTION_MIN_FILE_NUM, "10"}, {Options::COMPACTION_FORCE_REWRITE_ALL_FILES, "true"}, + {Options::COMPACTION_FORCE_UP_LEVEL_0, "true"}, + {Options::COMPACTION_MAX_SIZE_AMPLIFICATION_PERCENT, "123"}, + {Options::COMPACTION_SIZE_RATIO, "9"}, + {Options::NUM_SORTED_RUNS_COMPACTION_TRIGGER, "11"}, + {Options::NUM_SORTED_RUNS_STOP_TRIGGER, "17"}, + {Options::LOOKUP_COMPACT, "gentle"}, + {Options::LOOKUP_COMPACT_MAX_INTERVAL, "7"}, {Options::COMPACTION_OPTIMIZATION_INTERVAL, "2s"}, {Options::COMPACTION_TOTAL_SIZE_THRESHOLD, "5 GB"}, {Options::COMPACTION_INCREMENTAL_SIZE_THRESHOLD, "12 kB"}, @@ -279,7 +293,14 @@ TEST(CoreOptionsTest, TestFromMap) { ASSERT_EQ(375809637, core_options.GetCompactionFileSize(/*has_primary_key=*/false)); ASSERT_TRUE(core_options.WriteOnly()); ASSERT_EQ(10, core_options.GetCompactionMinFileNum()); + ASSERT_EQ(123, core_options.GetCompactionMaxSizeAmplificationPercent()); + ASSERT_EQ(9, core_options.GetCompactionSizeRatio()); + ASSERT_EQ(11, core_options.GetNumSortedRunsCompactionTrigger()); + ASSERT_EQ(17, core_options.GetNumSortedRunsStopTrigger()); + ASSERT_EQ(LookupCompactMode::GENTLE, core_options.GetLookupCompactMode()); + ASSERT_EQ(11, core_options.GetLookupCompactMaxInterval()); ASSERT_TRUE(core_options.CompactionForceRewriteAllFiles()); + ASSERT_TRUE(core_options.CompactionForceUpLevel0()); ASSERT_EQ(2000, core_options.GetOptimizedCompactionInterval().value()); ASSERT_EQ(5l * 1024 * 1024 * 1024, core_options.GetCompactionTotalSizeThreshold().value()); ASSERT_EQ(12l * 1024, core_options.GetCompactionIncrementalSizeThreshold().value()); @@ -306,6 +327,63 @@ TEST(CoreOptionsTest, TestInvalidCase) { "invalid merge engine: invalid"); ASSERT_NOK_WITH_MSG(CoreOptions::FromMap({{Options::CHANGELOG_PRODUCER, "invalid"}}), "invalid changelog producer: invalid"); + ASSERT_NOK_WITH_MSG(CoreOptions::FromMap({{Options::LOOKUP_COMPACT, "invalid"}}), + "invalid lookup mode: invalid"); + ASSERT_NOK_WITH_MSG(CoreOptions::FromMap({{Options::LOOKUP_COMPACT_MAX_INTERVAL, "invalid"}}), + "Invalid Config [lookup-compact.max-interval: invalid]"); +} + +TEST(CoreOptionsTest, TestLookupCompactMaxIntervalComputedValue) { + std::map options = { + {Options::NUM_SORTED_RUNS_COMPACTION_TRIGGER, "11"}, + {Options::LOOKUP_COMPACT_MAX_INTERVAL, "13"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_EQ(13, core_options.GetLookupCompactMaxInterval()); +} + +TEST(CoreOptionsTest, TestNumSortedRunsStopTriggerFloorAndDefault) { + { + std::map options = { + {Options::NUM_SORTED_RUNS_COMPACTION_TRIGGER, "11"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_EQ(14, core_options.GetNumSortedRunsStopTrigger()); + } + + { + std::map options = { + {Options::NUM_SORTED_RUNS_COMPACTION_TRIGGER, "11"}, + {Options::NUM_SORTED_RUNS_STOP_TRIGGER, "7"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + ASSERT_EQ(11, core_options.GetNumSortedRunsStopTrigger()); + } +} + +TEST(CoreOptionsTest, TestLookupStrategy) { + { + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap({})); + auto strategy = core_options.GetLookupStrategy(); + ASSERT_FALSE(strategy.is_first_row); + ASSERT_FALSE(strategy.produce_changelog); + ASSERT_FALSE(strategy.deletion_vector); + ASSERT_FALSE(strategy.need_lookup); + } + { + std::map options = { + {Options::MERGE_ENGINE, "first-row"}, + {Options::CHANGELOG_PRODUCER, "lookup"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + {Options::FORCE_LOOKUP, "true"}, + }; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); + auto strategy = core_options.GetLookupStrategy(); + ASSERT_TRUE(strategy.is_first_row); + ASSERT_TRUE(strategy.produce_changelog); + ASSERT_TRUE(strategy.deletion_vector); + ASSERT_TRUE(strategy.need_lookup); + } } TEST(CoreOptionsTest, TestInvalidFileFormatPerLevel) { @@ -422,6 +500,7 @@ TEST(CoreOptionsTest, TestNormalizeValueInCoreOption) { {Options::MERGE_ENGINE, "first-ROW"}, {Options::CHANGELOG_PRODUCER, "LOOKUP"}, {Options::DATA_FILE_EXTERNAL_PATHS_STRATEGY, "ROUND-ROBIN"}, + {Options::LOOKUP_COMPACT, "GENTLE"}, {Options::SCAN_MODE, "DEFAULT"}, }; ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap(options)); @@ -431,6 +510,7 @@ TEST(CoreOptionsTest, TestNormalizeValueInCoreOption) { ASSERT_EQ(ChangelogProducer::LOOKUP, core_options.GetChangelogProducer()); ASSERT_EQ(MergeEngine::FIRST_ROW, core_options.GetMergeEngine()); ASSERT_EQ(SortEngine::MIN_HEAP, core_options.GetSortEngine()); + ASSERT_EQ(LookupCompactMode::GENTLE, core_options.GetLookupCompactMode()); ASSERT_TRUE(core_options.SequenceFieldSortOrderIsAscending()); } } // namespace paimon::test diff --git a/src/paimon/core/deletionvectors/deletion_vector.cpp b/src/paimon/core/deletionvectors/deletion_vector.cpp index 549fbf8f..d8217c03 100644 --- a/src/paimon/core/deletionvectors/deletion_vector.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector.cpp @@ -22,6 +22,7 @@ #include "fmt/format.h" #include "paimon/core/deletionvectors/bitmap64_deletion_vector.h" #include "paimon/core/deletionvectors/bitmap_deletion_vector.h" +#include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" #include "paimon/core/table/source/deletion_file.h" #include "paimon/fs/file_system.h" #include "paimon/io/data_input_stream.h" @@ -29,6 +30,35 @@ namespace paimon { +DeletionVector::Factory DeletionVector::CreateFactory( + const std::shared_ptr& file_system, + const std::unordered_map& deletion_file_map, + const std::shared_ptr& pool) { + return [file_system, deletion_file_map, + pool](const std::string& file_name) -> Result> { + auto iter = deletion_file_map.find(file_name); + if (iter != deletion_file_map.end()) { + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr dv, + DeletionVector::Read(file_system.get(), iter->second, pool.get())); + return dv; + } + return std::shared_ptr(); + }; +} + +DeletionVector::Factory DeletionVector::CreateFactory( + const std::shared_ptr& dv_maintainer) { + return + [dv_maintainer](const std::string& file_name) -> Result> { + if (dv_maintainer) { + return dv_maintainer->DeletionVectorOf(file_name).value_or( + std::shared_ptr()); + } + return std::shared_ptr(); + }; +} + Result> DeletionVector::DeserializeFromBytes(const Bytes* bytes, MemoryPool* pool) { return BitmapDeletionVector::Deserialize(bytes->data(), bytes->size(), pool); diff --git a/src/paimon/core/deletionvectors/deletion_vector.h b/src/paimon/core/deletionvectors/deletion_vector.h index 91bcc67e..af86877d 100644 --- a/src/paimon/core/deletionvectors/deletion_vector.h +++ b/src/paimon/core/deletionvectors/deletion_vector.h @@ -33,6 +33,7 @@ namespace paimon { class FileSystem; class DataOutputStream; +class BucketedDvMaintainer; struct DeletionFile; /// The DeletionVector can efficiently record the positions of rows that are deleted in a file, @@ -41,6 +42,13 @@ class DeletionVector { public: using Factory = std::function>(const std::string&)>; + static Factory CreateFactory( + const std::shared_ptr& file_system, + const std::unordered_map& deletion_file_map, + const std::shared_ptr& pool); + + static Factory CreateFactory(const std::shared_ptr& dv_maintainer); + virtual ~DeletionVector() = default; /// Marks the row at the specified position as deleted. 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 e77d34ad..8205587f 100644 --- a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp +++ b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.cpp @@ -24,11 +24,12 @@ ChangelogMergeTreeRewriter::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) - : MergeTreeCompactRewriter(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), + const std::shared_ptr& pool, + const std::shared_ptr& cancellation_controller) + : MergeTreeCompactRewriter( + 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, cancellation_controller), max_level_(max_level), force_drop_delete_(force_drop_delete) {} 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 56a22a49..b0bf0d8b 100644 --- a/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h +++ b/src/paimon/core/mergetree/compact/changelog_merge_tree_rewriter.h @@ -32,17 +32,16 @@ class ChangelogMergeTreeRewriter : public MergeTreeCompactRewriter { const std::shared_ptr& file) override; 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, - 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( + 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, 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, + const std::shared_ptr& cancellation_controller); struct UpgradeStrategy { static UpgradeStrategy NoChangelogNoRewrite() { @@ -77,13 +76,12 @@ class ChangelogMergeTreeRewriter : public MergeTreeCompactRewriter { bool RewriteLookupChangelog(int32_t output_level, const std::vector>& sections) const; + int32_t max_level_; + bool force_drop_delete_; + private: Result RewriteOrProduceChangelog( int32_t output_level, const std::vector>& sections, bool drop_delete, bool rewrite_compact_file); - - protected: - int32_t max_level_; - bool force_drop_delete_; }; } // namespace paimon diff --git a/src/paimon/core/mergetree/compact/file_rewrite_compact_task.h b/src/paimon/core/mergetree/compact/file_rewrite_compact_task.h new file mode 100644 index 00000000..f1f2b6c5 --- /dev/null +++ b/src/paimon/core/mergetree/compact/file_rewrite_compact_task.h @@ -0,0 +1,65 @@ +/* + * 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/compact/compact_task.h" +#include "paimon/core/compact/compact_unit.h" +#include "paimon/core/mergetree/compact/compact_rewriter.h" +#include "paimon/core/mergetree/sorted_run.h" + +namespace paimon { + +/// Compact task for file rewrite compaction. +class FileRewriteCompactTask : public CompactTask { + public: + FileRewriteCompactTask(const std::shared_ptr& rewriter, + const CompactUnit& unit, bool drop_delete, + const std::shared_ptr& metrics_reporter) + : CompactTask(metrics_reporter), + rewriter_(rewriter), + output_level_(unit.output_level), + files_(unit.files), + drop_delete_(drop_delete) {} + + protected: + Result> DoCompact() override { + auto result = std::make_shared(); + for (const auto& file : files_) { + PAIMON_RETURN_NOT_OK(RewriteFile(file, result.get())); + } + return result; + } + + private: + Status RewriteFile(const std::shared_ptr& file, CompactResult* to_update) { + std::vector> candidate = {{SortedRun::FromSingle(file)}}; + PAIMON_ASSIGN_OR_RAISE(CompactResult rewritten, + rewriter_->Rewrite(output_level_, drop_delete_, candidate)); + return to_update->Merge(rewritten); + } + + std::shared_ptr rewriter_; + int32_t output_level_; + std::vector> files_; + bool drop_delete_; +}; + +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/interval_partition.h b/src/paimon/core/mergetree/compact/interval_partition.h index af87804b..229b7133 100644 --- a/src/paimon/core/mergetree/compact/interval_partition.h +++ b/src/paimon/core/mergetree/compact/interval_partition.h @@ -24,8 +24,6 @@ #include "paimon/core/utils/fields_comparator.h" namespace paimon { -class FieldsComparator; -struct DataFileMeta; /// Algorithm to partition several data files into the minimum number of `SortedRun`s. class IntervalPartition { 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 63a2c4e1..a7fa7731 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 @@ -34,12 +34,13 @@ LookupMergeTreeCompactRewriter::LookupMergeTreeCompactRewriter( 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, std::move(dv_factory), - path_factory_cache, std::move(merge_file_split_read), - std::move(merge_function_wrapper_factory), pool), + const std::shared_ptr& pool, + const std::shared_ptr& cancellation_controller) + : 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, cancellation_controller), lookup_levels_(std::move(lookup_levels)), dv_maintainer_(dv_maintainer) {} @@ -52,7 +53,8 @@ LookupMergeTreeCompactRewriter::Create( 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) { + const CoreOptions& options, const std::shared_ptr& pool, + const std::shared_ptr& cancellation_controller) { PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, table_schema->TrimmedPrimaryKeys()); auto data_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); @@ -77,7 +79,7 @@ LookupMergeTreeCompactRewriter::Create( std::move(lookup_levels), dv_maintainer, max_level, partition, bucket, table_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)); + std::move(merge_function_wrapper_factory), pool, cancellation_controller)); } 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 eb090e51..db20f176 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 @@ -36,7 +36,8 @@ class LookupMergeTreeCompactRewriter : public ChangelogMergeTreeRewriter { 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); + const CoreOptions& options, const std::shared_ptr& pool, + const std::shared_ptr& cancellation_controller); Status Close() override { return lookup_levels_->Close(); @@ -64,7 +65,8 @@ 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); + const std::shared_ptr& pool, + const std::shared_ptr& cancellation_controller); bool RewriteChangelog(int32_t output_level, bool drop_delete, const std::vector>& sections) const override { 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 c054769a..70252637 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 @@ -25,6 +25,7 @@ #include "paimon/common/factories/io_hook.h" #include "paimon/common/table/special_fields.h" #include "paimon/common/utils/path_util.h" +#include "paimon/core/compact/noop_compact_manager.h" #include "paimon/core/core_options.h" #include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" #include "paimon/core/io/data_file_path_factory.h" @@ -90,11 +91,12 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { auto merge_function_wrapper = std::make_shared(std::move(mfunc)); + auto cancellation_controller = std::make_shared(); 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_); + arrow_schema_, options, std::make_shared(), pool_); // write data ArrowArray c_src_array; @@ -138,9 +140,7 @@ 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 dv_factory = DeletionVector::CreateFactory(std::shared_ptr()); auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); auto merge_function_wrapper_factory = @@ -152,22 +152,20 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { lookup_levels_ptr); return merge_function_wrapper; }; - + auto cancellation_controller = std::make_shared(); 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, std::move(dv_factory), - path_factory_cache, options, pool_); + path_factory_cache, options, pool_, cancellation_controller); } 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 dv_factory = DeletionVector::CreateFactory(std::shared_ptr()); auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); auto merge_function_wrapper_factory = @@ -186,12 +184,12 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { /*user_defined_seq_comparator=*/nullptr, lookup_levels_ptr)); return merge_function_wrapper; }; - + auto cancellation_controller = std::make_shared(); 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, std::move(dv_factory), - path_factory_cache, options, pool_); + path_factory_cache, options, pool_, cancellation_controller); } Result>> @@ -200,9 +198,7 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { 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(); - }; + dv_factory = DeletionVector::CreateFactory(std::shared_ptr()); } auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); @@ -227,12 +223,12 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { /*user_defined_seq_comparator=*/nullptr, lookup_levels_ptr)); return merge_function_wrapper; }; - + auto cancellation_controller = std::make_shared(); 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, std::move(dv_factory), - path_factory_cache, options, pool_); + path_factory_cache, options, pool_, cancellation_controller); } Result>> @@ -240,9 +236,7 @@ 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 dv_factory = DeletionVector::CreateFactory(std::shared_ptr()); auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); @@ -271,12 +265,12 @@ class LookupMergeTreeCompactRewriterTest : public testing::Test { /*user_defined_seq_comparator=*/nullptr, lookup_levels_ptr)); return merge_function_wrapper; }; - + auto cancellation_controller = std::make_shared(); 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, std::move(dv_factory), - path_factory_cache, options, pool_); + path_factory_cache, options, pool_, cancellation_controller); } void CheckResult(const std::string& compact_file_name, @@ -1001,6 +995,7 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestIOException) { } TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { + auto cancellation_controller = std::make_shared(); auto create_meta = [this](int32_t level, std::optional delete_row_count) { return std::make_shared( "file.orc", 100l, /*row_count=*/4, @@ -1017,13 +1012,13 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { { 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, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, - /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_, + cancellation_controller); auto file = create_meta(/*level=*/1, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::NoChangelogNoRewrite(), rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); @@ -1032,13 +1027,13 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { 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, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, - /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_, + cancellation_controller); auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), rewriter.GenerateUpgradeStrategy(/*output_level=*/5, file)); @@ -1056,7 +1051,8 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, - /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_, + cancellation_controller); auto file = create_meta(/*level=*/0, /*delete_row_count=*/1); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); @@ -1070,7 +1066,8 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, - /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_, + cancellation_controller); auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogNoRewrite(), rewriter.GenerateUpgradeStrategy(/*output_level=*/5, file)); @@ -1084,7 +1081,8 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, - /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_, + cancellation_controller); auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogNoRewrite(), rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); @@ -1099,7 +1097,8 @@ TEST_F(LookupMergeTreeCompactRewriterTest, TestGenerateUpgradeStrategy) { /*max_level=*/5, BinaryRow::EmptyRow(), /*bucket=*/0, /*schema_id=*/0, /*trimmed_primary_keys=*/{"key"}, core_options, /*data_schema=*/nullptr, /*write_schema=*/nullptr, /*dv_factory*/ nullptr, /*path_factory_cache=*/nullptr, - /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_); + /*merge_file_split_read=*/nullptr, /*merge_function_wrapper_factory=*/nullptr, pool_, + cancellation_controller); auto file = create_meta(/*level=*/0, /*delete_row_count=*/std::nullopt); ASSERT_EQ(ChangelogMergeTreeRewriter::UpgradeStrategy::ChangelogWithRewrite(), rewriter.GenerateUpgradeStrategy(/*output_level=*/2, file)); diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_manager.cpp b/src/paimon/core/mergetree/compact/merge_tree_compact_manager.cpp new file mode 100644 index 00000000..08904795 --- /dev/null +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_manager.cpp @@ -0,0 +1,236 @@ +/* + * 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/merge_tree_compact_manager.h" + +#include +#include + +#include "paimon/common/executor/future.h" +#include "paimon/core/compact/compact_deletion_file.h" +#include "paimon/core/compact/compact_task.h" +#include "paimon/core/mergetree/compact/file_rewrite_compact_task.h" +#include "paimon/core/mergetree/compact/interval_partition.h" +#include "paimon/core/mergetree/compact/merge_tree_compact_task.h" + +namespace paimon { + +std::string RunsToString(const std::vector& runs) { + std::stringstream ss; + for (const auto& run : runs) { + ss << run.ToString() << "\n"; + } + return ss.str(); +} + +std::string FilesToString(const std::vector& runs) { + std::stringstream ss; + bool first = true; + for (const auto& level_sorted_run : runs) { + for (const auto& file : level_sorted_run.run.Files()) { + if (!first) { + ss << ", "; + } + ss << "(" << file->file_name << ", " << file->level << ", " << file->file_size << ")"; + first = false; + } + } + return ss.str(); +} + +MergeTreeCompactManager::MergeTreeCompactManager( + const std::shared_ptr& executor, const std::shared_ptr& levels, + const std::shared_ptr& strategy, + const std::shared_ptr& key_comparator, int64_t compaction_file_size, + int32_t num_sorted_run_stop_trigger, const std::shared_ptr& rewriter, + const std::shared_ptr& metrics_reporter, + const std::shared_ptr& dv_maintainer, bool lazy_gen_deletion_file, + bool need_lookup, bool force_rewrite_all_files, bool force_keep_delete, + const std::shared_ptr& cancellation_controller) + : executor_(executor), + levels_(levels), + strategy_(strategy), + key_comparator_(key_comparator), + compaction_file_size_(compaction_file_size), + num_sorted_run_stop_trigger_(num_sorted_run_stop_trigger), + rewriter_(rewriter), + metrics_reporter_(metrics_reporter), + dv_maintainer_(dv_maintainer), + lazy_gen_deletion_file_(lazy_gen_deletion_file), + need_lookup_(need_lookup), + force_rewrite_all_files_(force_rewrite_all_files), + force_keep_delete_(force_keep_delete), + cancellation_controller_(cancellation_controller), + logger_(Logger::GetLogger("MergeTreeCompactManager")) { + assert(cancellation_controller_ != nullptr); + ReportMetrics(); +} + +bool MergeTreeCompactManager::ShouldWaitForLatestCompaction() const { + return levels_->NumberOfSortedRuns() > num_sorted_run_stop_trigger_; +} + +bool MergeTreeCompactManager::ShouldWaitForPreparingCheckpoint() const { + // cast to long to avoid Numeric overflow + return levels_->NumberOfSortedRuns() > (static_cast(num_sorted_run_stop_trigger_) + 1); +} + +Status MergeTreeCompactManager::AddNewFile(const std::shared_ptr& file) { + // if overwrite an empty partition, the snapshot will be changed to APPEND, then its files + // might be upgraded to high level, thus we should use #update + PAIMON_RETURN_NOT_OK(levels_->Update(/*before=*/{}, /*after=*/{file})); + ReportMetrics(); + return Status::OK(); +} + +std::vector> MergeTreeCompactManager::AllFiles() const { + return levels_->AllFiles(); +} + +Status MergeTreeCompactManager::TriggerCompaction(bool full_compaction) { + std::optional optional_unit; + std::vector runs = levels_->LevelSortedRuns(); + + if (full_compaction) { + if (task_future_.valid()) { + return Status::Invalid( + "A compaction task is still running while the user forces a new compaction. This " + "is unexpected."); + } + + PAIMON_LOG_DEBUG(logger_, "Trigger forced full compaction. Picking from runs:\n%s", + RunsToString(runs).c_str()); + optional_unit = CompactStrategy::PickFullCompaction(levels_->NumberOfLevels(), runs, + force_rewrite_all_files_); + } else { + if (task_future_.valid()) { + return Status::OK(); + } + + PAIMON_LOG_DEBUG(logger_, "Trigger normal compaction. Picking from the following runs:\n%s", + RunsToString(runs).c_str()); + + PAIMON_ASSIGN_OR_RAISE(std::optional picked, + strategy_->Pick(levels_->NumberOfLevels(), runs)); + if (picked && !picked->files.empty() && + (picked->files.size() > 1 || picked->files[0]->level != picked->output_level)) { + optional_unit = picked; + } + } + + if (!optional_unit) { + return Status::OK(); + } + + // As long as there is no older data, We can drop the deletion. + // If the output level is 0, there may be older data not involved in compaction. + // If the output level is bigger than 0, as long as there is no older data in + // the current levels, the output is the oldest, so we can drop the deletion. + // See CompactStrategy::Pick. + const CompactUnit& unit = optional_unit.value(); + bool drop_delete = + !force_keep_delete_ && unit.output_level != 0 && + (unit.output_level >= levels_->NonEmptyHighestLevel() || dv_maintainer_ != nullptr); + + PAIMON_LOG_DEBUG(logger_, "Submit compaction with files (name, level, size): %s", + FilesToString(levels_->LevelSortedRuns()).c_str()); + return SubmitCompaction(unit, drop_delete); +} + +void MergeTreeCompactManager::CancelCompaction() { + cancellation_controller_->Cancel(); + CompactFutureManager::CancelCompaction(); +} + +Status MergeTreeCompactManager::SubmitCompaction(const CompactUnit& unit, bool drop_delete) { + cancellation_controller_->Reset(); + if (unit.file_rewrite) { + auto task = std::make_shared(rewriter_, unit, drop_delete, + metrics_reporter_); + task_future_ = Via(executor_.get(), [task]() -> Result> { + return task->Execute(); + }); + } else { + MergeTreeCompactTask::DeletionFileSupplier compact_df_supplier = + []() -> Result> { + return std::shared_ptr(); + }; + if (dv_maintainer_ != nullptr) { + if (lazy_gen_deletion_file_) { + compact_df_supplier = [dv = dv_maintainer_]() { + return CompactDeletionFile::LazyGeneration(dv); + }; + } else { + compact_df_supplier = [dv = dv_maintainer_]() { + return CompactDeletionFile::GenerateFiles(dv); + }; + } + } + + auto task = std::make_shared( + key_comparator_, compaction_file_size_, rewriter_, unit, drop_delete, + levels_->MaxLevel(), metrics_reporter_, compact_df_supplier, force_rewrite_all_files_); + task_future_ = Via(executor_.get(), [task]() -> Result> { + return task->Execute(); + }); + } + + if (metrics_reporter_) { + metrics_reporter_->IncreaseCompactionsQueuedCount(); + metrics_reporter_->IncreaseCompactionsTotalCount(); + } + return Status::OK(); +} + +Result>> MergeTreeCompactManager::GetCompactionResult( + bool blocking) { + PAIMON_ASSIGN_OR_RAISE(std::optional> result, + InnerGetCompactionResult(blocking)); + if (result) { + const auto& compact_result = result.value(); + PAIMON_RETURN_NOT_OK(levels_->Update(compact_result->Before(), compact_result->After())); + ReportMetrics(); + PAIMON_LOG_DEBUG(logger_, "Levels in compact manager updated. Current runs are\n%s", + RunsToString(levels_->LevelSortedRuns()).c_str()); + } + return result; +} + +bool MergeTreeCompactManager::CompactNotCompleted() const { + // If it is a lookup compaction, we should ensure that all level 0 files are consumed, so + // here we need to make the outside think that we still need to do unfinished compact + // working + return CompactFutureManager::CompactNotCompleted() || + (need_lookup_ && !levels_->GetLevel0().empty()); +} + +Status MergeTreeCompactManager::Close() { + PAIMON_RETURN_NOT_OK(rewriter_->Close()); + if (metrics_reporter_) { + metrics_reporter_->Unregister(); + metrics_reporter_.reset(); + } + return Status::OK(); +} + +void MergeTreeCompactManager::ReportMetrics() const { + if (metrics_reporter_) { + metrics_reporter_->ReportLevel0FileCount(levels_->GetLevel0().size()); + metrics_reporter_->ReportTotalFileSize(levels_->TotalFileSize()); + } +} + +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_manager.h b/src/paimon/core/mergetree/compact/merge_tree_compact_manager.h new file mode 100644 index 00000000..4bb2fdae --- /dev/null +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_manager.h @@ -0,0 +1,102 @@ +/* + * 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/compact/cancellation_controller.h" +#include "paimon/core/compact/compact_future_manager.h" +#include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" +#include "paimon/core/mergetree/compact/compact_rewriter.h" +#include "paimon/core/mergetree/compact/compact_strategy.h" +#include "paimon/core/mergetree/levels.h" +#include "paimon/core/operation/metrics/compaction_metrics.h" +#include "paimon/core/utils/fields_comparator.h" +#include "paimon/executor.h" +#include "paimon/logging.h" + +namespace paimon { + +/// Compact manager for key value tables. +class MergeTreeCompactManager : public CompactFutureManager { + public: + MergeTreeCompactManager(const std::shared_ptr& executor, + const std::shared_ptr& levels, + const std::shared_ptr& strategy, + const std::shared_ptr& key_comparator, + int64_t compaction_file_size, int32_t num_sorted_run_stop_trigger, + const std::shared_ptr& rewriter, + const std::shared_ptr& metrics_reporter, + const std::shared_ptr& dv_maintainer, + bool lazy_gen_deletion_file, bool need_lookup, + bool force_rewrite_all_files, bool force_keep_delete, + const std::shared_ptr& cancellation_controller); + + ~MergeTreeCompactManager() override = default; + + bool ShouldWaitForLatestCompaction() const override; + + bool ShouldWaitForPreparingCheckpoint() const override; + + Status AddNewFile(const std::shared_ptr& file) override; + + std::vector> AllFiles() const override; + + Status TriggerCompaction(bool full_compaction) override; + + void CancelCompaction() override; + + Result>> GetCompactionResult( + bool blocking) override; + + bool CompactNotCompleted() const override; + + Status Close() override; + + std::shared_ptr GetLevels() const { + return levels_; + } + + std::shared_ptr GetStrategy() const { + return strategy_; + } + + private: + Status SubmitCompaction(const CompactUnit& unit, bool drop_delete); + + void ReportMetrics() const; + + std::shared_ptr executor_; + std::shared_ptr levels_; + std::shared_ptr strategy_; + std::shared_ptr key_comparator_; + int64_t compaction_file_size_; + int32_t num_sorted_run_stop_trigger_; + std::shared_ptr rewriter_; + std::shared_ptr metrics_reporter_; + std::shared_ptr dv_maintainer_; + bool lazy_gen_deletion_file_; + bool need_lookup_; + bool force_rewrite_all_files_; + bool force_keep_delete_; + std::shared_ptr cancellation_controller_; + std::unique_ptr logger_; +}; + +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_manager_test.cpp b/src/paimon/core/mergetree/compact/merge_tree_compact_manager_test.cpp new file mode 100644 index 00000000..2dbdb247 --- /dev/null +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_manager_test.cpp @@ -0,0 +1,435 @@ +/* + * 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/merge_tree_compact_manager.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "arrow/api.h" +#include "gtest/gtest.h" +#include "paimon/core/io/data_file_meta.h" +#include "paimon/core/manifest/file_source.h" +#include "paimon/core/mergetree/level_sorted_run.h" +#include "paimon/core/mergetree/levels.h" +#include "paimon/core/mergetree/sorted_run.h" +#include "paimon/core/stats/simple_stats.h" +#include "paimon/core/utils/fields_comparator.h" +#include "paimon/testing/utils/binary_row_generator.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::test { + +namespace { + +class InlineExecutor final : public Executor { + public: + void Add(std::function func) override { + func(); + } + + void ShutdownNow() override {} +}; + +class QueuedExecutor final : public Executor { + public: + void Add(std::function func) override { + tasks_.push_back(std::move(func)); + } + + void ShutdownNow() override {} + + void RunAll() { + for (auto& task : tasks_) { + task(); + } + tasks_.clear(); + } + + private: + std::vector> tasks_; +}; + +class FunctionalCompactStrategy final : public CompactStrategy { + public: + using PickFunc = + std::function(int32_t, const std::vector&)>; + + explicit FunctionalCompactStrategy(PickFunc func) : func_(std::move(func)) {} + + Result> Pick(int32_t num_levels, + const std::vector& runs) override { + return func_(num_levels, runs); + } + + private: + PickFunc func_; +}; + +class TestRewriter final : public CompactRewriter { + public: + explicit TestRewriter(bool expected_drop_delete) + : expected_drop_delete_(expected_drop_delete) {} + + Result Rewrite(int32_t output_level, bool drop_delete, + const std::vector>& sections) override { + EXPECT_EQ(drop_delete, expected_drop_delete_); + + int32_t min_key = std::numeric_limits::max(); + int32_t max_key = std::numeric_limits::min(); + int64_t max_sequence = 0; + std::vector> before; + + for (const auto& section : sections) { + for (const auto& run : section) { + for (const auto& file : run.Files()) { + before.push_back(file); + min_key = std::min(min_key, file->min_key.GetInt(0)); + max_key = std::max(max_key, file->max_key.GetInt(0)); + max_sequence = std::max(max_sequence, file->max_sequence_number); + } + } + } + + if (before.empty()) { + return CompactResult({}, {}); + } + + auto after = std::make_shared( + "rewrite-" + std::to_string(rewrite_id_++), + /*file_size=*/max_key - min_key + 1, + /*row_count=*/0, BinaryRowGenerator::GenerateRow({min_key}, pool_.get()), + BinaryRowGenerator::GenerateRow({max_key}, pool_.get()), SimpleStats::EmptyStats(), + SimpleStats::EmptyStats(), + /*min_sequence_number=*/min_key, + /*max_sequence_number=*/max_sequence, + /*schema_id=*/0, output_level, std::vector>(), + Timestamp(1, 0), std::nullopt, nullptr, FileSource::Append(), std::nullopt, + std::nullopt, std::nullopt, std::nullopt); + + return CompactResult(before, {after}); + } + + Result Upgrade(int32_t output_level, + const std::shared_ptr& file) override { + PAIMON_ASSIGN_OR_RAISE(auto upgraded, file->Upgrade(output_level)); + return CompactResult({file}, {upgraded}); + } + + Status Close() override { + return Status::OK(); + } + + private: + bool expected_drop_delete_; + mutable int64_t rewrite_id_ = 0; + std::shared_ptr pool_ = GetDefaultPool(); +}; + +struct LevelMinMax { + int32_t level; + int32_t min; + int32_t max; + + LevelMinMax(int32_t level_value, int32_t min_value, int32_t max_value) + : level(level_value), min(min_value), max(max_value) {} + + explicit LevelMinMax(const std::shared_ptr& file) + : level(file->level), min(file->min_key.GetInt(0)), max(file->max_key.GetInt(0)) {} + + bool operator==(const LevelMinMax& other) const { + return level == other.level && min == other.min && max == other.max; + } +}; + +} // namespace + +class MergeTreeCompactManagerTest : public testing::Test { + protected: + using StrategyFn = + std::function(int32_t, const std::vector&)>; + + void SetUp() override { + pool_ = GetDefaultPool(); + std::vector data_fields; + data_fields.emplace_back(/*id=*/0, arrow::field("f0", arrow::int32())); + ASSERT_OK_AND_ASSIGN(auto comparator, + FieldsComparator::Create(data_fields, /*is_ascending_order=*/true, + /*use_view=*/false)); + comparator_ = std::shared_ptr(std::move(comparator)); + file_id_ = 0; + } + + std::shared_ptr ToFile(const LevelMinMax& minmax, int64_t max_sequence) { + const int64_t file_size = minmax.max - minmax.min + 1; + return std::make_shared( + "f-" + std::to_string(file_id_++), + /*file_size=*/file_size, + /*row_count=*/0, BinaryRowGenerator::GenerateRow({minmax.min}, pool_.get()), + BinaryRowGenerator::GenerateRow({minmax.max}, pool_.get()), SimpleStats::EmptyStats(), + SimpleStats::EmptyStats(), + /*min_sequence_number=*/minmax.min, + /*max_sequence_number=*/max_sequence, + /*schema_id=*/0, minmax.level, std::vector>(), + Timestamp(1, 0), std::nullopt, nullptr, FileSource::Append(), std::nullopt, + std::nullopt, std::nullopt, std::nullopt); + } + + StrategyFn TestStrategy() { + return [](int32_t num_levels, + const std::vector& runs) -> std::optional { + return CompactUnit::FromLevelRuns(num_levels - 1, runs); + }; + } + + Result> CreateLevels( + const std::vector>& files) { + PAIMON_ASSIGN_OR_RAISE(auto levels, Levels::Create(comparator_, files, /*num_levels=*/3)); + return std::shared_ptr(std::move(levels)); + } + + void InnerTest(const std::vector& inputs, + const std::vector& expected) { + InnerTest(inputs, expected, TestStrategy(), /*expected_drop_delete=*/true); + } + + void InnerTest(const std::vector& inputs, const std::vector& expected, + const StrategyFn& strategy, bool expected_drop_delete) { + std::vector> files; + files.reserve(inputs.size()); + for (size_t i = 0; i < inputs.size(); ++i) { + files.push_back(ToFile(inputs[i], static_cast(i))); + } + + ASSERT_OK_AND_ASSIGN(std::shared_ptr levels, CreateLevels(files)); + + auto manager = std::make_shared( + std::make_shared(), levels, + std::make_shared(strategy), comparator_, + /*compaction_file_size=*/2, + /*num_sorted_run_stop_trigger=*/std::numeric_limits::max(), + std::make_shared(expected_drop_delete), + /*metrics_reporter=*/nullptr, + /*dv_maintainer=*/nullptr, + /*lazy_gen_deletion_file=*/false, + /*need_lookup=*/false, + /*force_rewrite_all_files=*/false, + /*force_keep_delete=*/false, std::make_shared()); + + ASSERT_OK(manager->TriggerCompaction(/*full_compaction=*/false)); + ASSERT_OK_AND_ASSIGN(auto compact_result, manager->GetCompactionResult(/*blocking=*/true)); + (void)compact_result; + + std::vector outputs; + for (const auto& file : levels->AllFiles()) { + outputs.emplace_back(file); + } + + ASSERT_EQ(outputs.size(), expected.size()); + EXPECT_EQ(outputs, expected); + } + + std::shared_ptr pool_; + std::shared_ptr comparator_; + int64_t file_id_ = 0; +}; + +TEST_F(MergeTreeCompactManagerTest, TestOutputToZeroLevel) { + InnerTest( + {LevelMinMax(0, 1, 3), LevelMinMax(0, 1, 5), LevelMinMax(0, 1, 8)}, + {LevelMinMax(0, 1, 8), LevelMinMax(0, 1, 3)}, + [](int32_t, const std::vector& runs) -> std::optional { + return CompactUnit::FromLevelRuns(0, {runs[0], runs[1]}); + }, + /*expected_drop_delete=*/false); +} + +TEST_F(MergeTreeCompactManagerTest, TestCompactToPenultimateLayer) { + InnerTest( + {LevelMinMax(0, 1, 3), LevelMinMax(0, 1, 5), LevelMinMax(2, 1, 7)}, + {LevelMinMax(1, 1, 5), LevelMinMax(2, 1, 7)}, + [](int32_t, const std::vector& runs) -> std::optional { + return CompactUnit::FromLevelRuns(1, {runs[0], runs[1]}); + }, + /*expected_drop_delete=*/false); +} + +TEST_F(MergeTreeCompactManagerTest, TestNoCompaction) { + InnerTest( + {LevelMinMax(3, 1, 3)}, {LevelMinMax(3, 1, 3)}, + [](int32_t, const std::vector&) -> std::optional { + return std::nullopt; + }, + /*expected_drop_delete=*/true); +} + +TEST_F(MergeTreeCompactManagerTest, TestNormal) { + InnerTest({LevelMinMax(0, 1, 3), LevelMinMax(1, 1, 5), LevelMinMax(1, 6, 7)}, + {LevelMinMax(2, 1, 5), LevelMinMax(2, 6, 7)}); +} + +TEST_F(MergeTreeCompactManagerTest, TestUpgrade) { + InnerTest({LevelMinMax(0, 1, 3), LevelMinMax(0, 1, 5), LevelMinMax(0, 6, 8)}, + {LevelMinMax(2, 1, 5), LevelMinMax(2, 6, 8)}); +} + +TEST_F(MergeTreeCompactManagerTest, TestSmallFiles) { + InnerTest({LevelMinMax(0, 1, 1), LevelMinMax(0, 2, 2)}, {LevelMinMax(2, 1, 2)}); +} + +TEST_F(MergeTreeCompactManagerTest, TestSmallFilesNoCompact) { + InnerTest( + {LevelMinMax(0, 1, 5), LevelMinMax(0, 6, 6), LevelMinMax(1, 7, 8), LevelMinMax(1, 9, 10)}, + {LevelMinMax(2, 1, 5), LevelMinMax(2, 6, 6), LevelMinMax(2, 7, 8), LevelMinMax(2, 9, 10)}); +} + +TEST_F(MergeTreeCompactManagerTest, TestSmallFilesCrossLevel) { + InnerTest( + {LevelMinMax(0, 1, 5), LevelMinMax(0, 6, 6), LevelMinMax(1, 7, 7), LevelMinMax(1, 9, 10)}, + {LevelMinMax(2, 1, 5), LevelMinMax(2, 6, 7), LevelMinMax(2, 9, 10)}); +} + +TEST_F(MergeTreeCompactManagerTest, TestComplex) { + InnerTest( + {LevelMinMax(0, 1, 5), LevelMinMax(0, 6, 6), LevelMinMax(1, 1, 4), LevelMinMax(1, 6, 8), + LevelMinMax(1, 10, 11), LevelMinMax(2, 1, 3), LevelMinMax(2, 4, 6)}, + {LevelMinMax(2, 1, 8), LevelMinMax(2, 10, 11)}); +} + +TEST_F(MergeTreeCompactManagerTest, TestSmallInComplex) { + InnerTest( + {LevelMinMax(0, 1, 5), LevelMinMax(0, 6, 6), LevelMinMax(1, 1, 4), LevelMinMax(1, 6, 8), + LevelMinMax(1, 10, 10), LevelMinMax(2, 1, 3), LevelMinMax(2, 4, 6)}, + {LevelMinMax(2, 1, 10)}); +} + +TEST_F(MergeTreeCompactManagerTest, TestIsCompacting) { + std::vector inputs = {LevelMinMax(0, 1, 3), LevelMinMax(1, 1, 5), + LevelMinMax(1, 6, 7)}; + std::vector> files; + files.reserve(inputs.size()); + for (size_t i = 0; i < inputs.size(); ++i) { + files.push_back(ToFile(inputs[i], static_cast(i))); + } + + ASSERT_OK_AND_ASSIGN(std::shared_ptr lookup_levels, CreateLevels(files)); + ASSERT_OK_AND_ASSIGN(std::shared_ptr default_levels, CreateLevels(files)); + + auto strategy = std::make_shared(TestStrategy()); + + auto lookup_manager = std::make_shared( + std::make_shared(), lookup_levels, strategy, comparator_, + /*compaction_file_size=*/2, + /*num_sorted_run_stop_trigger=*/std::numeric_limits::max(), + std::make_shared(/*expected_drop_delete=*/true), + /*metrics_reporter=*/nullptr, + /*dv_maintainer=*/nullptr, + /*lazy_gen_deletion_file=*/false, + /*need_lookup=*/true, + /*force_rewrite_all_files=*/false, + /*force_keep_delete=*/false, std::make_shared()); + + auto default_manager = std::make_shared( + std::make_shared(), default_levels, strategy, comparator_, + /*compaction_file_size=*/2, + /*num_sorted_run_stop_trigger=*/std::numeric_limits::max(), + std::make_shared(/*expected_drop_delete=*/true), + /*metrics_reporter=*/nullptr, + /*dv_maintainer=*/nullptr, + /*lazy_gen_deletion_file=*/false, + /*need_lookup=*/false, + /*force_rewrite_all_files=*/false, + /*force_keep_delete=*/false, std::make_shared()); + + EXPECT_TRUE(lookup_manager->CompactNotCompleted()); + EXPECT_FALSE(default_manager->CompactNotCompleted()); +} + +TEST_F(MergeTreeCompactManagerTest, TestTriggerFullCompaction) { + std::vector inputs = {LevelMinMax(0, 1, 3), LevelMinMax(1, 2, 5), + LevelMinMax(1, 6, 7)}; + std::vector> files; + files.reserve(inputs.size()); + for (size_t i = 0; i < inputs.size(); ++i) { + files.push_back(ToFile(inputs[i], static_cast(i))); + } + + ASSERT_OK_AND_ASSIGN(std::shared_ptr levels, CreateLevels(files)); + + auto manager = std::make_shared( + std::make_shared(), levels, + std::make_shared(TestStrategy()), comparator_, + /*compaction_file_size=*/2, + /*num_sorted_run_stop_trigger=*/std::numeric_limits::max(), + std::make_shared(/*expected_drop_delete=*/true), + /*metrics_reporter=*/nullptr, + /*dv_maintainer=*/nullptr, + /*lazy_gen_deletion_file=*/false, + /*need_lookup=*/false, + /*force_rewrite_all_files=*/false, + /*force_keep_delete=*/false, std::make_shared()); + + ASSERT_OK(manager->TriggerCompaction(/*full_compaction=*/true)); + ASSERT_OK_AND_ASSIGN(auto compact_result, manager->GetCompactionResult(/*blocking=*/true)); + ASSERT_TRUE(compact_result.has_value()); + ASSERT_FALSE(compact_result.value()->Before().empty()); + ASSERT_FALSE(compact_result.value()->After().empty()); + for (const auto& after : compact_result.value()->After()) { + EXPECT_EQ(after->level, 2); + } +} + +TEST_F(MergeTreeCompactManagerTest, TestRejectReentrantFullCompaction) { + std::vector inputs = {LevelMinMax(0, 1, 3), LevelMinMax(1, 2, 5), + LevelMinMax(1, 6, 7)}; + std::vector> files; + files.reserve(inputs.size()); + for (size_t i = 0; i < inputs.size(); ++i) { + files.push_back(ToFile(inputs[i], static_cast(i))); + } + + ASSERT_OK_AND_ASSIGN(std::shared_ptr levels, CreateLevels(files)); + + auto queued_executor = std::make_shared(); + auto manager = std::make_shared( + queued_executor, levels, std::make_shared(TestStrategy()), + comparator_, + /*compaction_file_size=*/2, + /*num_sorted_run_stop_trigger=*/std::numeric_limits::max(), + std::make_shared(/*expected_drop_delete=*/true), + /*metrics_reporter=*/nullptr, + /*dv_maintainer=*/nullptr, + /*lazy_gen_deletion_file=*/false, + /*need_lookup=*/false, + /*force_rewrite_all_files=*/false, + /*force_keep_delete=*/false, std::make_shared()); + + ASSERT_OK(manager->TriggerCompaction(/*full_compaction=*/true)); + Status status = manager->TriggerCompaction(/*full_compaction=*/true); + ASSERT_TRUE(status.IsInvalid()); + + queued_executor->RunAll(); + ASSERT_OK_AND_ASSIGN(auto compact_result, manager->GetCompactionResult(/*blocking=*/true)); + ASSERT_TRUE(compact_result.has_value()); +} + +} // 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 54dd9b56..bd17ea99 100644 --- a/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_rewriter.cpp @@ -15,6 +15,8 @@ */ #include "paimon/core/mergetree/compact/merge_tree_compact_rewriter.h" +#include + #include "arrow/c/bridge.h" #include "arrow/c/helpers.h" #include "paimon/common/table/special_fields.h" @@ -38,7 +40,8 @@ MergeTreeCompactRewriter::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) + const std::shared_ptr& pool, + const std::shared_ptr& cancellation_controller) : options_(options), merge_file_split_read_(std::move(merge_file_split_read)), pool_(pool), @@ -50,13 +53,17 @@ MergeTreeCompactRewriter::MergeTreeCompactRewriter( 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)) {} + merge_function_wrapper_factory_(std::move(merge_function_wrapper_factory)), + cancellation_controller_(cancellation_controller) { + assert(cancellation_controller_ != nullptr); +} 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) { + const CoreOptions& options, const std::shared_ptr& pool, + const std::shared_ptr& cancellation_controller) { PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, table_schema->TrimmedPrimaryKeys()); auto data_schema = DataField::ConvertDataFieldsToArrowSchema(table_schema->Fields()); @@ -84,7 +91,7 @@ Result> MergeTreeCompactRewriter::Crea return std::unique_ptr(new MergeTreeCompactRewriter( partition, bucket, table_schema->Id(), trimmed_primary_keys, options, data_schema, write_schema, std::move(dv_factory), path_factory_cache, std::move(merge_file_split_read), - merge_function_wrapper_factory, pool)); + merge_function_wrapper_factory, pool, cancellation_controller)); } Result MergeTreeCompactRewriter::Upgrade(int32_t output_level, @@ -201,6 +208,9 @@ Status MergeTreeCompactRewriter::MergeReadAndWrite( reader_holders.push_back(async_key_value_producer_consumer); // read KeyValueBatch from SortMergeReader and write to RollingWriter while (true) { + if (cancellation_controller_->IsCancelled()) { + return Status::Cancelled("Compaction is cancelled"); + } PAIMON_ASSIGN_OR_RAISE(KeyValueBatch key_value_batch, async_key_value_producer_consumer->NextBatch()); if (key_value_batch.batch == nullptr) { 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 06cffdaf..ce1bc1be 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,9 @@ */ #pragma once + #include "arrow/api.h" +#include "paimon/core/compact/cancellation_controller.h" #include "paimon/core/core_options.h" #include "paimon/core/io/async_key_value_producer_and_consumer.h" #include "paimon/core/io/data_file_meta.h" @@ -38,7 +40,8 @@ class MergeTreeCompactRewriter : public CompactRewriter { 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& memory_pool); + const CoreOptions& options, const std::shared_ptr& memory_pool, + const std::shared_ptr& cancellation_controller); Result Rewrite(int32_t output_level, bool drop_delete, const std::vector>& sections) override; @@ -65,16 +68,16 @@ class MergeTreeCompactRewriter : public CompactRewriter { static std::vector> ExtractFilesFromSections( const std::vector>& sections); - 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, - 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( + 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, 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, + const std::shared_ptr& cancellation_controller); using KeyValueRollingFileWriter = RollingFileWriter>; @@ -113,6 +116,7 @@ class MergeTreeCompactRewriter : public CompactRewriter { DeletionVector::Factory dv_factory_; std::shared_ptr path_factory_cache_; MergeFunctionWrapperFactory merge_function_wrapper_factory_; + std::shared_ptr cancellation_controller_; }; } // 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 4b309f88..1524d99e 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 @@ -44,10 +44,12 @@ class MergeTreeCompactRewriterTest : public testing::Test { return std::shared_ptr(); }; + auto cancellation_controller = std::make_shared(); auto path_factory_cache = std::make_shared(table_path, table_schema, options, pool_); return MergeTreeCompactRewriter::Create(bucket, partition, table_schema, dv_factory, - path_factory_cache, options, pool_); + path_factory_cache, options, pool_, + cancellation_controller); } Result>> GenerateSortedRuns( diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_task.cpp b/src/paimon/core/mergetree/compact/merge_tree_compact_task.cpp new file mode 100644 index 00000000..07698abe --- /dev/null +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_task.cpp @@ -0,0 +1,125 @@ +/* + * 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/merge_tree_compact_task.h" + +namespace paimon { + +MergeTreeCompactTask::MergeTreeCompactTask( + const std::shared_ptr& key_comparator, int64_t min_file_size, + const std::shared_ptr& rewriter, const CompactUnit& unit, bool drop_delete, + int32_t max_level, const std::shared_ptr& metrics_reporter, + DeletionFileSupplier compact_df_supplier, bool force_rewrite_all_files) + : CompactTask(metrics_reporter), + min_file_size_(min_file_size), + rewriter_(rewriter), + output_level_(unit.output_level), + compact_df_supplier_(std::move(compact_df_supplier)), + partitioned_(IntervalPartition(unit.files, key_comparator).Partition()), + drop_delete_(drop_delete), + max_level_(max_level), + force_rewrite_all_files_(force_rewrite_all_files) {} + +Result> MergeTreeCompactTask::DoCompact() { + std::vector> candidate; + auto result = std::make_shared(); + + // Checking the order and compacting adjacent and contiguous files + // Note: can't skip an intermediate file to compact, this will destroy the overall orderliness + for (const auto& section : partitioned_) { + if (section.size() > 1) { + candidate.push_back(section); + } else if (!section.empty()) { + const auto& run = section[0]; + // No overlapping: + // We can just upgrade the large file and just change the level instead of rewriting + // it. But for small files, we will try to compact it. + for (const auto& file : run.Files()) { + if (file->file_size < min_file_size_) { + // Smaller files are rewritten along with the previous files. + candidate.push_back({SortedRun::FromSingle(file)}); + } else { + // Large file appears, rewrite previous and upgrade it. + PAIMON_RETURN_NOT_OK(Rewrite(candidate, result.get())); + PAIMON_RETURN_NOT_OK(Upgrade(file, result.get())); + } + } + } else { + assert(false); + return Status::Invalid("Unexpected empty section in partitioned runs"); + } + } + + PAIMON_RETURN_NOT_OK(Rewrite(candidate, result.get())); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr deletion_file, + compact_df_supplier_()); + result->SetDeletionFile(deletion_file); + return result; +} + +Status MergeTreeCompactTask::Upgrade(const std::shared_ptr& file, + CompactResult* to_update) { + // TODO(yonghao.fyh): check expire + if ((output_level_ == max_level_ && ContainsDeleteRecords(file)) || force_rewrite_all_files_) { + std::vector> candidate = {{SortedRun::FromSingle(file)}}; + return RewriteImpl(candidate, to_update); + } + + if (file->level != output_level_) { + PAIMON_ASSIGN_OR_RAISE(CompactResult upgraded, rewriter_->Upgrade(output_level_, file)); + PAIMON_RETURN_NOT_OK(to_update->Merge(upgraded)); + ++upgrade_files_num_; + } + return Status::OK(); +} + +Status MergeTreeCompactTask::Rewrite(std::vector>& candidate, + CompactResult* to_update) { + if (candidate.empty()) { + return Status::OK(); + } + + if (candidate.size() == 1) { + const auto& section = candidate[0]; + if (section.empty()) { + return Status::OK(); + } + if (section.size() == 1) { + for (const auto& file : section[0].Files()) { + PAIMON_RETURN_NOT_OK(Upgrade(file, to_update)); + } + candidate.clear(); + return Status::OK(); + } + } + + return RewriteImpl(candidate, to_update); +} + +Status MergeTreeCompactTask::RewriteImpl(std::vector>& candidate, + CompactResult* to_update) { + PAIMON_ASSIGN_OR_RAISE(CompactResult rewritten, + rewriter_->Rewrite(output_level_, drop_delete_, candidate)); + PAIMON_RETURN_NOT_OK(to_update->Merge(rewritten)); + candidate.clear(); + return Status::OK(); +} + +bool MergeTreeCompactTask::ContainsDeleteRecords(const std::shared_ptr& file) { + return !file->delete_row_count.has_value() || file->delete_row_count.value() > 0; +} + +} // namespace paimon diff --git a/src/paimon/core/mergetree/compact/merge_tree_compact_task.h b/src/paimon/core/mergetree/compact/merge_tree_compact_task.h new file mode 100644 index 00000000..b9100de0 --- /dev/null +++ b/src/paimon/core/mergetree/compact/merge_tree_compact_task.h @@ -0,0 +1,69 @@ +/* + * 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 + +#include "paimon/core/compact/compact_deletion_file.h" +#include "paimon/core/compact/compact_task.h" +#include "paimon/core/compact/compact_unit.h" +#include "paimon/core/mergetree/compact/compact_rewriter.h" +#include "paimon/core/mergetree/compact/interval_partition.h" +#include "paimon/core/mergetree/sorted_run.h" +#include "paimon/core/utils/fields_comparator.h" + +namespace paimon { + +/// Compact task for merge tree compaction. +class MergeTreeCompactTask : public CompactTask { + public: + using DeletionFileSupplier = std::function>()>; + + // TODO(yonghao.fyh): Support RecordLevelExpire + MergeTreeCompactTask(const std::shared_ptr& key_comparator, + int64_t min_file_size, const std::shared_ptr& rewriter, + const CompactUnit& unit, bool drop_delete, int32_t max_level, + const std::shared_ptr& metrics_reporter, + DeletionFileSupplier compact_df_supplier, bool force_rewrite_all_files); + + protected: + Result> DoCompact() override; + + private: + Status Upgrade(const std::shared_ptr& file, CompactResult* to_update); + + Status Rewrite(std::vector>& candidate, CompactResult* to_update); + + Status RewriteImpl(std::vector>& candidate, CompactResult* to_update); + + static bool ContainsDeleteRecords(const std::shared_ptr& file); + + int64_t min_file_size_; + std::shared_ptr rewriter_; + int32_t output_level_; + DeletionFileSupplier compact_df_supplier_; + std::vector> partitioned_; + bool drop_delete_; + int32_t max_level_; + bool force_rewrite_all_files_; + int32_t upgrade_files_num_ = 0; +}; + +} // namespace paimon diff --git a/src/paimon/core/mergetree/lookup_levels_test.cpp b/src/paimon/core/mergetree/lookup_levels_test.cpp index 34d1b564..e45439d9 100644 --- a/src/paimon/core/mergetree/lookup_levels_test.cpp +++ b/src/paimon/core/mergetree/lookup_levels_test.cpp @@ -23,6 +23,7 @@ #include "gtest/gtest.h" #include "paimon/catalog/catalog.h" #include "paimon/common/utils/path_util.h" +#include "paimon/core/compact/noop_compact_manager.h" #include "paimon/core/core_options.h" #include "paimon/core/io/data_file_path_factory.h" #include "paimon/core/mergetree/compact/deduplicate_merge_function.h" @@ -51,6 +52,7 @@ class LookupLevelsTest : public testing::Test { tmp_dir_ = UniqueTestDirectory::Create("local"); dir_ = UniqueTestDirectory::Create("local"); fs_ = dir_->GetFileSystem(); + noop_compact_manager_ = std::make_shared(); } void TearDown() override {} @@ -77,7 +79,7 @@ class LookupLevelsTest : public testing::Test { /*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_); + arrow_schema_, options, noop_compact_manager_, pool_); // write data ArrowArray c_src_array; @@ -163,6 +165,7 @@ class LookupLevelsTest : public testing::Test { std::unique_ptr tmp_dir_; std::unique_ptr dir_; std::shared_ptr fs_; + std::shared_ptr noop_compact_manager_; }; TEST_F(LookupLevelsTest, TestMultiLevels) { diff --git a/src/paimon/core/mergetree/merge_tree_writer.cpp b/src/paimon/core/mergetree/merge_tree_writer.cpp index dd14278e..50043738 100644 --- a/src/paimon/core/mergetree/merge_tree_writer.cpp +++ b/src/paimon/core/mergetree/merge_tree_writer.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include "arrow/api.h" @@ -67,7 +68,8 @@ MergeTreeWriter::MergeTreeWriter( const std::shared_ptr& user_defined_seq_comparator, const std::shared_ptr>& merge_function_wrapper, int64_t schema_id, const std::shared_ptr& value_schema, - const CoreOptions& options, const std::shared_ptr& pool) + const CoreOptions& options, const std::shared_ptr& compact_manager, + const std::shared_ptr& pool) : last_sequence_number_(last_sequence_number + 1), current_memory_in_bytes_(0), pool_(pool), @@ -79,6 +81,7 @@ MergeTreeWriter::MergeTreeWriter( merge_function_wrapper_(merge_function_wrapper), schema_id_(schema_id), value_type_(arrow::struct_(value_schema->fields())), + compact_manager_(compact_manager), metrics_(std::make_shared()) { write_schema_ = SpecialFields::CompleteSequenceAndValueKindField(value_schema); } @@ -101,77 +104,181 @@ Status MergeTreeWriter::Write(std::unique_ptr&& moved_batch) { batch_vec_.push_back(std::move(value_struct_array)); row_kinds_vec_.push_back(batch->GetRowKind()); if (current_memory_in_bytes_ >= options_.GetWriteBufferSize()) { - return Flush(); + return Flush(/*wait_for_latest_compaction=*/false, /*forced_full_compaction=*/false); } return Status::OK(); } -Result MergeTreeWriter::PrepareCommit(bool wait_compaction) { - // TODO(xinyu.lxy): support wait_compaction - PAIMON_RETURN_NOT_OK(Flush()); - return DrainIncrement(); +Status MergeTreeWriter::Compact(bool full_compaction) { + return Flush(/*wait_for_latest_compaction=*/true, full_compaction); } -Status MergeTreeWriter::Flush() { - if (batch_vec_.empty()) { - return Status::OK(); +Status MergeTreeWriter::Sync() { + return TrySyncLatestCompaction(/*blocking=*/true); +} + +Status MergeTreeWriter::TrySyncLatestCompaction(bool blocking) { + PAIMON_ASSIGN_OR_RAISE(std::optional> result, + compact_manager_->GetCompactionResult(blocking)); + if (result) { + PAIMON_RETURN_NOT_OK(UpdateCompactResult(result.value())); } - // 1. create key value iter for each record batch - std::vector> readers; - readers.reserve(batch_vec_.size()); - for (size_t i = 0; i < batch_vec_.size(); ++i) { - int64_t sequence_number = last_sequence_number_; - last_sequence_number_ += batch_vec_[i]->length(); - auto in_memory_reader = std::make_unique( - sequence_number, std::move(batch_vec_[i]), std::move(row_kinds_vec_[i]), - trimmed_primary_keys_, options_.GetSequenceField(), key_comparator_, - merge_function_wrapper_, pool_); - readers.push_back(std::move(in_memory_reader)); + return Status::OK(); +} + +Status MergeTreeWriter::UpdateCompactResult(const std::shared_ptr& compact_result) { + std::unordered_set after_files; + after_files.reserve(compact_result->After().size()); + for (const auto& file : compact_result->After()) { + after_files.insert(file->file_name); } - batch_vec_.clear(); - row_kinds_vec_.clear(); - current_memory_in_bytes_ = 0; - // 2. prepare loser tree sort merge reader - auto sort_merge_reader = std::make_unique( - std::move(readers), key_comparator_, user_defined_seq_comparator_, merge_function_wrapper_); - // 3. project key value to arrow array - auto create_consumer = [target_schema = write_schema_, pool = pool_]() - -> Result>> { - return KeyValueMetaProjectionConsumer::Create(target_schema, pool); + + auto in_compact_before = [this](const std::string& file_name) { + return std::any_of(compact_before_.begin(), compact_before_.end(), + [&file_name](const std::shared_ptr& meta) { + return meta->file_name == file_name; + }); }; - // consumer batch size is WriteBatchSize - auto async_key_value_producer_consumer = - std::make_unique>( - std::move(sort_merge_reader), create_consumer, options_.GetWriteBatchSize(), - /*projection_thread_num=*/1, pool_); - auto rolling_writer = CreateRollingRowWriter(); - ScopeGuard write_guard([&]() -> void { - rolling_writer->Abort(); - async_key_value_producer_consumer->Close(); - }); - while (true) { - PAIMON_ASSIGN_OR_RAISE(KeyValueBatch key_value_batch, - async_key_value_producer_consumer->NextBatch()); - if (key_value_batch.batch == nullptr) { - break; + + for (const auto& file : compact_result->Before()) { + auto compact_after_it = + std::find_if(compact_after_.begin(), compact_after_.end(), + [&file](const std::shared_ptr& candidate) { + return candidate->file_name == file->file_name; + }); + if (compact_after_it != compact_after_.end()) { + compact_after_.erase(compact_after_it); + // This is an intermediate file (not a new data file), which is no longer needed + // after compaction and can be deleted directly, but upgrade file is required by + // previous snapshot and following snapshot, so we should ensure: + // 1. This file is not the output of upgraded. + // 2. This file is not the input of upgraded. + if (!in_compact_before(file->file_name) && + after_files.find(file->file_name) == after_files.end()) { + auto fs = options_.GetFileSystem(); + [[maybe_unused]] auto s = fs->Delete(path_factory_->ToPath(file)); + } + } else { + compact_before_.push_back(file); + } + } + + compact_after_.insert(compact_after_.end(), compact_result->After().begin(), + compact_result->After().end()); + // TODO(yonghao.fyh): support compact changelog + return UpdateCompactDeletionFile(compact_result->DeletionFile()); +} + +Status MergeTreeWriter::UpdateCompactDeletionFile( + const std::shared_ptr& new_deletion_file) { + if (new_deletion_file) { + if (compact_deletion_file_ == nullptr) { + compact_deletion_file_ = new_deletion_file; + } else { + PAIMON_ASSIGN_OR_RAISE(compact_deletion_file_, + new_deletion_file->MergeOldFile(compact_deletion_file_)); + } + } + return Status::OK(); +} + +Result MergeTreeWriter::PrepareCommit(bool wait_compaction) { + PAIMON_RETURN_NOT_OK(Flush(wait_compaction, /*forced_full_compaction=*/false)); + if (options_.CommitForceCompact()) { + wait_compaction = true; + } + // Decide again whether to wait here. + // For example, in the case of repeated failures in writing, it is possible that Level 0 + // files were successfully committed, but failed to restart during the compaction phase, + // which may result in an increasing number of Level 0 files. This wait can avoid this + // situation. + if (compact_manager_->ShouldWaitForPreparingCheckpoint()) { + wait_compaction = true; + } + PAIMON_RETURN_NOT_OK(TrySyncLatestCompaction(wait_compaction)); + return DrainIncrement(); +} + +Result MergeTreeWriter::CompactNotCompleted() { + PAIMON_RETURN_NOT_OK(compact_manager_->TriggerCompaction(/*full_compaction=*/false)); + return compact_manager_->CompactNotCompleted(); +} + +Status MergeTreeWriter::Flush(bool wait_for_latest_compaction, bool forced_full_compaction) { + if (!batch_vec_.empty()) { + if (compact_manager_->ShouldWaitForLatestCompaction()) { + wait_for_latest_compaction = true; + } + // 1. create key value iter for each record batch + std::vector> readers; + readers.reserve(batch_vec_.size()); + for (size_t i = 0; i < batch_vec_.size(); ++i) { + int64_t sequence_number = last_sequence_number_; + last_sequence_number_ += batch_vec_[i]->length(); + auto in_memory_reader = std::make_unique( + sequence_number, std::move(batch_vec_[i]), std::move(row_kinds_vec_[i]), + trimmed_primary_keys_, options_.GetSequenceField(), key_comparator_, + merge_function_wrapper_, pool_); + readers.push_back(std::move(in_memory_reader)); + } + batch_vec_.clear(); + row_kinds_vec_.clear(); + current_memory_in_bytes_ = 0; + // 2. prepare loser tree sort merge reader + auto sort_merge_reader = std::make_unique( + std::move(readers), key_comparator_, user_defined_seq_comparator_, + merge_function_wrapper_); + // 3. project key value to arrow array + auto create_consumer = [target_schema = write_schema_, pool = pool_]() + -> Result>> { + return KeyValueMetaProjectionConsumer::Create(target_schema, pool); + }; + // consumer batch size is WriteBatchSize + auto async_key_value_producer_consumer = + std::make_unique>( + std::move(sort_merge_reader), create_consumer, options_.GetWriteBatchSize(), + /*projection_thread_num=*/1, pool_); + auto rolling_writer = CreateRollingRowWriter(); + ScopeGuard write_guard([&]() -> void { + rolling_writer->Abort(); + async_key_value_producer_consumer->Close(); + }); + while (true) { + PAIMON_ASSIGN_OR_RAISE(KeyValueBatch key_value_batch, + async_key_value_producer_consumer->NextBatch()); + if (key_value_batch.batch == nullptr) { + break; + } + PAIMON_RETURN_NOT_OK(rolling_writer->Write(std::move(key_value_batch))); } - PAIMON_RETURN_NOT_OK(rolling_writer->Write(std::move(key_value_batch))); + PAIMON_RETURN_NOT_OK(rolling_writer->Close()); + PAIMON_ASSIGN_OR_RAISE(std::vector> flushed_files, + rolling_writer->GetResult()); + write_guard.Release(); + + for (const auto& flushed_file : flushed_files) { + new_files_.emplace_back(flushed_file); + PAIMON_RETURN_NOT_OK(compact_manager_->AddNewFile(flushed_file)); + } + metrics_->Merge(rolling_writer->GetMetrics()); } - PAIMON_RETURN_NOT_OK(rolling_writer->Close()); - PAIMON_ASSIGN_OR_RAISE(std::vector> flushed_files, - rolling_writer->GetResult()); - write_guard.Release(); - new_files_.insert(new_files_.end(), flushed_files.begin(), flushed_files.end()); - metrics_->Merge(rolling_writer->GetMetrics()); + PAIMON_RETURN_NOT_OK(TrySyncLatestCompaction(wait_for_latest_compaction)); + PAIMON_RETURN_NOT_OK(compact_manager_->TriggerCompaction(forced_full_compaction)); return Status::OK(); } Result MergeTreeWriter::DrainIncrement() { DataIncrement data_increment(std::move(new_files_), std::move(deleted_files_), {}); - CompactIncrement compact_increment({}, {}, {}); + CompactIncrement compact_increment(std::move(compact_before_), std::move(compact_after_), {}); + auto drain_deletion_file = compact_deletion_file_; + new_files_.clear(); deleted_files_.clear(); - return CommitIncrement(data_increment, compact_increment, /*compact_deletion_file=*/nullptr); + compact_before_.clear(); + compact_after_.clear(); + compact_deletion_file_ = nullptr; + + return CommitIncrement(data_increment, compact_increment, drain_deletion_file); } std::unique_ptr>> diff --git a/src/paimon/core/mergetree/merge_tree_writer.h b/src/paimon/core/mergetree/merge_tree_writer.h index ce457aab..64c64d03 100644 --- a/src/paimon/core/mergetree/merge_tree_writer.h +++ b/src/paimon/core/mergetree/merge_tree_writer.h @@ -21,6 +21,7 @@ #include #include "arrow/api.h" +#include "paimon/core/compact/compact_manager.h" #include "paimon/core/core_options.h" #include "paimon/core/io/data_file_meta.h" #include "paimon/core/io/data_file_path_factory.h" @@ -59,23 +60,17 @@ class MergeTreeWriter : public BatchWriter { const std::shared_ptr& user_defined_seq_comparator, const std::shared_ptr>& merge_function_wrapper, int64_t schema_id, const std::shared_ptr& value_schema, - const CoreOptions& options, const std::shared_ptr& pool); + const CoreOptions& options, + const std::shared_ptr& compact_manager, + const std::shared_ptr& pool); - ~MergeTreeWriter() override { - [[maybe_unused]] auto status = DoClose(); - } Status Write(std::unique_ptr&& batch) override; - Status Compact(bool full_compaction) override { - return Status::NotImplemented("not implemented"); - } - Result CompactNotCompleted() override { - return false; - } + Status Compact(bool full_compaction) override; - Status Sync() override { - return Status::NotImplemented("not implemented"); - } + Result CompactNotCompleted() override; + + Status Sync() override; Result PrepareCommit(bool wait_compaction) override; @@ -89,16 +84,31 @@ class MergeTreeWriter : public BatchWriter { private: Status DoClose() { + // cancel compaction so that it does not block job cancelling + compact_manager_->CancelCompaction(); + PAIMON_RETURN_NOT_OK(Sync()); + PAIMON_RETURN_NOT_OK(compact_manager_->Close()); + batch_vec_.clear(); row_kinds_vec_.clear(); + + if (compact_deletion_file_) { + compact_deletion_file_->Clean(); + compact_deletion_file_.reset(); + } return Status::OK(); } - Status Flush(); + Status Flush(bool wait_for_latest_compaction, bool forced_full_compaction); Result DrainIncrement(); std::unique_ptr>> CreateRollingRowWriter() const; + + Status TrySyncLatestCompaction(bool blocking); + Status UpdateCompactResult(const std::shared_ptr& compact_result); + Status UpdateCompactDeletionFile(const std::shared_ptr& new_deletion_file); + static Result EstimateMemoryUse(const std::shared_ptr& array); private: @@ -116,11 +126,18 @@ class MergeTreeWriter : public BatchWriter { std::shared_ptr value_type_; std::shared_ptr write_schema_; + std::shared_ptr compact_manager_; + std::vector> batch_vec_; std::vector> row_kinds_vec_; std::shared_ptr metrics_; + std::vector> new_files_; std::vector> deleted_files_; + std::vector> compact_before_; + std::vector> compact_after_; + + std::shared_ptr compact_deletion_file_; }; } // namespace paimon diff --git a/src/paimon/core/mergetree/merge_tree_writer_test.cpp b/src/paimon/core/mergetree/merge_tree_writer_test.cpp index 98c5422a..a5a23580 100644 --- a/src/paimon/core/mergetree/merge_tree_writer_test.cpp +++ b/src/paimon/core/mergetree/merge_tree_writer_test.cpp @@ -35,6 +35,7 @@ #include "paimon/common/table/special_fields.h" #include "paimon/common/types/data_field.h" #include "paimon/common/utils/scope_guard.h" +#include "paimon/core/compact/noop_compact_manager.h" #include "paimon/core/io/compact_increment.h" #include "paimon/core/io/data_file_path_factory.h" #include "paimon/core/io/data_increment.h" @@ -83,6 +84,7 @@ class MergeTreeWriterTest : public ::testing::Test { auto mfunc = std::make_unique(/*ignore_delete=*/false); merge_function_wrapper_ = std::make_shared(std::move(mfunc)); + noop_compact_manager_ = std::make_shared(); } void TearDown() override {} @@ -121,6 +123,7 @@ class MergeTreeWriterTest : public ::testing::Test { std::shared_ptr write_type_; std::shared_ptr key_comparator_; std::shared_ptr> merge_function_wrapper_; + std::shared_ptr noop_compact_manager_; }; TEST_F(MergeTreeWriterTest, TestSimple) { @@ -136,7 +139,7 @@ TEST_F(MergeTreeWriterTest, TestSimple) { auto merge_writer = std::make_shared( /*last_sequence_number=*/-1, primary_keys_, path_factory, key_comparator_, /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper_, /*schema_id=*/1, - value_schema_, options, pool_); + value_schema_, options, noop_compact_manager_, pool_); // write batch std::shared_ptr array1 = @@ -208,7 +211,7 @@ TEST_F(MergeTreeWriterTest, TestWriteMultiBatch) { auto merge_writer = std::make_shared( /*last_sequence_number=*/9, primary_keys_, path_factory, key_comparator_, /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper_, /*schema_id=*/0, - value_schema_, options, pool_); + value_schema_, options, noop_compact_manager_, pool_); // batch1 std::shared_ptr array1 = arrow::ipc::internal::json::ArrayFromJSON(value_type_, R"([ @@ -296,7 +299,7 @@ TEST_F(MergeTreeWriterTest, TestWriteWithDeleteRow) { auto merge_writer = std::make_shared( /*last_sequence_number=*/9, primary_keys_, path_factory, key_comparator_, user_defined_seq_comparator, merge_function_wrapper_, /*schema_id=*/0, value_schema_, - options, pool_); + options, noop_compact_manager_, pool_); // batch1 std::shared_ptr array1 = arrow::ipc::internal::json::ArrayFromJSON(value_type_, R"([ @@ -372,7 +375,7 @@ TEST_F(MergeTreeWriterTest, TestMultiplePrepareCommit) { auto merge_writer = std::make_shared( /*last_sequence_number=*/9, primary_keys_, path_factory, key_comparator_, /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper_, /*schema_id=*/0, - value_schema_, options, pool_); + value_schema_, options, noop_compact_manager_, pool_); // batch1 std::shared_ptr array1 = arrow::ipc::internal::json::ArrayFromJSON(value_type_, R"([ @@ -508,7 +511,7 @@ TEST_F(MergeTreeWriterTest, TestPrepareCommitForEmptyData) { auto merge_writer = std::make_shared( /*last_sequence_number=*/-1, primary_keys_, path_factory, key_comparator_, /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper_, /*schema_id=*/0, - value_schema_, options, pool_); + value_schema_, options, noop_compact_manager_, pool_); // prepare commit, without write ASSERT_OK_AND_ASSIGN(CommitIncrement commit_increment, @@ -548,7 +551,7 @@ TEST_F(MergeTreeWriterTest, TestCloseBeforePrepareCommit) { auto merge_writer = std::make_shared( /*last_sequence_number=*/-1, primary_keys_, path_factory, key_comparator_, /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper_, /*schema_id=*/0, - value_schema_, options, pool_); + value_schema_, options, noop_compact_manager_, pool_); // write batch std::shared_ptr array1 = @@ -577,7 +580,7 @@ TEST_F(MergeTreeWriterTest, TestAutoFlush) { auto merge_writer = std::make_shared( /*last_sequence_number=*/9, primary_keys_, path_factory, key_comparator_, /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper_, /*schema_id=*/0, - value_schema_, options, pool_); + value_schema_, options, noop_compact_manager_, pool_); // batch1 std::shared_ptr array1 = arrow::ipc::internal::json::ArrayFromJSON(value_type_, R"([ @@ -699,7 +702,7 @@ TEST_F(MergeTreeWriterTest, TestIOException) { auto merge_writer = std::make_shared( /*last_sequence_number=*/-1, primary_keys_, path_factory, key_comparator_, /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper_, /*schema_id=*/0, - value_schema_, options, pool_); + value_schema_, options, noop_compact_manager_, pool_); // write batch std::shared_ptr array = @@ -810,7 +813,7 @@ TEST_F(MergeTreeWriterTest, TestBulkData) { auto merge_writer = std::make_shared( /*last_sequence_number=*/-1, primary_keys_, path_factory, key_comparator_, /*user_defined_seq_comparator=*/nullptr, merge_function_wrapper_, /*schema_id=*/0, - value_schema_, options, pool_); + value_schema_, options, noop_compact_manager_, pool_); // multi batch size_t batch_size = 500; for (size_t i = 0; i < batch_size; ++i) { diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index 5448a06c..eeb79e8d 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -187,8 +187,9 @@ Result>> AbstractFileStoreWrite::Prep auto compact_deletion_file = increment.GetCompactDeletionFile(); auto& compact_increment = increment.GetCompactIncrement(); if (compact_deletion_file) { - std::optional> dv_index_file_meta = - compact_deletion_file->GetOrCompute(); + PAIMON_ASSIGN_OR_RAISE( + std::optional> dv_index_file_meta, + compact_deletion_file->GetOrCompute()); if (dv_index_file_meta) { compact_increment.AddNewIndexFiles({dv_index_file_meta.value()}); } diff --git a/src/paimon/core/operation/abstract_split_read.cpp b/src/paimon/core/operation/abstract_split_read.cpp index 539730f3..f5f37631 100644 --- a/src/paimon/core/operation/abstract_split_read.cpp +++ b/src/paimon/core/operation/abstract_split_read.cpp @@ -124,21 +124,6 @@ std::unordered_map AbstractSplitRead::CreateDeletionF return deletion_file_map; } -DeletionVector::Factory AbstractSplitRead::CreateDeletionVectorFactory( - const std::unordered_map& deletion_file_map) const { - return [this, deletion_file_map]( - const std::string& file_name) -> Result> { - auto iter = deletion_file_map.find(file_name); - if (iter != deletion_file_map.end()) { - PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr dv, - DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); - return dv; - } - return std::shared_ptr(); - }; -} - Result> AbstractSplitRead::ApplyPredicateFilterIfNeeded( std::unique_ptr&& reader, const std::shared_ptr& predicate) const { if (!context_->EnablePredicateFilter() || predicate == nullptr) { diff --git a/src/paimon/core/operation/abstract_split_read.h b/src/paimon/core/operation/abstract_split_read.h index 94a1c4ec..98bc0efd 100644 --- a/src/paimon/core/operation/abstract_split_read.h +++ b/src/paimon/core/operation/abstract_split_read.h @@ -81,9 +81,6 @@ class AbstractSplitRead : public SplitRead { const std::vector>& data_files, const std::vector>& deletion_files); - DeletionVector::Factory CreateDeletionVectorFactory( - const std::unordered_map& deletion_file_map) const; - Result> ApplyPredicateFilterIfNeeded( std::unique_ptr&& reader, const std::shared_ptr& predicate) const; 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 6fc66305..9725e64e 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -16,7 +16,6 @@ #include "paimon/core/operation/append_only_file_store_write.h" -#include #include #include "paimon/common/data/binary_row.h" @@ -110,7 +109,7 @@ Result> AppendOnlyFileStoreWrite::CreateFileStore Result>> AppendOnlyFileStoreWrite::CompactRewrite( const BinaryRow& partition, int32_t bucket, DeletionVector::Factory dv_factory, const std::vector>& to_compact, - const std::shared_ptr& cancel_flag) { + const std::shared_ptr& cancellation_controller) { if (to_compact.empty()) { return std::vector>{}; } @@ -135,7 +134,7 @@ Result>> AppendOnlyFileStoreWrite::Com }); while (true) { - if (cancel_flag->load(std::memory_order_relaxed)) { + if (cancellation_controller->IsCancelled()) { return Status::Cancelled("Compaction cancelled while rewriting files."); } PAIMON_ASSIGN_OR_RAISE(BatchReader::ReadBatch batch, reader->NextBatch()); @@ -188,12 +187,13 @@ Result> AppendOnlyFileStoreWrite::CreateWriter( } return std::shared_ptr(); }; - auto cancel_flag = std::make_shared(false); + auto cancellation_controller = std::make_shared(); - auto rewriter = [this, partition, bucket, dv_factory, - cancel_flag](const std::vector>& to_compact) + auto rewriter = [this, partition, bucket, dv_factory, cancellation_controller]( + const std::vector>& to_compact) -> Result>> { - return CompactRewrite(partition, bucket, dv_factory, to_compact, cancel_flag); + return CompactRewrite(partition, bucket, dv_factory, to_compact, + cancellation_controller); }; compact_manager = std::make_shared( @@ -202,7 +202,7 @@ Result> AppendOnlyFileStoreWrite::CreateWriter( options_.GetTargetFileSize(/*has_primary_key=*/false), options_.GetCompactionFileSize(/*has_primary_key=*/false), options_.CompactionForceRewriteAllFiles(), rewriter, - compaction_metrics_->CreateReporter(partition, bucket), cancel_flag); + compaction_metrics_->CreateReporter(partition, bucket), cancellation_controller); } auto writer = std::make_shared( diff --git a/src/paimon/core/operation/append_only_file_store_write.h b/src/paimon/core/operation/append_only_file_store_write.h index e112db62..cdae3332 100644 --- a/src/paimon/core/operation/append_only_file_store_write.h +++ b/src/paimon/core/operation/append_only_file_store_write.h @@ -16,7 +16,6 @@ #pragma once -#include #include #include #include @@ -27,6 +26,7 @@ #include "arrow/type.h" #include "paimon/common/data/binary_row.h" +#include "paimon/core/compact/cancellation_controller.h" #include "paimon/core/core_options.h" #include "paimon/core/deletionvectors/deletion_vector.h" #include "paimon/core/io/single_file_writer.h" @@ -94,7 +94,7 @@ class AppendOnlyFileStoreWrite : public AbstractFileStoreWrite { Result>> CompactRewrite( const BinaryRow& partition, int32_t bucket, DeletionVector::Factory dv_factory, const std::vector>& to_compact, - const std::shared_ptr& cancel_flag); + const std::shared_ptr& cancellation_controller); SingleFileWriterCreator GetDataFileWriterCreator( const BinaryRow& partition, int32_t bucket, const std::shared_ptr& schema, diff --git a/src/paimon/core/operation/file_store_write.cpp b/src/paimon/core/operation/file_store_write.cpp index 9d7a4d4d..7decc55c 100644 --- a/src/paimon/core/operation/file_store_write.cpp +++ b/src/paimon/core/operation/file_store_write.cpp @@ -166,10 +166,16 @@ Result> FileStoreWrite::Create(std::unique_ptrTrimmedPrimaryKeys()); PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_key_fields, schema->GetFields(trimmed_primary_keys)); + // use_view=true: for in-memory Arrow row comparisons in MergeTreeWriter PAIMON_ASSIGN_OR_RAISE(std::shared_ptr key_comparator, FieldsComparator::Create(trimmed_primary_key_fields, options.SequenceFieldSortOrderIsAscending(), /*use_view=*/true)); + // use_view=false: for BinaryRow min_key/max_key in DataFileMeta (CompactManager/Levels) + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr key_comparator_for_compact, + FieldsComparator::Create(trimmed_primary_key_fields, + options.SequenceFieldSortOrderIsAscending(), + /*use_view=*/false)); auto primary_keys = schema->PrimaryKeys(); PAIMON_ASSIGN_OR_RAISE( std::unique_ptr merge_function, @@ -183,10 +189,26 @@ Result> FileStoreWrite::Create(std::unique_ptr sequence_fields_comparator, PrimaryKeyTableUtils::CreateSequenceFieldsComparator(schema->Fields(), options)); + + std::shared_ptr dv_maintainer_factory; + if (options.DeletionVectorsEnabled()) { + PAIMON_ASSIGN_OR_RAISE( + std::unique_ptr index_manifest_file, + IndexManifestFile::Create(options.GetFileSystem(), options.GetManifestFormat(), + options.GetManifestCompression(), file_store_path_factory, + options.GetBucket(), ctx->GetMemoryPool(), options)); + auto index_file_handler = std::make_shared( + options.GetFileSystem(), std::move(index_manifest_file), + std::make_shared(file_store_path_factory), + options.DeletionVectorsBitmap64(), ctx->GetMemoryPool()); + dv_maintainer_factory = + std::make_shared(index_file_handler); + } + return std::make_unique( file_store_path_factory, snapshot_manager, schema_manager, ctx->GetCommitUser(), - ctx->GetRootPath(), schema, arrow_schema, partition_schema, - /*dv_maintainer_factory=*/nullptr, key_comparator, sequence_fields_comparator, + ctx->GetRootPath(), schema, arrow_schema, partition_schema, dv_maintainer_factory, + key_comparator, key_comparator_for_compact, sequence_fields_comparator, merge_function_wrapper, options, ignore_previous_files, ctx->IsStreamingMode(), ctx->IgnoreNumBucketCheck(), ctx->GetExecutor(), ctx->GetMemoryPool()); } diff --git a/src/paimon/core/operation/key_value_file_store_write.cpp b/src/paimon/core/operation/key_value_file_store_write.cpp index 6f16590b..b4c45734 100644 --- a/src/paimon/core/operation/key_value_file_store_write.cpp +++ b/src/paimon/core/operation/key_value_file_store_write.cpp @@ -16,14 +16,23 @@ #include "paimon/core/operation/key_value_file_store_write.h" +#include +#include #include #include #include "paimon/common/data/binary_row.h" +#include "paimon/common/utils/string_utils.h" +#include "paimon/core/compact/noop_compact_manager.h" #include "paimon/core/core_options.h" #include "paimon/core/io/data_file_meta.h" #include "paimon/core/manifest/manifest_file.h" #include "paimon/core/manifest/manifest_list.h" +#include "paimon/core/mergetree/compact/force_up_level0_compaction.h" +#include "paimon/core/mergetree/compact/merge_tree_compact_manager.h" +#include "paimon/core/mergetree/compact/merge_tree_compact_rewriter.h" +#include "paimon/core/mergetree/compact/universal_compaction.h" +#include "paimon/core/mergetree/levels.h" #include "paimon/core/mergetree/merge_tree_writer.h" #include "paimon/core/operation/file_store_scan.h" #include "paimon/core/operation/key_value_file_store_scan.h" @@ -55,6 +64,7 @@ KeyValueFileStoreWrite::KeyValueFileStoreWrite( const std::shared_ptr& partition_schema, const std::shared_ptr& dv_maintainer_factory, const std::shared_ptr& key_comparator, + const std::shared_ptr& key_comparator_for_compact, const std::shared_ptr& user_defined_seq_comparator, const std::shared_ptr>& merge_function_wrapper, const CoreOptions& options, bool ignore_previous_files, bool is_streaming_mode, @@ -66,6 +76,7 @@ KeyValueFileStoreWrite::KeyValueFileStoreWrite( ignore_previous_files, is_streaming_mode, ignore_num_bucket_check, executor, pool), key_comparator_(key_comparator), + key_comparator_for_compact_(key_comparator_for_compact), user_defined_seq_comparator_(user_defined_seq_comparator), merge_function_wrapper_(merge_function_wrapper), logger_(Logger::GetLogger("KeyValueFileStoreWrite")) {} @@ -99,11 +110,100 @@ Result> KeyValueFileStoreWrite::CreateWriter( file_store_path_factory_->CreateDataFilePathFactory(partition, bucket)); PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, table_schema_->TrimmedPrimaryKeys()); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr levels, + Levels::Create(key_comparator_for_compact_, restore_data_files, options_.GetNumLevels())); + auto compact_strategy = CreateCompactStrategy(options_); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr compact_manager, + CreateCompactManager(partition, bucket, compact_strategy, + compact_executor_, levels, dv_maintainer)); + auto writer = std::make_shared( restore_max_seq_number, trimmed_primary_keys, data_file_path_factory, key_comparator_, user_defined_seq_comparator_, merge_function_wrapper_, table_schema_->Id(), schema_, - options_, pool_); + options_, compact_manager, pool_); return std::shared_ptr(writer); } +std::shared_ptr KeyValueFileStoreWrite::CreateCompactStrategy( + const CoreOptions& options) const { + auto universal = std::make_shared( + options.GetCompactionMaxSizeAmplificationPercent(), options.GetCompactionSizeRatio(), + options.GetNumSortedRunsCompactionTrigger(), EarlyFullCompaction::Create(options), + OffPeakHours::Create(options)); + + if (options.NeedLookup()) { + std::optional compact_max_interval = std::nullopt; + switch (options.GetLookupCompactMode()) { + case LookupCompactMode::GENTLE: + compact_max_interval = options.GetLookupCompactMaxInterval(); + break; + case LookupCompactMode::RADICAL: + break; + } + return std::make_shared(universal, compact_max_interval); + } + + if (options.CompactionForceUpLevel0()) { + return std::make_shared(universal, + /*max_compact_interval=*/std::nullopt); + } + return universal; +} + +Result> KeyValueFileStoreWrite::CreateCompactManager( + const BinaryRow& partition, int32_t bucket, + const std::shared_ptr& compact_strategy, + const std::shared_ptr& compact_executor, const std::shared_ptr& levels, + const std::shared_ptr& dv_maintainer) { + if (options_.WriteOnly()) { + return std::make_shared(); + } + + auto cancellation_controller = std::make_shared(); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr rewriter, + CreateRewriter(partition, bucket, key_comparator_, user_defined_seq_comparator_, levels, + dv_maintainer, cancellation_controller)); + auto reporter = + compaction_metrics_ ? compaction_metrics_->CreateReporter(partition, bucket) : nullptr; + + return std::make_shared( + compact_executor, levels, compact_strategy, key_comparator_for_compact_, + options_.GetCompactionFileSize(/*has_primary_key=*/true), + options_.GetNumSortedRunsStopTrigger(), rewriter, reporter, dv_maintainer, + /*lazy_gen_deletion_file=*/false, options_.GetLookupStrategy().need_lookup, + options_.CompactionForceRewriteAllFiles(), + /*force_keep_delete=*/false, cancellation_controller); +} + +Result> KeyValueFileStoreWrite::CreateRewriter( + const BinaryRow& partition, int32_t bucket, + const std::shared_ptr& key_comparator, + const std::shared_ptr& user_defined_seq_comparator, + const std::shared_ptr& levels, + const std::shared_ptr& dv_maintainer, + const std::shared_ptr& cancellation_controller) { + (void)key_comparator; + (void)user_defined_seq_comparator; + (void)levels; + LookupStrategy lookup_strategy = options_.GetLookupStrategy(); + ChangelogProducer changelog_producer = options_.GetChangelogProducer(); + auto path_factory_cache = + std::make_shared(root_path_, table_schema_, options_, pool_); + auto dv_factory = DeletionVector::CreateFactory(dv_maintainer); + + if (changelog_producer == ChangelogProducer::FULL_COMPACTION) { + return Status::NotImplemented("not support full changelog merge tree compact rewriter"); + } else if (lookup_strategy.need_lookup) { + return Status::NotImplemented("not support lookup merge tree compact rewriter"); + } else { + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr rewriter, + MergeTreeCompactRewriter::Create( + bucket, partition, table_schema_, dv_factory, path_factory_cache, + options_, pool_, cancellation_controller)); + return std::shared_ptr(std::move(rewriter)); + } +} + } // namespace paimon diff --git a/src/paimon/core/operation/key_value_file_store_write.h b/src/paimon/core/operation/key_value_file_store_write.h index d18c57d7..88158fdf 100644 --- a/src/paimon/core/operation/key_value_file_store_write.h +++ b/src/paimon/core/operation/key_value_file_store_write.h @@ -21,6 +21,7 @@ #include #include +#include "paimon/core/compact/cancellation_controller.h" #include "paimon/core/mergetree/compact/merge_function_wrapper.h" #include "paimon/core/operation/abstract_file_store_write.h" #include "paimon/core/utils/batch_writer.h" @@ -37,9 +38,13 @@ class FieldsComparator; class FileStoreScan; class ScanFilter; class BinaryRow; +class CompactStrategy; +class CompactManager; +class CompactRewriter; class CoreOptions; class Executor; class FileStorePathFactory; +class Levels; class MemoryPool; class SchemaManager; class SnapshotManager; @@ -59,6 +64,7 @@ class KeyValueFileStoreWrite : public AbstractFileStoreWrite { const std::shared_ptr& partition_schema, const std::shared_ptr& dv_maintainer_factory, const std::shared_ptr& key_comparator, + const std::shared_ptr& key_comparator_for_compact, const std::shared_ptr& user_defined_seq_comparator, const std::shared_ptr>& merge_function_wrapper, const CoreOptions& options, bool ignore_previous_files, bool is_streaming_mode, @@ -75,8 +81,27 @@ class KeyValueFileStoreWrite : public AbstractFileStoreWrite { Result> CreateFileStoreScan( const std::shared_ptr& filter) const override; + std::shared_ptr CreateCompactStrategy(const CoreOptions& options) const; + + Result> CreateCompactManager( + const BinaryRow& partition, int32_t bucket, + const std::shared_ptr& compact_strategy, + const std::shared_ptr& compact_executor, const std::shared_ptr& levels, + const std::shared_ptr& dv_maintainer); + + Result> CreateRewriter( + const BinaryRow& partition, int32_t bucket, + const std::shared_ptr& key_comparator, + const std::shared_ptr& user_defined_seq_comparator, + const std::shared_ptr& levels, + const std::shared_ptr& dv_maintainer, + const std::shared_ptr& cancellation_controller); + private: std::shared_ptr key_comparator_; + // key_comparator_for_compact_ uses use_view=false, safe to compare BinaryRow + // min_key/max_key stored in DataFileMeta (not backed by a live Arrow buffer). + std::shared_ptr key_comparator_for_compact_; std::shared_ptr user_defined_seq_comparator_; std::shared_ptr> merge_function_wrapper_; std::unique_ptr logger_; diff --git a/src/paimon/core/operation/merge_file_split_read.cpp b/src/paimon/core/operation/merge_file_split_read.cpp index a1732d60..080eebf2 100644 --- a/src/paimon/core/operation/merge_file_split_read.cpp +++ b/src/paimon/core/operation/merge_file_split_read.cpp @@ -222,7 +222,8 @@ Result> MergeFileSplitRead::ApplyIndexAndDvRead Result> MergeFileSplitRead::CreateMergeReader( const std::shared_ptr& data_split, const std::shared_ptr& data_file_path_factory) { - auto dv_factory = CreateDeletionVectorFactory(CreateDeletionFileMap(*data_split)); + auto dv_factory = DeletionVector::CreateFactory(options_.GetFileSystem(), + CreateDeletionFileMap(*data_split), pool_); std::vector> sections = IntervalPartition(data_split->DataFiles(), interval_partition_comparator_).Partition(); @@ -243,7 +244,8 @@ Result> MergeFileSplitRead::CreateMergeReader( Result> MergeFileSplitRead::CreateNoMergeReader( const std::shared_ptr& data_split, bool only_filter_key, const std::shared_ptr& data_file_path_factory) const { - auto dv_factory = CreateDeletionVectorFactory(CreateDeletionFileMap(*data_split)); + auto dv_factory = DeletionVector::CreateFactory(options_.GetFileSystem(), + CreateDeletionFileMap(*data_split), pool_); // create read schema without extra fields (e.g., completed key, sequence fields) auto row_kind_field = DataField::ConvertDataFieldToArrowField(SpecialFields::ValueKind()); diff --git a/src/paimon/core/operation/raw_file_split_read.cpp b/src/paimon/core/operation/raw_file_split_read.cpp index 41928d3b..802c0904 100644 --- a/src/paimon/core/operation/raw_file_split_read.cpp +++ b/src/paimon/core/operation/raw_file_split_read.cpp @@ -95,8 +95,8 @@ Result> RawFileSplitRead::CreateReader( const BinaryRow& partition, int32_t bucket, const std::vector>& data_files, const std::vector>& deletion_files) { - auto dv_factory = - CreateDeletionVectorFactory(CreateDeletionFileMap(data_files, deletion_files)); + auto dv_factory = DeletionVector::CreateFactory( + options_.GetFileSystem(), CreateDeletionFileMap(data_files, deletion_files), pool_); return CreateReader(partition, bucket, data_files, dv_factory); } diff --git a/src/paimon/core/options/lookup_compact_mode.h b/src/paimon/core/options/lookup_compact_mode.h new file mode 100644 index 00000000..481d8bc3 --- /dev/null +++ b/src/paimon/core/options/lookup_compact_mode.h @@ -0,0 +1,27 @@ +/* + * 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 { +/// The compact mode for lookup compaction. +enum class LookupCompactMode { + /// Lookup compaction will use ForceUpLevel0Compaction strategy to radically compact new files. + RADICAL = 1, + /// Lookup compaction will use UniversalCompaction strategy to gently compact new files. + GENTLE = 2 +}; +} // namespace paimon diff --git a/src/paimon/core/options/lookup_strategy.h b/src/paimon/core/options/lookup_strategy.h index c13bd4a1..210f7731 100644 --- a/src/paimon/core/options/lookup_strategy.h +++ b/src/paimon/core/options/lookup_strategy.h @@ -19,13 +19,7 @@ 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) {} - + public: bool operator==(const LookupStrategy& other) const { if (this == &other) { return true; @@ -34,9 +28,23 @@ struct LookupStrategy { deletion_vector == other.deletion_vector && need_lookup == other.need_lookup; } - bool is_first_row; - bool produce_changelog; - bool deletion_vector; - bool need_lookup; + static LookupStrategy From(bool is_first_row, bool produce_changelog, bool deletion_vector, + bool force_lookup) { + return LookupStrategy(is_first_row, produce_changelog, deletion_vector, force_lookup); + } + + const bool need_lookup; + const bool is_first_row; + const bool produce_changelog; + const bool deletion_vector; + + private: + LookupStrategy(bool is_first_row, bool produce_changelog, bool deletion_vector, + bool force_lookup) + : need_lookup(produce_changelog || deletion_vector || is_first_row || force_lookup), + is_first_row(is_first_row), + produce_changelog(produce_changelog), + deletion_vector(deletion_vector) {} }; + } // namespace paimon diff --git a/src/paimon/core/options/lookup_strategy_test.cpp b/src/paimon/core/options/lookup_strategy_test.cpp new file mode 100644 index 00000000..8bab080b --- /dev/null +++ b/src/paimon/core/options/lookup_strategy_test.cpp @@ -0,0 +1,43 @@ +/* + * 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/options/lookup_strategy.h" + +#include "gtest/gtest.h" + +namespace paimon::test { + +TEST(LookupStrategyTest, TestFrom) { + auto strategy = LookupStrategy::From( + /*is_first_row=*/true, + /*produce_changelog=*/false, + /*deletion_vector=*/false, + /*force_lookup=*/false); + + ASSERT_TRUE(strategy.need_lookup); + ASSERT_TRUE(strategy.is_first_row); + ASSERT_FALSE(strategy.produce_changelog); + ASSERT_FALSE(strategy.deletion_vector); +} + +TEST(LookupStrategyTest, TestNeedLookupCombinations) { + ASSERT_FALSE(LookupStrategy::From(false, false, false, false).need_lookup); + ASSERT_TRUE(LookupStrategy::From(false, true, false, false).need_lookup); + ASSERT_TRUE(LookupStrategy::From(false, false, true, false).need_lookup); + ASSERT_TRUE(LookupStrategy::From(false, false, false, true).need_lookup); +} + +} // namespace paimon::test diff --git a/test/inte/CMakeLists.txt b/test/inte/CMakeLists.txt index 2eda120a..d02b7913 100644 --- a/test/inte/CMakeLists.txt +++ b/test/inte/CMakeLists.txt @@ -83,7 +83,14 @@ if(PAIMON_BUILD_TESTS) test_utils_static ${GTEST_LINK_TOOLCHAIN}) - add_paimon_test(compaction_inte_test + add_paimon_test(append_compaction_inte_test + STATIC_LINK_LIBS + paimon_shared + ${TEST_STATIC_LINK_LIBS} + test_utils_static + ${GTEST_LINK_TOOLCHAIN}) + + add_paimon_test(key_value_compaction_inte_test STATIC_LINK_LIBS paimon_shared ${TEST_STATIC_LINK_LIBS} diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/append_compaction_inte_test.cpp similarity index 97% rename from test/inte/compaction_inte_test.cpp rename to test/inte/append_compaction_inte_test.cpp index 299b8c0e..5532a05f 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/append_compaction_inte_test.cpp @@ -44,7 +44,8 @@ namespace paimon::test { -class CompactionInteTest : public testing::Test, public ::testing::WithParamInterface { +class AppendCompactionInteTest : public testing::Test, + public ::testing::WithParamInterface { public: void SetUp() override { pool_ = GetDefaultPool(); @@ -140,7 +141,7 @@ class CompactionInteTest : public testing::Test, public ::testing::WithParamInte std::shared_ptr pool_; }; -std::vector GetTestValuesForCompactionInteTest() { +std::vector GetTestValuesForAppendCompactionInteTest() { std::vector values; values.emplace_back("parquet"); #ifdef PAIMON_ENABLE_ORC @@ -155,10 +156,10 @@ std::vector GetTestValuesForCompactionInteTest() { return values; } -INSTANTIATE_TEST_SUITE_P(FileFormat, CompactionInteTest, - ::testing::ValuesIn(GetTestValuesForCompactionInteTest())); +INSTANTIATE_TEST_SUITE_P(FileFormat, AppendCompactionInteTest, + ::testing::ValuesIn(GetTestValuesForAppendCompactionInteTest())); -TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompaction) { +TEST_P(AppendCompactionInteTest, TestAppendTableStreamWriteFullCompaction) { auto dir = UniqueTestDirectory::Create(); ASSERT_TRUE(dir); arrow::FieldVector fields = { @@ -243,7 +244,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompaction) { } } -TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompactionWithDv) { +TEST_P(AppendCompactionInteTest, TestAppendTableStreamWriteFullCompactionWithDv) { auto dir = UniqueTestDirectory::Create(); ASSERT_TRUE(dir); arrow::FieldVector fields = { @@ -328,7 +329,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompactionWithDv) { } } -TEST_P(CompactionInteTest, TestAppendTableStreamWriteBestEffortCompaction) { +TEST_P(AppendCompactionInteTest, TestAppendTableStreamWriteBestEffortCompaction) { auto dir = UniqueTestDirectory::Create(); ASSERT_TRUE(dir); arrow::FieldVector fields = { @@ -413,7 +414,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteBestEffortCompaction) { } } -TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) { +TEST_P(AppendCompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) { auto dir = UniqueTestDirectory::Create(); ASSERT_TRUE(dir); auto external_dir = UniqueTestDirectory::Create(); @@ -504,7 +505,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) } } -TEST_F(CompactionInteTest, TestAppendTableCompactionWithIOException) { +TEST_F(AppendCompactionInteTest, TestAppendTableCompactionWithIOException) { arrow::FieldVector fields = { arrow::field("f0", arrow::utf8()), arrow::field("f1", arrow::int32()), arrow::field("f2", arrow::int32()), arrow::field("f3", arrow::float64())}; diff --git a/test/inte/key_value_compaction_inte_test.cpp b/test/inte/key_value_compaction_inte_test.cpp new file mode 100644 index 00000000..ea3cd43e --- /dev/null +++ b/test/inte/key_value_compaction_inte_test.cpp @@ -0,0 +1,286 @@ +/* + * 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 +#include +#include +#include + +#include "arrow/c/bridge.h" +#include "gtest/gtest.h" +#include "paimon/commit_context.h" +#include "paimon/common/data/binary_row.h" +#include "paimon/common/factories/io_hook.h" +#include "paimon/common/utils/scope_guard.h" +#include "paimon/core/append/bucketed_append_compact_manager.h" +#include "paimon/core/io/data_file_meta.h" +#include "paimon/core/operation/append_only_file_store_write.h" +#include "paimon/core/operation/restore_files.h" +#include "paimon/core/table/sink/commit_message_impl.h" +#include "paimon/core/table/source/data_split_impl.h" +#include "paimon/executor.h" +#include "paimon/file_store_commit.h" +#include "paimon/file_store_write.h" +#include "paimon/result.h" +#include "paimon/testing/utils/binary_row_generator.h" +#include "paimon/testing/utils/data_generator.h" +#include "paimon/testing/utils/io_exception_helper.h" +#include "paimon/testing/utils/test_helper.h" +#include "paimon/testing/utils/testharness.h" +#include "paimon/write_context.h" + +namespace paimon::test { + +class KeyValueCompactionInteTest : public testing::Test, + public ::testing::WithParamInterface { + public: + void SetUp() override { + pool_ = GetDefaultPool(); + } + + void PrepareSimpleKeyValueData(const std::shared_ptr& gen, TestHelper* helper, + int64_t* identifier) { + auto& commit_identifier = *identifier; + std::vector datas_1; + datas_1.push_back( + BinaryRowGenerator::GenerateRow({std::string("Alice"), 10, 1, 11.1}, pool_.get())); + datas_1.push_back( + BinaryRowGenerator::GenerateRow({std::string("Bob"), 10, 0, 12.1}, pool_.get())); + datas_1.push_back( + BinaryRowGenerator::GenerateRow({std::string("Emily"), 10, 0, 13.1}, pool_.get())); + datas_1.push_back( + BinaryRowGenerator::GenerateRow({std::string("Tony"), 10, 0, 14.1}, pool_.get())); + datas_1.push_back( + BinaryRowGenerator::GenerateRow({std::string("Lucy"), 20, 1, 14.1}, pool_.get())); + ASSERT_OK_AND_ASSIGN(auto batches_1, gen->SplitArrayByPartitionAndBucket(datas_1)); + ASSERT_EQ(3, batches_1.size()); + ASSERT_OK_AND_ASSIGN( + auto commit_msgs, + helper->WriteAndCommit(std::move(batches_1), commit_identifier++, std::nullopt)); + ASSERT_OK_AND_ASSIGN(std::optional snapshot1, helper->LatestSnapshot()); + ASSERT_TRUE(snapshot1); + ASSERT_EQ(1, snapshot1.value().Id()); + ASSERT_EQ(5, snapshot1.value().TotalRecordCount().value()); + ASSERT_EQ(5, snapshot1.value().DeltaRecordCount().value()); + + std::vector datas_2; + datas_2.push_back( + BinaryRowGenerator::GenerateRow({std::string("Emily"), 10, 0, 15.1}, pool_.get())); + datas_2.push_back( + BinaryRowGenerator::GenerateRow({std::string("Bob"), 10, 0, 12.1}, pool_.get())); + datas_2.push_back( + BinaryRowGenerator::GenerateRow({std::string("Alex"), 10, 0, 16.1}, pool_.get())); + datas_2.push_back( + BinaryRowGenerator::GenerateRow({std::string("Paul"), 20, 1, NullType()}, pool_.get())); + ASSERT_OK_AND_ASSIGN(auto batches_2, gen->SplitArrayByPartitionAndBucket(datas_2)); + ASSERT_EQ(2, batches_2.size()); + ASSERT_OK_AND_ASSIGN( + auto commit_msgs_2, + helper->WriteAndCommit(std::move(batches_2), commit_identifier++, std::nullopt)); + ASSERT_OK_AND_ASSIGN(std::optional snapshot2, helper->LatestSnapshot()); + ASSERT_TRUE(snapshot2); + ASSERT_EQ(2, snapshot2.value().Id()); + ASSERT_EQ(9, snapshot2.value().TotalRecordCount().value()); + ASSERT_EQ(4, snapshot2.value().DeltaRecordCount().value()); + + std::vector datas_3; + datas_3.push_back( + BinaryRowGenerator::GenerateRow({std::string("David"), 10, 0, 17.1}, pool_.get())); + ASSERT_OK_AND_ASSIGN(auto batches_3, gen->SplitArrayByPartitionAndBucket(datas_3)); + ASSERT_EQ(1, batches_3.size()); + ASSERT_OK_AND_ASSIGN( + auto commit_msgs_3, + helper->WriteAndCommit(std::move(batches_3), commit_identifier++, std::nullopt)); + ASSERT_OK_AND_ASSIGN(std::optional snapshot3, helper->LatestSnapshot()); + ASSERT_TRUE(snapshot3); + ASSERT_EQ(3, snapshot3.value().Id()); + ASSERT_EQ(10, snapshot3.value().TotalRecordCount().value()); + ASSERT_EQ(1, snapshot3.value().DeltaRecordCount().value()); + } + + private: + std::shared_ptr pool_; +}; + +std::vector GetTestValuesForCompactionInteTest() { + std::vector values; + values.emplace_back("parquet"); +#ifdef PAIMON_ENABLE_ORC + values.emplace_back("orc"); +#endif +#ifdef PAIMON_ENABLE_LANCE + values.emplace_back("lance"); +#endif +#ifdef PAIMON_ENABLE_AVRO + values.emplace_back("avro"); +#endif + return values; +} + +INSTANTIATE_TEST_SUITE_P(FileFormat, KeyValueCompactionInteTest, + ::testing::ValuesIn(GetTestValuesForCompactionInteTest())); + +TEST_F(KeyValueCompactionInteTest, TestKeyValueTableCompactionWithIOException) { + arrow::FieldVector fields = { + arrow::field("f0", arrow::utf8()), arrow::field("f1", arrow::int32()), + arrow::field("f2", arrow::int32()), arrow::field("f3", arrow::float64())}; + auto schema = arrow::schema(fields); + + std::vector primary_keys = {"f0", "f1", "f2"}; + std::vector partition_keys = {"f1"}; + std::map options = {{Options::FILE_FORMAT, "parquet"}, + {Options::BUCKET, "2"}, + {Options::BUCKET_KEY, "f2"}, + {Options::FILE_SYSTEM, "local"}, + {Options::DELETION_VECTORS_ENABLED, "false"}}; + + bool compaction_run_complete = false; + auto io_hook = IOHook::GetInstance(); + for (size_t i = 0; i < 600; ++i) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + + ASSERT_OK_AND_ASSIGN(auto helper, + TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, + options, /*is_streaming_mode=*/true)); + ASSERT_OK_AND_ASSIGN(std::optional> table_schema, + helper->LatestSchema()); + ASSERT_TRUE(table_schema); + + auto gen = std::make_shared(table_schema.value(), pool_); + int64_t commit_identifier = 0; + PrepareSimpleKeyValueData(gen, helper.get(), &commit_identifier); + + std::vector data; + data.push_back( + BinaryRowGenerator::GenerateRow({std::string("Lily"), 10, 0, 17.1}, pool_.get())); + ASSERT_OK_AND_ASSIGN(auto batches, gen->SplitArrayByPartitionAndBucket(data)); + ASSERT_EQ(1, batches.size()); + + ASSERT_OK_AND_ASSIGN( + auto helper2, + TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, + /*is_streaming_mode=*/true, /*ignore_if_exists=*/true)); + + ScopeGuard guard([&io_hook]() { io_hook->Clear(); }); + io_hook->Reset(i, IOHook::Mode::RETURN_ERROR); + + CHECK_HOOK_STATUS(helper2->write_->Write(std::move(batches[0])), i); + CHECK_HOOK_STATUS(helper2->write_->Compact(/*partition=*/{{"f1", "10"}}, /*bucket=*/1, + /*full_compaction=*/true), + i); + + Result>> commit_messages = + helper2->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier); + CHECK_HOOK_STATUS(commit_messages.status(), i); + CHECK_HOOK_STATUS(helper2->commit_->Commit(commit_messages.value(), commit_identifier), i); + + compaction_run_complete = true; + io_hook->Clear(); + + ASSERT_OK_AND_ASSIGN(std::optional latest_snapshot, helper2->LatestSnapshot()); + ASSERT_TRUE(latest_snapshot); + ASSERT_EQ(Snapshot::CommitKind::Compact(), latest_snapshot->GetCommitKind()); + break; + } + + ASSERT_TRUE(compaction_run_complete); +} + +TEST_P(KeyValueCompactionInteTest, TestKeyValueTableStreamWriteFullCompaction) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + arrow::FieldVector fields = { + arrow::field("f0", arrow::utf8()), arrow::field("f1", arrow::int32()), + arrow::field("f2", arrow::int32()), arrow::field("f3", arrow::float64())}; + auto schema = arrow::schema(fields); + + std::vector primary_keys = {"f0", "f1", "f2"}; + std::vector partition_keys = {"f1"}; + auto file_format = GetParam(); + std::map options = {{Options::FILE_FORMAT, "parquet"}, + {Options::BUCKET, "2"}, + {Options::BUCKET_KEY, "f2"}, + {Options::FILE_SYSTEM, "local"}, + {Options::DELETION_VECTORS_ENABLED, "false"}}; + + ASSERT_OK_AND_ASSIGN( + auto helper, TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, + /*is_streaming_mode=*/true)); + ASSERT_OK_AND_ASSIGN(std::optional> table_schema, + helper->LatestSchema()); + ASSERT_TRUE(table_schema); + auto gen = std::make_shared(table_schema.value(), pool_); + int64_t commit_identifier = 0; + PrepareSimpleKeyValueData(gen, helper.get(), &commit_identifier); + std::vector datas_4; + datas_4.push_back( + BinaryRowGenerator::GenerateRow({std::string("Lily"), 10, 0, 17.1}, pool_.get())); + ASSERT_OK_AND_ASSIGN(auto batches_4, gen->SplitArrayByPartitionAndBucket(datas_4)); + ASSERT_EQ(1, batches_4.size()); + + ASSERT_OK(helper->write_->Write(std::move(batches_4[0]))); + ASSERT_OK(helper->write_->Compact(/*partition=*/{{"f1", "10"}}, /*bucket=*/1, + /*full_compaction=*/true)); + ASSERT_OK_AND_ASSIGN( + std::vector> commit_messages, + helper->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier)); + ASSERT_OK(helper->commit_->Commit(commit_messages, commit_identifier)); + ASSERT_OK_AND_ASSIGN(std::optional snapshot5, helper->LatestSnapshot()); + ASSERT_EQ(5, snapshot5.value().Id()); + ASSERT_EQ(9, snapshot5.value().TotalRecordCount().value()); + ASSERT_EQ(-2, snapshot5.value().DeltaRecordCount().value()); + ASSERT_EQ(Snapshot::CommitKind::Compact(), snapshot5.value().GetCommitKind()); + ASSERT_OK_AND_ASSIGN(std::vector> data_splits, + helper->NewScan(StartupMode::LatestFull(), /*snapshot_id=*/std::nullopt)); + ASSERT_EQ(data_splits.size(), 3); + std::map, std::string> expected_datas; + expected_datas[std::make_pair("f1=10/", 0)] = R"([ +[0, "Alice", 10, 1, 11.1] +])"; + + expected_datas[std::make_pair("f1=10/", 1)] = R"([ +[0, "Alex", 10, 0, 16.1], +[0, "Bob", 10, 0, 12.1], +[0, "David", 10, 0, 17.1], +[0, "Emily", 10, 0, 15.1], +[0, "Lily", 10, 0, 17.1], +[0, "Tony", 10, 0, 14.1] +])"; + + expected_datas[std::make_pair("f1=20/", 0)] = R"([ +[0, "Lucy", 20, 1, 14.1], +[0, "Paul", 20, 1, null] +])"; + + arrow::FieldVector fields_with_row_kind = fields; + fields_with_row_kind.insert(fields_with_row_kind.begin(), + arrow::field("_VALUE_KIND", arrow::int8())); + auto data_type = arrow::struct_(fields_with_row_kind); + + for (const auto& split : data_splits) { + auto split_impl = dynamic_cast(split.get()); + ASSERT_OK_AND_ASSIGN(std::string partition_str, + helper->PartitionStr(split_impl->Partition())); + auto iter = expected_datas.find(std::make_pair(partition_str, split_impl->Bucket())); + ASSERT_TRUE(iter != expected_datas.end()); + ASSERT_OK_AND_ASSIGN(bool success, + helper->ReadAndCheckResult(data_type, {split}, iter->second)); + ASSERT_TRUE(success); + } +} + +} // namespace paimon::test diff --git a/test/inte/write_inte_test.cpp b/test/inte/write_inte_test.cpp index 0db0b2a4..5a0c25bf 100644 --- a/test/inte/write_inte_test.cpp +++ b/test/inte/write_inte_test.cpp @@ -1700,8 +1700,7 @@ TEST_P(WriteInteTest, TestPkTableForceLookup) { {Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, file_format}, {Options::TARGET_FILE_SIZE, "1024"}, {Options::BUCKET, "1"}, {Options::BUCKET_KEY, "f0"}, {Options::FILE_SYSTEM, "local"}, - {Options::FORCE_LOOKUP, "true"}, - }; + {Options::FORCE_LOOKUP, "true"}, {Options::WRITE_ONLY, "true"}}; ASSERT_OK_AND_ASSIGN( auto helper, TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, /*is_streaming_mode=*/true)); @@ -1759,15 +1758,14 @@ TEST_P(WriteInteTest, TestPkTableEnableDeletionVector) { std::vector primary_keys = {"f0", "f1"}; std::vector partition_keys = {}; auto file_format = GetParam(); - std::map options = { - {Options::MANIFEST_FORMAT, "orc"}, - {Options::FILE_FORMAT, file_format}, - {Options::TARGET_FILE_SIZE, "1024"}, - {Options::BUCKET, "1"}, - {Options::BUCKET_KEY, "f0"}, - {Options::FILE_SYSTEM, "local"}, - {Options::DELETION_VECTORS_ENABLED, "true"}, - }; + std::map options = {{Options::MANIFEST_FORMAT, "orc"}, + {Options::FILE_FORMAT, file_format}, + {Options::TARGET_FILE_SIZE, "1024"}, + {Options::BUCKET, "1"}, + {Options::BUCKET_KEY, "f0"}, + {Options::FILE_SYSTEM, "local"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, + {Options::WRITE_ONLY, "true"}}; ASSERT_OK_AND_ASSIGN( auto helper, TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, /*is_streaming_mode=*/true));