From dd348c218687a1cb871134b9e5deb95800cf064f Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 6 Mar 2026 14:49:52 +0800 Subject: [PATCH 01/21] feat(compaction): support compaction for append table --- include/paimon/executor.h | 3 + include/paimon/file_store_write.h | 4 + src/paimon/CMakeLists.txt | 3 + src/paimon/common/executor/executor.cpp | 28 ++- src/paimon/common/reader/reader_utils.cpp | 24 +++ src/paimon/common/reader/reader_utils.h | 5 + src/paimon/core/append/append_only_writer.cpp | 81 ++++++- src/paimon/core/append/append_only_writer.h | 22 +- .../core/append/append_only_writer_test.cpp | 15 +- .../bucketed_append_compact_manager.cpp | 204 ++++++++++++++++++ .../append/bucketed_append_compact_manager.h | 151 ++++++++++++- .../core/compact/compact_deletion_file.h | 94 ++++++++ .../core/compact/compact_future_manager.h | 76 +++++++ src/paimon/core/compact/compact_manager.h | 61 ++++++ src/paimon/core/compact/compact_result.h | 1 + src/paimon/core/compact/compact_task.h | 37 ++++ .../core/compact/noop_compact_manager.h | 81 +++++++ .../deletionvectors/bucketed_dv_maintainer.h | 74 +++++++ .../deletionvectors/deletion_file_writer.cpp | 12 +- .../core/mergetree/merge_tree_writer.cpp | 2 +- src/paimon/core/mergetree/merge_tree_writer.h | 12 ++ .../operation/abstract_file_store_write.cpp | 55 +++-- .../operation/abstract_file_store_write.h | 12 +- .../core/operation/abstract_split_read.cpp | 12 +- .../core/operation/abstract_split_read.h | 4 + .../append_only_file_store_write.cpp | 151 ++++++++++++- .../operation/append_only_file_store_write.h | 20 +- .../append_only_file_store_write_test.cpp | 28 ++- .../core/operation/file_store_commit_impl.cpp | 10 + .../operation/file_system_write_restore.h | 78 +++++++ .../operation/key_value_file_store_write.cpp | 14 +- .../core/operation/raw_file_split_read.cpp | 22 +- .../core/operation/raw_file_split_read.h | 4 + src/paimon/core/operation/restore_files.h | 74 +++++++ src/paimon/core/operation/write_restore.cpp | 42 ++++ src/paimon/core/operation/write_restore.h | 46 ++++ .../core/postpone/postpone_bucket_writer.cpp | 2 +- .../core/postpone/postpone_bucket_writer.h | 12 ++ src/paimon/core/utils/batch_writer.h | 16 ++ src/paimon/core/utils/commit_increment.h | 15 +- test/inte/CMakeLists.txt | 8 + test/inte/compaction_inte_test.cpp | 135 ++++++++++++ 42 files changed, 1660 insertions(+), 90 deletions(-) create mode 100644 src/paimon/core/append/bucketed_append_compact_manager.cpp create mode 100644 src/paimon/core/compact/compact_deletion_file.h create mode 100644 src/paimon/core/compact/compact_future_manager.h create mode 100644 src/paimon/core/compact/compact_manager.h create mode 100644 src/paimon/core/compact/compact_task.h create mode 100644 src/paimon/core/compact/noop_compact_manager.h create mode 100644 src/paimon/core/deletionvectors/bucketed_dv_maintainer.h create mode 100644 src/paimon/core/operation/file_system_write_restore.h create mode 100644 src/paimon/core/operation/restore_files.h create mode 100644 src/paimon/core/operation/write_restore.cpp create mode 100644 src/paimon/core/operation/write_restore.h create mode 100644 test/inte/compaction_inte_test.cpp diff --git a/include/paimon/executor.h b/include/paimon/executor.h index 1c9fce8d..0ba96e31 100644 --- a/include/paimon/executor.h +++ b/include/paimon/executor.h @@ -51,6 +51,9 @@ class PAIMON_EXPORT Executor { /// @note This method should be thread-safe and can be called from multiple threads /// simultaneously. virtual void Add(std::function func) = 0; + + /// Shutdown the executor immediately, discarding all pending tasks. + virtual void ShutdownNow() = 0; }; } // namespace paimon diff --git a/include/paimon/file_store_write.h b/include/paimon/file_store_write.h index b791777d..6721a660 100644 --- a/include/paimon/file_store_write.h +++ b/include/paimon/file_store_write.h @@ -17,6 +17,7 @@ #pragma once #include +#include #include #include @@ -51,6 +52,9 @@ class PAIMON_EXPORT FileStoreWrite { /// the corresponding array in `batch` must have zero null entries. virtual Status Write(std::unique_ptr&& batch) = 0; + virtual Status Compact(const std::map& partition, int32_t bucket, + bool full_compaction) = 0; + /// Generate a list of commit messages with the latest generated data file meta /// information of the current snapshot. /// diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index b734d05a..4ed45a0d 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -145,6 +145,7 @@ set(PAIMON_COMMON_SRCS set(PAIMON_CORE_SRCS core/append/append_only_writer.cpp + core/append/bucketed_append_compact_manager.cpp core/casting/binary_to_string_cast_executor.cpp core/casting/boolean_to_decimal_cast_executor.cpp core/casting/boolean_to_numeric_cast_executor.cpp @@ -256,6 +257,7 @@ set(PAIMON_CORE_SRCS core/operation/read_context.cpp core/operation/scan_context.cpp core/operation/write_context.cpp + core/operation/write_restore.cpp core/postpone/postpone_bucket_writer.cpp core/schema/arrow_schema_validator.cpp core/schema/schema_manager.cpp @@ -488,6 +490,7 @@ if(PAIMON_BUILD_TESTS) add_paimon_test(core_test SOURCES core/append/append_only_writer_test.cpp + core/append/bucketed_append_compact_manager_test.cpp core/casting/cast_executor_factory_test.cpp core/casting/cast_executor_test.cpp core/casting/casted_row_test.cpp diff --git a/src/paimon/common/executor/executor.cpp b/src/paimon/common/executor/executor.cpp index e758cb81..62e67f1c 100644 --- a/src/paimon/common/executor/executor.cpp +++ b/src/paimon/common/executor/executor.cpp @@ -33,9 +33,13 @@ class DefaultExecutor : public Executor { void Add(std::function func) override; + void ShutdownNow() override; + private: void WorkerThread(); + void ShutdownInternal(bool drain_queue); + uint32_t thread_count_; std::vector workers_; std::queue> tasks_; @@ -51,17 +55,37 @@ DefaultExecutor::DefaultExecutor(uint32_t thread_count) : thread_count_(thread_c } } -DefaultExecutor::~DefaultExecutor() { +void DefaultExecutor::ShutdownInternal(bool drain_queue) { { std::unique_lock lock(queue_mutex_); + if (stop_) { + return; + } stop_ = true; + if (!drain_queue) { + // Discard all pending tasks immediately. + std::queue> empty; + tasks_.swap(empty); + } condition_.notify_all(); } for (std::thread& worker : workers_) { - worker.join(); + if (worker.joinable()) { + worker.join(); + } } } +DefaultExecutor::~DefaultExecutor() { + // Graceful shutdown: wait for all pending tasks to complete. + ShutdownInternal(/*drain_queue=*/true); +} + +void DefaultExecutor::ShutdownNow() { + // Immediate shutdown: discard all pending tasks. + ShutdownInternal(/*drain_queue=*/false); +} + void DefaultExecutor::Add(std::function func) { if (!func) { return; diff --git a/src/paimon/common/reader/reader_utils.cpp b/src/paimon/common/reader/reader_utils.cpp index 2e879984..13fc5595 100644 --- a/src/paimon/common/reader/reader_utils.cpp +++ b/src/paimon/common/reader/reader_utils.cpp @@ -23,6 +23,7 @@ #include "arrow/api.h" #include "arrow/array/array_base.h" +#include "arrow/array/array_nested.h" #include "arrow/c/abi.h" #include "arrow/c/bridge.h" #include "arrow/c/helpers.h" @@ -105,4 +106,27 @@ BatchReader::ReadBatchWithBitmap ReaderUtils::AddAllValidBitmap(BatchReader::Rea all_valid.AddRange(0, batch.first->length); return std::make_pair(std::move(batch), std::move(all_valid)); } + +Result> ReaderUtils::RemoveFieldFromStructArray( + const std::shared_ptr& struct_array, const std::string& field_name) { + auto struct_type = std::static_pointer_cast(struct_array->type()); + int32_t field_idx = struct_type->GetFieldIndex(field_name); + if (field_idx == -1) { + return struct_array; + } + std::vector> new_arrays; + std::vector> new_fields; + for (int32_t i = 0; i < struct_type->num_fields(); ++i) { + if (i != field_idx) { + new_arrays.emplace_back(struct_array->field(i)); + new_fields.emplace_back(struct_type->field(i)); + } + } + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW( + std::shared_ptr array, + arrow::StructArray::Make(new_arrays, new_fields, struct_array->null_bitmap(), + struct_array->null_count())); + return array; +} + } // namespace paimon diff --git a/src/paimon/common/reader/reader_utils.h b/src/paimon/common/reader/reader_utils.h index 4ed3e100..b8cdb48c 100644 --- a/src/paimon/common/reader/reader_utils.h +++ b/src/paimon/common/reader/reader_utils.h @@ -17,6 +17,7 @@ #pragma once #include +#include #include "arrow/api.h" #include "paimon/reader/batch_reader.h" @@ -25,6 +26,7 @@ namespace arrow { class MemoryPool; class Array; +class StructArray; } // namespace arrow namespace paimon { @@ -52,5 +54,8 @@ class ReaderUtils { /// Precondition: input bitmap is not empty static Result GenerateFilteredArrayVector( const std::shared_ptr& src_array, const RoaringBitmap32& bitmap); + + static Result> RemoveFieldFromStructArray( + const std::shared_ptr& struct_array, const std::string& field_name); }; } // namespace paimon diff --git a/src/paimon/core/append/append_only_writer.cpp b/src/paimon/core/append/append_only_writer.cpp index abae2106..936a37fc 100644 --- a/src/paimon/core/append/append_only_writer.cpp +++ b/src/paimon/core/append/append_only_writer.cpp @@ -55,7 +55,8 @@ AppendOnlyWriter::AppendOnlyWriter(const CoreOptions& options, int64_t schema_id const std::optional>& write_cols, int64_t max_sequence_number, const std::shared_ptr& path_factory, - const std::shared_ptr& memory_pool) + const std::shared_ptr& memory_pool, + const std::shared_ptr& compact_manager) : options_(options), schema_id_(schema_id), write_schema_(write_schema), @@ -63,6 +64,7 @@ AppendOnlyWriter::AppendOnlyWriter(const CoreOptions& options, int64_t schema_id seq_num_counter_(std::make_shared(max_sequence_number + 1)), path_factory_(path_factory), memory_pool_(memory_pool), + compact_manager_(compact_manager), metrics_(std::make_shared()) {} AppendOnlyWriter::~AppendOnlyWriter() = default; @@ -83,24 +85,67 @@ Status AppendOnlyWriter::Write(std::unique_ptr&& batch) { } Result AppendOnlyWriter::PrepareCommit(bool wait_compaction) { - PAIMON_RETURN_NOT_OK(Flush()); + PAIMON_RETURN_NOT_OK( + Flush(/*wait_for_latest_compaction=*/false, /*force_full_compaction=*/false)); + PAIMON_RETURN_NOT_OK(TrySyncLatestCompaction(wait_compaction || options_.CommitForceCompact())); return DrainIncrement(); } Result AppendOnlyWriter::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_before_.clear(); + compact_after_.clear(); + compact_deletion_file_ = nullptr; + + return CommitIncrement(data_increment, compact_increment, drain_deletion_file); +} + +Status AppendOnlyWriter::TrySyncLatestCompaction(bool blocking) { + PAIMON_ASSIGN_OR_RAISE(std::optional> result, + compact_manager_->GetCompactionResult(blocking)); + if (result.has_value()) { + const auto& compaction_result = result.value(); + const auto& before = compaction_result->Before(); + compact_before_.insert(compact_before_.end(), before.begin(), before.end()); + const auto& after = compaction_result->After(); + compact_after_.insert(compact_after_.end(), after.begin(), after.end()); + PAIMON_RETURN_NOT_OK(UpdateCompactDeletionFile(compaction_result->DeletionFile())); + } + return Status::OK(); +} + +Status AppendOnlyWriter::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(); } -Status AppendOnlyWriter::Flush() { +Status AppendOnlyWriter::Flush(bool wait_for_latest_compaction, bool forced_full_compaction) { + std::vector> flushed_files; if (writer_) { PAIMON_RETURN_NOT_OK(writer_->Close()); - PAIMON_ASSIGN_OR_RAISE(std::vector> flushed_files, - writer_->GetResult()); - new_files_.insert(new_files_.end(), flushed_files.begin(), flushed_files.end()); + PAIMON_ASSIGN_OR_RAISE(flushed_files, writer_->GetResult()); + } + // add new generated files + for (const auto& flushed_file : flushed_files) { + compact_manager_->AddNewFile(flushed_file); + } + PAIMON_RETURN_NOT_OK(TrySyncLatestCompaction(wait_for_latest_compaction)); + PAIMON_RETURN_NOT_OK(compact_manager_->TriggerCompaction(forced_full_compaction)); + new_files_.insert(new_files_.end(), flushed_files.begin(), flushed_files.end()); + if (writer_) { metrics_->Merge(writer_->GetMetrics()); writer_.reset(); } @@ -198,11 +243,31 @@ AppendOnlyWriter::RollingFileWriterResult AppendOnlyWriter::CreateRollingBlobWri rolling_blob_file_writer_creator, arrow::struct_(write_schema_->fields())); } +Status AppendOnlyWriter::Sync() { + return TrySyncLatestCompaction(/*blocking=*/true); +} + Status AppendOnlyWriter::Close() { + // 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()); + auto fs = options_.GetFileSystem(); + for (const auto& file : compact_after_) { + // AppendOnlyCompactManager will rewrite the file and no file upgrade will occur, so we + // can directly delete the file in compact_after_. + [[maybe_unused]] auto s = fs->Delete(path_factory_->ToPath(file)); + } + if (writer_) { writer_->Abort(); writer_.reset(); } + + if (compact_deletion_file_ != nullptr) { + compact_deletion_file_->Clean(); + } return Status::OK(); } diff --git a/src/paimon/core/append/append_only_writer.h b/src/paimon/core/append/append_only_writer.h index fc76b56f..b70bd117 100644 --- a/src/paimon/core/append/append_only_writer.h +++ b/src/paimon/core/append/append_only_writer.h @@ -24,6 +24,7 @@ #include #include "paimon/common/data/blob_utils.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/single_file_writer.h" @@ -58,11 +59,21 @@ class AppendOnlyWriter : public BatchWriter { const std::optional>& write_cols, int64_t max_sequence_number, const std::shared_ptr& path_factory, - const std::shared_ptr& memory_pool); + const std::shared_ptr& memory_pool, + const std::shared_ptr& compact_manager); + ~AppendOnlyWriter() override; Status Write(std::unique_ptr&& batch) override; + Status Compact(bool full_compaction) override { + return Flush(/*wait_for_latest_compaction=*/true, full_compaction); + } Result PrepareCommit(bool wait_compaction) override; + Result CompactNotCompleted() override { + PAIMON_RETURN_NOT_OK(compact_manager_->TriggerCompaction(/*full_compaction=*/false)); + return compact_manager_->CompactNotCompleted(); + } + Status Sync() override; Status Close() override; bool IsCompacting() const override { return false; @@ -82,7 +93,7 @@ class AppendOnlyWriter : public BatchWriter { const BlobUtils::SeparatedSchemas& schemas) const; Result DrainIncrement(); - Status Flush(); + Status Flush(bool wait_for_latest_compaction, bool forced_full_compaction); SingleFileWriterCreator GetDataFileWriterCreator( const std::shared_ptr& schema, @@ -93,6 +104,9 @@ class AppendOnlyWriter : public BatchWriter { const std::shared_ptr& stats_extractor, const std::optional>& write_cols) const; + Status TrySyncLatestCompaction(bool blocking); + Status UpdateCompactDeletionFile(const std::shared_ptr& new_deletion_file); + CoreOptions options_; int64_t schema_id_; std::shared_ptr write_schema_; @@ -100,11 +114,15 @@ class AppendOnlyWriter : public BatchWriter { std::shared_ptr seq_num_counter_; std::shared_ptr path_factory_; std::shared_ptr memory_pool_; + std::shared_ptr compact_manager_; 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_; std::unique_ptr>> writer_; }; diff --git a/src/paimon/core/append/append_only_writer_test.cpp b/src/paimon/core/append/append_only_writer_test.cpp index fc8066a3..562dc5a5 100644 --- a/src/paimon/core/append/append_only_writer_test.cpp +++ b/src/paimon/core/append/append_only_writer_test.cpp @@ -31,6 +31,7 @@ #include "arrow/type.h" #include "gtest/gtest.h" #include "paimon/common/fs/external_path_provider.h" +#include "paimon/core/compact/noop_compact_manager.h" #include "paimon/core/core_options.h" #include "paimon/core/io/compact_increment.h" #include "paimon/core/io/data_file_path_factory.h" @@ -53,10 +54,12 @@ class AppendOnlyWriterTest : public testing::Test { public: void SetUp() override { memory_pool_ = GetDefaultPool(); + compact_manager_ = std::make_shared(); } private: std::shared_ptr memory_pool_; + std::shared_ptr compact_manager_; }; TEST_F(AppendOnlyWriterTest, TestEmptyCommits) { @@ -79,7 +82,8 @@ TEST_F(AppendOnlyWriterTest, TestEmptyCommits) { ASSERT_OK(path_factory->Init(dir->Str(), "mock_format", options.DataFilePrefix(), nullptr)); AppendOnlyWriter writer(options, /*schema_id=*/0, schema, /*write_cols=*/std::nullopt, - /*max_sequence_number=*/-1, path_factory, memory_pool_); + /*max_sequence_number=*/-1, path_factory, memory_pool_, + compact_manager_); ASSERT_FALSE(writer.IsCompacting()); for (int i = 0; i < 3; i++) { ASSERT_OK_AND_ASSIGN(CommitIncrement inc, writer.PrepareCommit(true)); @@ -108,7 +112,8 @@ TEST_F(AppendOnlyWriterTest, TestWriteAndPrepareCommit) { auto path_factory = std::make_shared(); ASSERT_OK(path_factory->Init(dir->Str(), "mock_format", options.DataFilePrefix(), nullptr)); AppendOnlyWriter writer(options, /*schema_id=*/2, schema, /*write_cols=*/std::nullopt, - /*max_sequence_number=*/-1, path_factory, memory_pool_); + /*max_sequence_number=*/-1, path_factory, memory_pool_, + compact_manager_); ASSERT_FALSE(writer.IsCompacting()); arrow::StringBuilder builder; for (size_t j = 0; j < 100; j++) { @@ -150,7 +155,8 @@ TEST_F(AppendOnlyWriterTest, TestWriteAndClose) { auto path_factory = std::make_shared(); ASSERT_OK(path_factory->Init(dir->Str(), "orc", options.DataFilePrefix(), nullptr)); AppendOnlyWriter writer(options, /*schema_id=*/1, schema, /*write_cols=*/std::nullopt, - /*max_sequence_number=*/-1, path_factory, memory_pool_); + /*max_sequence_number=*/-1, path_factory, memory_pool_, + compact_manager_); ASSERT_FALSE(writer.IsCompacting()); auto struct_type = arrow::struct_(fields); @@ -195,7 +201,8 @@ TEST_F(AppendOnlyWriterTest, TestInvalidRowKind) { auto path_factory = std::make_shared(); ASSERT_OK(path_factory->Init(dir->Str(), "orc", options.DataFilePrefix(), nullptr)); AppendOnlyWriter writer(options, /*schema_id=*/1, schema, /*write_cols=*/std::nullopt, - /*max_sequence_number=*/-1, path_factory, memory_pool_); + /*max_sequence_number=*/-1, path_factory, memory_pool_, + compact_manager_); ASSERT_FALSE(writer.IsCompacting()); auto struct_type = arrow::struct_(fields); diff --git a/src/paimon/core/append/bucketed_append_compact_manager.cpp b/src/paimon/core/append/bucketed_append_compact_manager.cpp new file mode 100644 index 00000000..6d25783a --- /dev/null +++ b/src/paimon/core/append/bucketed_append_compact_manager.cpp @@ -0,0 +1,204 @@ +/* + * 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/append/bucketed_append_compact_manager.h" + +#include "paimon/common/executor/future.h" + +namespace paimon { + +BucketedAppendCompactManager::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) + : executor_(executor), + dv_maintainer_(dv_maintainer), + min_file_num_(min_file_num), + target_file_size_(target_file_size), + compaction_file_size_(compaction_file_size), + force_rewrite_all_files_(force_rewrite_all_files), + rewriter_(rewriter), + to_compact_( + [](const std::shared_ptr& lhs, const std::shared_ptr& rhs) { + return lhs->min_sequence_number > rhs->min_sequence_number; + }), + logger_(Logger::GetLogger("BucketedAppendCompactManager")) { + for (const auto& file : restored) { + to_compact_.push(file); + } +} + +Status BucketedAppendCompactManager::TriggerCompaction(bool full_compaction) { + if (full_compaction) { + PAIMON_RETURN_NOT_OK(TriggerFullCompaction()); + } else { + PAIMON_RETURN_NOT_OK(TriggerCompactionWithBestEffort()); + } + return Status::OK(); +} + +Status BucketedAppendCompactManager::TriggerFullCompaction() { + if (task_future_.valid()) { + return Status::Invalid( + "A compaction task is still running while the user forces a new compaction. This " + "is unexpected."); + } + // if all files are force picked or deletion vector enables, always trigger compaction. + if (!force_rewrite_all_files_ && + (to_compact_.empty() || + (dv_maintainer_ == nullptr && to_compact_.size() < FULL_COMPACT_MIN_FILE))) { + return Status::OK(); + } + + std::vector> compacting; + while (!to_compact_.empty()) { + compacting.push_back(to_compact_.top()); + to_compact_.pop(); + } + auto compact_task = std::make_shared( + dv_maintainer_, compacting, compaction_file_size_, force_rewrite_all_files_, rewriter_); + task_future_ = Via(executor_.get(), [compact_task]() -> Result> { + return compact_task->Execute(); + }); + compacting_ = compacting; + return Status::OK(); +} + +Status BucketedAppendCompactManager::TriggerCompactionWithBestEffort() { + if (task_future_.valid()) { + return Status::OK(); + } + std::optional>> picked = PickCompactBefore(); + if (picked) { + compacting_ = picked.value(); + auto compact_task = + std::make_shared(dv_maintainer_, compacting_.value(), rewriter_); + task_future_ = + Via(executor_.get(), [compact_task]() -> Result> { + return compact_task->Execute(); + }); + } + return Status::OK(); +} + +std::optional>> +BucketedAppendCompactManager::PickCompactBefore() { + if (to_compact_.empty()) { + return std::nullopt; + } + int64_t total_file_size = 0; + int32_t file_num = 0; + std::deque> candidates; + + while (!to_compact_.empty()) { + std::shared_ptr file = to_compact_.top(); + to_compact_.pop(); + candidates.push_back(file); + total_file_size += file->file_size; + file_num++; + if (file_num >= min_file_num_) { + return std::vector>(candidates.begin(), candidates.end()); + } else if (total_file_size >= target_file_size_ * 2) { + // let pointer shift one pos to right + std::shared_ptr removed = candidates.front(); + candidates.pop_front(); + total_file_size -= removed->file_size; + file_num--; + } + } + for (const auto& candidate : candidates) { + to_compact_.push(candidate); + } + return std::nullopt; +} + +std::vector> BucketedAppendCompactManager::AllFiles() const { + std::vector> all_files; + if (compacting_ != std::nullopt) { + all_files.insert(all_files.end(), compacting_.value().begin(), compacting_.value().end()); + } + auto to_compact = to_compact_; + while (!to_compact.empty()) { + all_files.push_back(to_compact.top()); + to_compact.pop(); + } + return all_files; +} + +Result>> +BucketedAppendCompactManager::GetCompactionResult(bool blocking) { + PAIMON_ASSIGN_OR_RAISE(std::optional> result, + InnerGetCompactionResult(blocking)); + if (result) { + std::shared_ptr compact_result = result.value(); + if (!compact_result->After().empty()) { + // if the last compacted file is still small, + // add it back to the head + std::shared_ptr last_file = compact_result->After().back(); + if (last_file->file_size < compaction_file_size_) { + to_compact_.push(last_file); + } + } + compacting_ = std::nullopt; + } + return result; +} + +Result> BucketedAppendCompactManager::FullCompactTask::DoCompact() { + // remove large files + while (!force_rewrite_all_files_ && !to_compact_.empty()) { + const auto& file = to_compact_.front(); + // the data file with deletion file always need to be compacted. + if (file->file_size >= compaction_file_size_ && !HasDeletionFile(file)) { + to_compact_.pop_front(); + continue; + } + break; + } + + // do compaction + if (dv_maintainer_ != nullptr) { + // if deletion vector enables, always trigger compaction. + return Compact( + dv_maintainer_, + std::vector>(to_compact_.begin(), to_compact_.end()), + rewriter_); + } else { + // compute small files + int32_t big = 0; + int32_t small = 0; + for (const auto& file : to_compact_) { + if (file->file_size >= compaction_file_size_) { + big++; + } else { + small++; + } + } + if (force_rewrite_all_files_ || + (small > big && to_compact_.size() >= FULL_COMPACT_MIN_FILE)) { + return Compact( + /*dv_maintainer=*/nullptr, + std::vector>(to_compact_.begin(), to_compact_.end()), + rewriter_); + } else { + return std::make_shared(); + } + } +} + +} // namespace paimon diff --git a/src/paimon/core/append/bucketed_append_compact_manager.h b/src/paimon/core/append/bucketed_append_compact_manager.h index a9065c3d..c76c7c9f 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.h +++ b/src/paimon/core/append/bucketed_append_compact_manager.h @@ -16,17 +16,35 @@ #pragma once +#include #include #include +#include +#include +#include +#include "paimon/common/executor/future.h" +#include "paimon/core/compact/compact_future_manager.h" +#include "paimon/core/compact/compact_task.h" +#include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" #include "paimon/core/io/data_file_meta.h" +#include "paimon/executor.h" +#include "paimon/logging.h" +#include "paimon/result.h" namespace paimon { + /// Compact manager for `AppendOnlyFileStore`. -class BucketedAppendCompactManager { +class BucketedAppendCompactManager : public CompactFutureManager { public: - BucketedAppendCompactManager() = delete; - ~BucketedAppendCompactManager() = delete; + using DataFileMetaPriorityQueue = + std::priority_queue, + std::vector>, + std::function&, + const std::shared_ptr&)>>; + + using CompactRewriter = std::function>>( + const std::vector>&)>; /// New files may be created during the compaction process, then the results of the compaction /// may be put after the new files, and this order will be disrupted. We need to ensure this @@ -46,11 +64,138 @@ class BucketedAppendCompactManager { }; } + 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); + ~BucketedAppendCompactManager() override = default; + + Status TriggerCompaction(bool full_compaction) override; + + bool ShouldWaitForLatestCompaction() const override { + return false; + } + bool ShouldWaitForPreparingCheckpoint() const override { + return false; + } + + void AddNewFile(const std::shared_ptr& file) override { + to_compact_.push(file); + } + + std::vector> AllFiles() const override; + + /// Finish current task, and update result files to to_compact_ + Result>> GetCompactionResult( + bool blocking) override; + + Status Close() override { + // TODO(yonghao.fyh): metrics reporter + return Status::OK(); + } + private: + static constexpr int32_t FULL_COMPACT_MIN_FILE = 3; + static bool IsOverlap(const std::shared_ptr& o1, const std::shared_ptr& o2) { return o2->min_sequence_number <= o1->max_sequence_number && o2->max_sequence_number >= o1->min_sequence_number; } + + static Result> Compact( + const std::shared_ptr& dv_maintainer, + const std::vector>& to_compact, CompactRewriter rewriter) { + PAIMON_ASSIGN_OR_RAISE(std::vector> rewrite, + rewriter(to_compact)); + auto result = std::make_shared(to_compact, rewrite); + if (dv_maintainer != nullptr) { + for (const auto& file : to_compact) { + dv_maintainer->RemoveDeletionVectorOf(file->file_name); + } + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr deletion_file, + CompactDeletionFile::GenerateFiles(dv_maintainer)); + result->SetDeletionFile(deletion_file); + } + return result; + } + + /// A `CompactTask` impl for full compaction of append-only table. + class FullCompactTask : public CompactTask { + public: + FullCompactTask(const std::shared_ptr& dv_maintainer, + const std::vector>& inputs, + int64_t compaction_file_size, bool force_rewrite_all_files, + CompactRewriter rewriter) + : dv_maintainer_(dv_maintainer), + to_compact_(inputs.begin(), inputs.end()), + compaction_file_size_(compaction_file_size), + force_rewrite_all_files_(force_rewrite_all_files), + rewriter_(rewriter) {} + + protected: + Result> DoCompact() override; + + private: + bool HasDeletionFile(const std::shared_ptr& file) const { + if (dv_maintainer_) { + return dv_maintainer_->DeletionVectorOf(file->file_name) != std::nullopt; + } + return false; + } + + static constexpr int32_t FULL_COMPACT_MIN_FILE = 3; + + std::shared_ptr dv_maintainer_; + std::deque> to_compact_; + int64_t compaction_file_size_; + bool force_rewrite_all_files_; + CompactRewriter rewriter_; + }; + + /// A `CompactTask` impl for append-only table auto-compaction. + /// + /// This task accepts an already-picked candidate to perform one-time rewrite. And for the + /// rest of input files, it is the duty of `AppendOnlyWriter` to invoke the next time + /// compaction. + class AutoCompactTask : public CompactTask { + public: + AutoCompactTask(const std::shared_ptr& dv_maintainer, + const std::vector>& to_compact, + CompactRewriter rewriter) + : dv_maintainer_(dv_maintainer), to_compact_(to_compact), rewriter_(rewriter) {} + + protected: + Result> DoCompact() override { + return Compact(dv_maintainer_, to_compact_, rewriter_); + } + + private: + std::shared_ptr dv_maintainer_; + std::vector> to_compact_; + CompactRewriter rewriter_; + }; + + DataFileMetaPriorityQueue GetToCompact() const { + return to_compact_; + } + + std::optional>> PickCompactBefore(); + Status TriggerFullCompaction(); + Status TriggerCompactionWithBestEffort(); + + std::shared_ptr executor_; + 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_; + std::optional>> compacting_; + DataFileMetaPriorityQueue to_compact_; + std::unique_ptr logger_; }; + } // namespace paimon diff --git a/src/paimon/core/compact/compact_deletion_file.h b/src/paimon/core/compact/compact_deletion_file.h new file mode 100644 index 00000000..b13a7a59 --- /dev/null +++ b/src/paimon/core/compact/compact_deletion_file.h @@ -0,0 +1,94 @@ +/* + * 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 "paimon/core/deletionvectors/bucketed_dv_maintainer.h" +#include "paimon/core/deletionvectors/deletion_vectors_index_file.h" +#include "paimon/core/index/index_file_meta.h" +#include "paimon/result.h" + +namespace paimon { + +class CompactDeletionFile { + public: + virtual ~CompactDeletionFile() = default; + + static Result> GenerateFiles( + const std::shared_ptr& maintainer); + + virtual std::optional> GetOrCompute() = 0; + + virtual Result> MergeOldFile( + const std::shared_ptr& old) = 0; + + virtual void Clean() = 0; +}; + +class GeneratedDeletionFile : public CompactDeletionFile, + public std::enable_shared_from_this { + public: + GeneratedDeletionFile(const std::shared_ptr& deletion_file, + const std::shared_ptr& dv_index_file) + : deletion_file_(deletion_file), dv_index_file_(dv_index_file) {} + + std::optional> GetOrCompute() override { + get_invoked_ = true; + return deletion_file_ ? std::optional>(deletion_file_) + : std::nullopt; + } + + // TODO(yonghao.fyh): check logic + Result> MergeOldFile( + const std::shared_ptr& old) override { + auto derived = dynamic_cast(old.get()); + if (derived == nullptr) { + return Status::Invalid("old should be a GeneratedDeletionFile, but it is not"); + } + if (derived->get_invoked_) { + return Status::Invalid("old should not be get, this is a bug."); + } + if (deletion_file_ == nullptr) { + return old; + } + old->Clean(); + return shared_from_this(); + } + + void Clean() override { + if (deletion_file_ != nullptr) { + dv_index_file_->Delete(deletion_file_); + } + } + + private: + std::shared_ptr deletion_file_; + std::shared_ptr dv_index_file_; + bool get_invoked_ = false; +}; + +inline Result> CompactDeletionFile::GenerateFiles( + const std::shared_ptr& maintainer) { + PAIMON_ASSIGN_OR_RAISE(std::optional> file, + maintainer->WriteDeletionVectorsIndex()); + return std::make_shared(file.value_or(nullptr), + maintainer->DvIndexFile()); +} + +} // namespace paimon diff --git a/src/paimon/core/compact/compact_future_manager.h b/src/paimon/core/compact/compact_future_manager.h new file mode 100644 index 00000000..092a84ac --- /dev/null +++ b/src/paimon/core/compact/compact_future_manager.h @@ -0,0 +1,76 @@ +/* + * 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 "paimon/core/compact/compact_manager.h" + +namespace paimon { + +class CompactFutureManager : public CompactManager { + public: + ~CompactFutureManager() override { + if (task_future_.valid()) { + task_future_.wait(); + } + for (auto& f : cancelled_futures_) { + if (f.valid()) { + f.wait(); + } + } + } + + /// Cancel the current compaction task if it is running. + /// @note: This method may leave behind orphan files. + void CancelCompaction() override { + // std::future does not support cancellation natively. + if (task_future_.valid()) { + // Detach the future so we don't block on destruction + cancelled_futures_.push_back(std::move(task_future_)); + } + } + + bool CompactNotCompleted() const override { + return task_future_.valid(); + } + + protected: + Result> ObtainCompactResult( + std::future>> task_future) { + return task_future.get(); + } + + Result>> InnerGetCompactionResult(bool blocking) { + if (!task_future_.valid()) { + return std::optional>(); + } + bool ready = blocking || + (task_future_.wait_for(std::chrono::seconds(0)) == std::future_status::ready); + if (ready) { + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr result, + ObtainCompactResult(std::move(task_future_))); + return std::make_optional(std::move(result)); + } + return std::optional>(); + } + + std::future>> task_future_; + std::vector>>> cancelled_futures_; +}; + +} // namespace paimon diff --git a/src/paimon/core/compact/compact_manager.h b/src/paimon/core/compact/compact_manager.h new file mode 100644 index 00000000..ed180db9 --- /dev/null +++ b/src/paimon/core/compact/compact_manager.h @@ -0,0 +1,61 @@ +/* + * 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_result.h" +#include "paimon/core/io/data_file_meta.h" +#include "paimon/result.h" +#include "paimon/status.h" + +namespace paimon { + +class CompactManager { + public: + virtual ~CompactManager() = default; + /// Add a new file. + virtual void AddNewFile(const std::shared_ptr& file) = 0; + + virtual std::vector> AllFiles() const = 0; + + /// Trigger a new compaction task. + /// + /// @param full_compaction if caller needs a guaranteed full compaction + virtual Status TriggerCompaction(bool full_compaction) = 0; + + /// Get compaction result. Wait finish if `blocking` is true. + virtual Result>> GetCompactionResult( + bool blocking) = 0; + + /// Cancel currently running compaction task. + virtual void CancelCompaction() = 0; + + /// Check if a compaction is in progress, or if a compaction result remains to be fetched, or if + /// a compaction should be triggered later. + virtual bool CompactNotCompleted() const = 0; + + virtual bool ShouldWaitForLatestCompaction() const = 0; + + virtual bool ShouldWaitForPreparingCheckpoint() const = 0; + + virtual Status Close() = 0; +}; + +} // namespace paimon diff --git a/src/paimon/core/compact/compact_result.h b/src/paimon/core/compact/compact_result.h index 54628527..3fec8a0f 100644 --- a/src/paimon/core/compact/compact_result.h +++ b/src/paimon/core/compact/compact_result.h @@ -20,6 +20,7 @@ #include #include "paimon/core/io/data_file_meta.h" +#include "paimon/status.h" namespace paimon { class CompactDeletionFile; diff --git a/src/paimon/core/compact/compact_task.h b/src/paimon/core/compact/compact_task.h new file mode 100644 index 00000000..b82c8427 --- /dev/null +++ b/src/paimon/core/compact/compact_task.h @@ -0,0 +1,37 @@ +/* + * Copyright 2026-present Alibaba Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "paimon/core/compact/compact_result.h" +#include "paimon/result.h" + +namespace paimon { + +/// Compact task. +class CompactTask { + public: + virtual ~CompactTask() = default; + // TODO(yonghao.fyh): support metrics + Result> Execute() { + return DoCompact(); + } + + protected: + virtual Result> DoCompact() = 0; +}; + +} // namespace paimon diff --git a/src/paimon/core/compact/noop_compact_manager.h b/src/paimon/core/compact/noop_compact_manager.h new file mode 100644 index 00000000..337abb42 --- /dev/null +++ b/src/paimon/core/compact/noop_compact_manager.h @@ -0,0 +1,81 @@ +/* + * 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 "fmt/format.h" +#include "paimon/core/compact/compact_manager.h" +#include "paimon/core/compact/compact_result.h" +#include "paimon/defs.h" + +namespace paimon { + +/// A `CompactManager` which never compacts. +class NoopCompactManager : public CompactManager { + public: + NoopCompactManager() = default; + ~NoopCompactManager() override {} + + void AddNewFile(const std::shared_ptr& file) override {} + + std::vector> AllFiles() const override { + static std::vector> empty; + return empty; + } + + Status TriggerCompaction(bool full_compaction) override { + if (full_compaction) { + return Status::Invalid( + fmt::format("NoopCompactManager does not support user triggered compaction.\n" + "If you really need a guaranteed compaction, please set {} property of " + "this table to false.", + Options::WRITE_ONLY)); + } + return Status::OK(); + } + + /// Get compaction result. Wait finish if `blocking` is true. + Result>> GetCompactionResult( + bool blocking) override { + return std::optional>(); + } + + /// Cancel currently running compaction task. + void CancelCompaction() override {} + + /// Check if a compaction is in progress, or if a compaction result remains to be fetched, or if + /// a compaction should be triggered later. + bool CompactNotCompleted() const override { + return false; + } + + bool ShouldWaitForLatestCompaction() const override { + return false; + } + + bool ShouldWaitForPreparingCheckpoint() const override { + return false; + } + + Status Close() override { + return Status::OK(); + } +}; + +} // namespace paimon diff --git a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h new file mode 100644 index 00000000..53fb955b --- /dev/null +++ b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h @@ -0,0 +1,74 @@ +/* + * 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/deletionvectors/deletion_vector.h" +#include "paimon/core/deletionvectors/deletion_vectors_index_file.h" +#include "paimon/core/index/index_file_meta.h" + +namespace paimon { + +// Maintainer of deletionVectors index. +class BucketedDvMaintainer { + public: + BucketedDvMaintainer( + const std::shared_ptr& dv_index_file, + const std::map>& deletion_vectors) + : dv_index_file_(dv_index_file), + deletion_vectors_(deletion_vectors), + bitmap64_(dv_index_file->Bitmap64()) {} + + std::optional> DeletionVectorOf(const std::string& file_name) { + if (auto it = deletion_vectors_.find(file_name); it != deletion_vectors_.end()) { + return it->second; + } + return std::nullopt; + } + + void RemoveDeletionVectorOf(const std::string& file_name) { + if (deletion_vectors_.erase(file_name) > 0) { + modified_ = true; + } + } + + Result>> WriteDeletionVectorsIndex() { + if (modified_) { + modified_ = false; + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr result, + dv_index_file_->WriteSingleFile(deletion_vectors_)); + return std::make_optional>(result); + } + return std::optional>(); + } + + std::shared_ptr DvIndexFile() const { + return dv_index_file_; + } + + private: + std::shared_ptr dv_index_file_; + std::map> deletion_vectors_; + bool bitmap64_; + bool modified_ = false; +}; + +} // namespace paimon diff --git a/src/paimon/core/deletionvectors/deletion_file_writer.cpp b/src/paimon/core/deletionvectors/deletion_file_writer.cpp index 13963bc1..55b218f5 100644 --- a/src/paimon/core/deletionvectors/deletion_file_writer.cpp +++ b/src/paimon/core/deletionvectors/deletion_file_writer.cpp @@ -52,10 +52,14 @@ Result> DeletionFileWriter::GetResult() const { return Status::Invalid( fmt::format("Deletion file result length {} out of int32 range.", length)); } - return std::make_unique( - DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, PathUtil::GetName(path_), length, - dv_metas_.size(), dv_metas_, - is_external_path_ ? std::optional(path_) : std::optional()); + std::optional final_path; + if (is_external_path_) { + PAIMON_ASSIGN_OR_RAISE(Path external_path, PathUtil::ToPath(path_)); + final_path = external_path.ToString(); + } + return std::make_unique(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, + PathUtil::GetName(path_), length, dv_metas_.size(), + dv_metas_, final_path); } } // namespace paimon diff --git a/src/paimon/core/mergetree/merge_tree_writer.cpp b/src/paimon/core/mergetree/merge_tree_writer.cpp index b6668950..dba6f17e 100644 --- a/src/paimon/core/mergetree/merge_tree_writer.cpp +++ b/src/paimon/core/mergetree/merge_tree_writer.cpp @@ -171,7 +171,7 @@ Result MergeTreeWriter::DrainIncrement() { CompactIncrement compact_increment({}, {}, {}); new_files_.clear(); deleted_files_.clear(); - return CommitIncrement(data_increment, compact_increment); + return CommitIncrement(data_increment, compact_increment, /*compact_deletion_file=*/nullptr); } std::unique_ptr>> diff --git a/src/paimon/core/mergetree/merge_tree_writer.h b/src/paimon/core/mergetree/merge_tree_writer.h index 0788c014..2e94e64c 100644 --- a/src/paimon/core/mergetree/merge_tree_writer.h +++ b/src/paimon/core/mergetree/merge_tree_writer.h @@ -65,6 +65,18 @@ class MergeTreeWriter : public BatchWriter { [[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 Status::NotImplemented("not implemented"); + } + + Status Sync() override { + return Status::NotImplemented("not implemented"); + } + Result PrepareCommit(bool wait_compaction) override; bool IsCompacting() const override { diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index f6d4537b..f2d8ca4f 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -26,6 +26,8 @@ #include "paimon/common/metrics/metrics_impl.h" #include "paimon/core/manifest/manifest_entry.h" #include "paimon/core/operation/file_store_scan.h" +#include "paimon/core/operation/file_system_write_restore.h" +#include "paimon/core/operation/restore_files.h" #include "paimon/core/schema/table_schema.h" #include "paimon/core/snapshot.h" #include "paimon/core/table/bucket_mode.h" @@ -72,7 +74,10 @@ AbstractFileStoreWrite::AbstractFileStoreWrite( is_streaming_mode_(is_streaming_mode), ignore_num_bucket_check_(ignore_num_bucket_check), metrics_(std::make_shared()), - logger_(Logger::GetLogger("AbstractFileStoreWrite")) {} + logger_(Logger::GetLogger("AbstractFileStoreWrite")) { + // TODO(yonghao.fyh): support with + compact_executor_ = CreateDefaultExecutor(4); +} Status AbstractFileStoreWrite::Write(std::unique_ptr&& batch) { if (PAIMON_UNLIKELY(batch == nullptr)) { @@ -120,6 +125,14 @@ Status AbstractFileStoreWrite::Write(std::unique_ptr&& batch) { return writer->Write(std::move(batch)); } +Status AbstractFileStoreWrite::Compact(const std::map& partition, + int32_t bucket, bool full_compaction) { + PAIMON_ASSIGN_OR_RAISE(BinaryRow part, file_store_path_factory_->ToBinaryRow(partition)) + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr writer, GetWriter(part, bucket)); + assert(writer); + return writer->Compact(full_compaction); +} + Result>> AbstractFileStoreWrite::PrepareCommit( bool wait_compaction, int64_t commit_identifier) { if (batch_committed_) { @@ -227,6 +240,7 @@ Status AbstractFileStoreWrite::Close() { } } writers_.clear(); + compact_executor_->ShutdownNow(); return Status::OK(); } @@ -238,9 +252,8 @@ int32_t AbstractFileStoreWrite::GetDefaultBucketNum() const { return options_.GetBucket(); } -Result AbstractFileStoreWrite::ScanExistingFileMetas( - const Snapshot& snapshot, const BinaryRow& partition, int32_t bucket, - std::vector>* restore_files) const { +Result> AbstractFileStoreWrite::ScanExistingFileMetas( + const Snapshot& snapshot, const BinaryRow& partition, int32_t bucket) const { PAIMON_ASSIGN_OR_RAISE(auto part_values, file_store_path_factory_->GeneratePartitionVector(partition)); std::map part_values_map; @@ -256,22 +269,28 @@ Result AbstractFileStoreWrite::ScanExistingFileMetas( /*vector_search=*/nullptr); PAIMON_ASSIGN_OR_RAISE(std::unique_ptr scan, CreateFileStoreScan(scan_filter)); - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr plan, - scan->WithSnapshot(snapshot)->CreatePlan()); - std::vector entries = plan->Files(); + // PAIMON_ASSIGN_OR_RAISE(std::shared_ptr plan, + // scan->WithSnapshot(snapshot)->CreatePlan()); + // std::vector entries = plan->Files(); + + // TODO(yonghao.fyh): create index file handler + FileSystemWriteRestore restore(options_, snapshot_manager_, std::move(scan)); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, restore.GetRestoreFiles()); + + std::optional restored_total_buckets = restore_files->TotalBuckets(); int32_t total_buckets = GetDefaultBucketNum(); - for (auto& entry : entries) { - if (!ignore_num_bucket_check_ && entry.TotalBuckets() != options_.GetBucket()) { - return Status::Invalid(fmt::format( - "Try to write table with a new bucket num {}, but the previous " - "bucket num is {}. Please switch to batch mode, and perform INSERT OVERWRITE to " - "rescale current data layout first.", - options_.GetBucket(), entry.TotalBuckets())); - } - total_buckets = entry.TotalBuckets(); - restore_files->push_back(std::move(entry.File())); + if (restored_total_buckets) { + total_buckets = restored_total_buckets.value(); + } + + if (!ignore_num_bucket_check_ && total_buckets != options_.GetBucket()) { + return Status::Invalid(fmt::format( + "Try to write table with a new bucket num {}, but the previous " + "bucket num is {}. Please switch to batch mode, and perform INSERT OVERWRITE to " + "rescale current data layout first.", + options_.GetBucket(), total_buckets)); } - return total_buckets; + return restore_files; } Result> AbstractFileStoreWrite::GetWriter(const BinaryRow& partition, diff --git a/src/paimon/core/operation/abstract_file_store_write.h b/src/paimon/core/operation/abstract_file_store_write.h index d2eb86f2..d76b82c6 100644 --- a/src/paimon/core/operation/abstract_file_store_write.h +++ b/src/paimon/core/operation/abstract_file_store_write.h @@ -57,6 +57,7 @@ class BinaryRow; class Executor; class MemoryPool; class RecordBatch; +class RestoreFiles; class AbstractFileStoreWrite : public FileStoreWrite { public: @@ -76,6 +77,9 @@ class AbstractFileStoreWrite : public FileStoreWrite { const std::shared_ptr& pool); Status Write(std::unique_ptr&& batch) override; + Status Compact(const std::map& partition, int32_t bucket, + bool full_compaction) override; + Result>> PrepareCommit( bool wait_compaction, int64_t commit_identifier) override; Status Close() override; @@ -104,10 +108,9 @@ class AbstractFileStoreWrite : public FileStoreWrite { virtual Result> CreateFileStoreScan( const std::shared_ptr& filter) const = 0; - // return actual total bucket in the specific partition - Result ScanExistingFileMetas( - const Snapshot& snapshot, const BinaryRow& partition, int32_t bucket, - std::vector>* restore_files) const; + Result> ScanExistingFileMetas(const Snapshot& snapshot, + const BinaryRow& partition, + int32_t bucket) const; int32_t GetDefaultBucketNum() const; std::shared_ptr pool_; @@ -122,6 +125,7 @@ class AbstractFileStoreWrite : public FileStoreWrite { std::shared_ptr table_schema_; std::shared_ptr partition_schema_; CoreOptions options_; + std::shared_ptr compact_executor_; private: Result> GetWriter(const BinaryRow& partition, int32_t bucket); diff --git a/src/paimon/core/operation/abstract_split_read.cpp b/src/paimon/core/operation/abstract_split_read.cpp index 0e75da6c..349f8a3d 100644 --- a/src/paimon/core/operation/abstract_split_read.cpp +++ b/src/paimon/core/operation/abstract_split_read.cpp @@ -105,17 +105,21 @@ bool AbstractSplitRead::NeedCompleteRowTrackingFields( std::unordered_map AbstractSplitRead::CreateDeletionFileMap( const DataSplitImpl& data_split) { + return CreateDeletionFileMap(data_split.DataFiles(), data_split.DeletionFiles()); +} + +std::unordered_map AbstractSplitRead::CreateDeletionFileMap( + const std::vector>& data_files, + const std::vector>& deletion_files) { std::unordered_map deletion_file_map; - auto deletion_files = data_split.DeletionFiles(); if (deletion_files.empty()) { return deletion_file_map; } - auto data_file_metas = data_split.DataFiles(); - assert(deletion_files.size() == data_file_metas.size()); + assert(deletion_files.size() == data_files.size()); size_t file_count = deletion_files.size(); for (size_t i = 0; i < file_count; i++) { if (deletion_files[i] != std::nullopt) { - deletion_file_map.emplace(data_file_metas[i]->file_name, deletion_files[i].value()); + deletion_file_map.emplace(data_files[i]->file_name, deletion_files[i].value()); } } return deletion_file_map; diff --git a/src/paimon/core/operation/abstract_split_read.h b/src/paimon/core/operation/abstract_split_read.h index 877b7fea..73b0549a 100644 --- a/src/paimon/core/operation/abstract_split_read.h +++ b/src/paimon/core/operation/abstract_split_read.h @@ -77,6 +77,10 @@ class AbstractSplitRead : public SplitRead { static std::unordered_map CreateDeletionFileMap( const DataSplitImpl& data_split); + static std::unordered_map CreateDeletionFileMap( + const std::vector>& data_files, + const std::vector>& deletion_files); + 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 9336e183..9a711a9e 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -19,18 +19,30 @@ #include #include "paimon/common/data/binary_row.h" +#include "paimon/common/reader/reader_utils.h" +#include "paimon/common/table/special_fields.h" #include "paimon/core/append/append_only_writer.h" +#include "paimon/core/append/bucketed_append_compact_manager.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/io/data_file_path_factory.h" +#include "paimon/core/io/data_file_writer.h" +#include "paimon/core/io/rolling_file_writer.h" #include "paimon/core/manifest/manifest_file.h" #include "paimon/core/manifest/manifest_list.h" #include "paimon/core/operation/append_only_file_store_scan.h" #include "paimon/core/operation/file_store_scan.h" +#include "paimon/core/operation/internal_read_context.h" +#include "paimon/core/operation/raw_file_split_read.h" +#include "paimon/core/operation/restore_files.h" #include "paimon/core/schema/table_schema.h" #include "paimon/core/snapshot.h" #include "paimon/core/utils/file_store_path_factory.h" #include "paimon/core/utils/snapshot_manager.h" +#include "paimon/executor.h" #include "paimon/logging.h" +#include "paimon/read_context.h" #include "paimon/result.h" namespace arrow { @@ -39,7 +51,6 @@ class Schema; namespace paimon { class DataFilePathFactory; -class Executor; class MemoryPool; class SchemaManager; @@ -59,6 +70,10 @@ AppendOnlyFileStoreWrite::AppendOnlyFileStoreWrite( ignore_num_bucket_check, executor, pool), logger_(Logger::GetLogger("AppendOnlyFileStoreWrite")) { write_cols_ = write_schema->field_names(); + auto schemas = BlobUtils::SeparateBlobSchema(schema_); + if (schemas.blob_schema && schemas.blob_schema->num_fields() > 0) { + with_blob_ = true; + } // optimize write_cols to null in following cases: // 1. write_schema contains all columns // 2. TODO(xinyu.lxy) write_schema contains all columns and append _ROW_ID & _SEQUENCE_NUMBER @@ -89,27 +104,145 @@ Result> AppendOnlyFileStoreWrite::CreateFileStore return scan; } +Result>> AppendOnlyFileStoreWrite::CompactRewrite( + const BinaryRow& partition, int32_t bucket, + const std::vector>& to_compact) { + if (to_compact.empty()) { + return std::vector>{}; + } + + auto rewriter = + std::make_unique>>( + options_.GetTargetFileSize(/*has_primary_key=*/false), + GetDataFileWriterCreator(partition, bucket, write_schema_, write_cols_, to_compact)); + // TODO(yonghao.fyh): support dv factory + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr reader, + CreateFilesReader(partition, bucket, to_compact)); + ScopeGuard guard([&]() { + if (rewriter) { + (void)rewriter->Close(); + } + }); + + while (true) { + PAIMON_ASSIGN_OR_RAISE(BatchReader::ReadBatch batch, reader->NextBatch()); + if (BatchReader::IsEofBatch(batch)) { + break; + } + auto& [c_array, c_schema] = batch; + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW(std::shared_ptr arrow_array, + arrow::ImportArray(c_array.get(), c_schema.get())); + auto struct_array = std::dynamic_pointer_cast(arrow_array); + if (!struct_array) { + return Status::Invalid( + "cannot cast array to StructArray in CompleteRowKindBatchReader"); + } + PAIMON_ASSIGN_OR_RAISE(struct_array, ReaderUtils::RemoveFieldFromStructArray( + struct_array, SpecialFields::ValueKind().Name())); + PAIMON_RETURN_NOT_OK_FROM_ARROW( + arrow::ExportArray(*struct_array, c_array.get(), c_schema.get())); + PAIMON_RETURN_NOT_OK(rewriter->Write(c_array.get())); + } + guard.Release(); + PAIMON_RETURN_NOT_OK(rewriter->Close()); + return rewriter->GetResult(); +} + Result>> AppendOnlyFileStoreWrite::CreateWriter( const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) { PAIMON_LOG_DEBUG(logger_, "Creating append only writer for partition %s, bucket %d", partition.ToString().c_str(), bucket); PAIMON_ASSIGN_OR_RAISE(std::optional latest_snapshot, snapshot_manager_->LatestSnapshot()); - std::vector> restore_files; int32_t total_buckets = GetDefaultBucketNum(); + std::vector> restore_data_files; if (!ignore_previous_files && latest_snapshot != std::nullopt) { - PAIMON_ASSIGN_OR_RAISE( - total_buckets, - ScanExistingFileMetas(latest_snapshot.value(), partition, bucket, &restore_files)); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, + ScanExistingFileMetas(latest_snapshot.value(), partition, bucket)); + restore_data_files = restore_files->DataFiles(); + if (restore_files->TotalBuckets()) { + total_buckets = restore_files->TotalBuckets().value(); + } } - int64_t max_sequence_number = DataFileMeta::GetMaxSequenceNumber(restore_files); + int64_t max_sequence_number = DataFileMeta::GetMaxSequenceNumber(restore_data_files); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr data_file_path_factory, file_store_path_factory_->CreateDataFilePathFactory(partition, bucket)); - auto writer = std::make_shared(options_, table_schema_->Id(), write_schema_, - write_cols_, max_sequence_number, - data_file_path_factory, pool_); + std::shared_ptr compact_manager; + auto schemas = BlobUtils::SeparateBlobSchema(write_schema_); + if (options_.WriteOnly() || with_blob_) { + // TODO(yonghao.fyh): check data evolution + compact_manager = std::make_shared(); + } else { + auto rewriter = [this, partition, + bucket](const std::vector>& files) + -> Result>> { + return CompactRewrite(partition, bucket, files); + }; + std::vector> restored; + compact_manager = std::make_shared( + compact_executor_, restore_data_files, /*dv_maintainer=*/nullptr, + options_.GetCompactionMinFileNum(), + options_.GetTargetFileSize(/*has_primary_key=*/false), + options_.GetCompactionFileSize(/*has_primary_key=*/false), + options_.CompactionForceRewriteAllFiles(), rewriter); + } + + auto writer = std::make_shared( + options_, table_schema_->Id(), write_schema_, write_cols_, max_sequence_number, + data_file_path_factory, pool_, compact_manager); return std::pair>(total_buckets, writer); } +AppendOnlyFileStoreWrite::SingleFileWriterCreator +AppendOnlyFileStoreWrite::GetDataFileWriterCreator( + const BinaryRow& partition, int32_t bucket, const std::shared_ptr& schema, + const std::optional>& write_cols, + const std::vector>& to_compact) const { + return + [this, partition, bucket, schema, write_cols, to_compact]() + -> Result< + std::unique_ptr>>> { + ::ArrowSchema arrow_schema; + ScopeGuard guard([&arrow_schema]() { ArrowSchemaRelease(&arrow_schema); }); + PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*schema, &arrow_schema)); + auto format = options_.GetWriteFileFormat(); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr writer_builder, + format->CreateWriterBuilder(&arrow_schema, options_.GetWriteBatchSize())); + writer_builder->WithMemoryPool(pool_); + + PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*schema, &arrow_schema)); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr stats_extractor, + format->CreateStatsExtractor(&arrow_schema)); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr data_file_path_factory, + file_store_path_factory_->CreateDataFilePathFactory(partition, bucket)); + // TODO(yonghao.fyh): check sequence number + auto writer = std::make_unique( + options_.GetFileCompression(), std::function(), + table_schema_->Id(), + std::make_shared(to_compact[0]->min_sequence_number), + FileSource::Compact(), stats_extractor, data_file_path_factory->IsExternalPath(), + write_cols, pool_); + PAIMON_RETURN_NOT_OK(writer->Init(options_.GetFileSystem(), + data_file_path_factory->NewPath(), writer_builder)); + return writer; + }; +} + +Result> AppendOnlyFileStoreWrite::CreateFilesReader( + const BinaryRow& partition, int32_t bucket, + const std::vector>& files) const { + ReadContextBuilder context_builder(root_path_); + context_builder.EnablePrefetch(false).SetPrefetchMaxParallelNum(1); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr read_context, context_builder.Finish()); + std::map map = options_.ToMap(); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr internal_read_context, + InternalReadContext::Create(read_context, table_schema_, map)); + auto read = std::make_unique(file_store_path_factory_, internal_read_context, + pool_, compact_executor_); + return read->CreateReader(partition, bucket, files, {}); +} + } // namespace paimon 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 29367d4c..ee3a475c 100644 --- a/src/paimon/core/operation/append_only_file_store_write.h +++ b/src/paimon/core/operation/append_only_file_store_write.h @@ -27,6 +27,7 @@ #include "arrow/type.h" #include "paimon/common/data/binary_row.h" #include "paimon/core/core_options.h" +#include "paimon/core/io/single_file_writer.h" #include "paimon/core/operation/abstract_file_store_write.h" #include "paimon/core/table/bucket_mode.h" #include "paimon/file_store_write.h" @@ -43,6 +44,7 @@ class Schema; namespace paimon { +struct DataFileMeta; class BatchWriter; class FileStorePathFactory; class FileStoreScan; @@ -74,14 +76,30 @@ class AppendOnlyFileStoreWrite : public AbstractFileStoreWrite { ~AppendOnlyFileStoreWrite() override; private: + using SingleFileWriterCreator = std::function< + Result>>>()>; + Result>> CreateWriter( const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) override; Result> CreateFileStoreScan( const std::shared_ptr& filter) const override; - private: + Result>> CompactRewrite( + const BinaryRow& partition, int32_t bucket, + const std::vector>& to_compact); + + SingleFileWriterCreator GetDataFileWriterCreator( + const BinaryRow& partition, int32_t bucket, const std::shared_ptr& schema, + const std::optional>& write_cols, + const std::vector>& to_compact) const; + + Result> CreateFilesReader( + const BinaryRow& partition, int32_t bucket, + const std::vector>& files) const; + std::optional> write_cols_; + bool with_blob_ = false; std::unique_ptr logger_; }; diff --git a/src/paimon/core/operation/append_only_file_store_write_test.cpp b/src/paimon/core/operation/append_only_file_store_write_test.cpp index 9b6fe21f..573f89e9 100644 --- a/src/paimon/core/operation/append_only_file_store_write_test.cpp +++ b/src/paimon/core/operation/append_only_file_store_write_test.cpp @@ -34,6 +34,7 @@ #include "paimon/common/data/binary_row_writer.h" #include "paimon/common/utils/path_util.h" #include "paimon/core/io/data_file_meta.h" +#include "paimon/core/operation/restore_files.h" #include "paimon/core/snapshot.h" #include "paimon/core/utils/snapshot_manager.h" #include "paimon/file_store_write.h" @@ -143,36 +144,33 @@ TEST_F(AppendOnlyFileStoreWriteTest, TestGetMaxSequenceNumberFromMultiPartition) BinaryRowWriter writer(&partition, 20, pool.get()); writer.WriteInt(0, 20); writer.WriteInt(1, 1); - std::vector> restore_files; - ASSERT_OK_AND_ASSIGN(int32_t total_buckets, + ASSERT_OK_AND_ASSIGN(std::shared_ptr restore_files, write->ScanExistingFileMetas(latest_snapshot.value(), partition, - /*bucket=*/0, &restore_files)); - ASSERT_EQ(-1, total_buckets); - ASSERT_EQ(0, DataFileMeta::GetMaxSequenceNumber(restore_files)); + /*bucket=*/0)); + ASSERT_EQ(-1, restore_files->TotalBuckets().value()); + ASSERT_EQ(0, DataFileMeta::GetMaxSequenceNumber(restore_files->DataFiles())); } { BinaryRow partition(2); BinaryRowWriter writer(&partition, 20, pool.get()); writer.WriteInt(0, 10); writer.WriteInt(1, 0); - std::vector> restore_files; - ASSERT_OK_AND_ASSIGN(int32_t total_buckets, + ASSERT_OK_AND_ASSIGN(std::shared_ptr restore_files, write->ScanExistingFileMetas(latest_snapshot.value(), partition, - /*bucket=*/0, &restore_files)); - ASSERT_EQ(-1, total_buckets); - ASSERT_EQ(2, DataFileMeta::GetMaxSequenceNumber(restore_files)); + /*bucket=*/0)); + ASSERT_EQ(-1, restore_files->TotalBuckets().value()); + ASSERT_EQ(2, DataFileMeta::GetMaxSequenceNumber(restore_files->DataFiles())); } { BinaryRow partition(2); BinaryRowWriter writer(&partition, 20, pool.get()); writer.WriteInt(0, 10); writer.WriteInt(1, 0); - std::vector> restore_files; - ASSERT_OK_AND_ASSIGN(int32_t total_buckets, + ASSERT_OK_AND_ASSIGN(std::shared_ptr restore_files, write->ScanExistingFileMetas(latest_snapshot.value(), partition, - /*bucket=*/1, &restore_files)); - ASSERT_EQ(-1, total_buckets); - ASSERT_EQ(-1, DataFileMeta::GetMaxSequenceNumber(restore_files)); + /*bucket=*/1)); + ASSERT_EQ(-1, restore_files->TotalBuckets().value()); + ASSERT_EQ(-1, DataFileMeta::GetMaxSequenceNumber(restore_files->DataFiles())); } } diff --git a/src/paimon/core/operation/file_store_commit_impl.cpp b/src/paimon/core/operation/file_store_commit_impl.cpp index a9c4a4d0..87bcfbcd 100644 --- a/src/paimon/core/operation/file_store_commit_impl.cpp +++ b/src/paimon/core/operation/file_store_commit_impl.cpp @@ -390,6 +390,16 @@ Status FileStoreCommitImpl::Commit(const std::shared_ptr& c attempt += cnt; ++generated_snapshot; } + + if (!compact_table_files.empty() || !compact_table_index_files.empty()) { + PAIMON_ASSIGN_OR_RAISE( + int32_t cnt, TryCommit(compact_table_files, compact_table_index_files, + committable->Identifier(), committable->Watermark(), + committable->LogOffsets(), committable->Properties(), + Snapshot::CommitKind::Compact(), /*check_append_files=*/true)); + attempt += cnt; + ++generated_snapshot; + } auto table_files_added = static_cast(append_table_files.size()); int32_t table_files_deleted = 0; int64_t compaction_input_file_size = 0; diff --git a/src/paimon/core/operation/file_system_write_restore.h b/src/paimon/core/operation/file_system_write_restore.h new file mode 100644 index 00000000..449d2743 --- /dev/null +++ b/src/paimon/core/operation/file_system_write_restore.h @@ -0,0 +1,78 @@ +/* + * Copyright 2026-present Alibaba Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "paimon/core/core_options.h" +#include "paimon/core/operation/file_store_scan.h" +#include "paimon/core/operation/restore_files.h" +#include "paimon/core/operation/write_restore.h" +#include "paimon/core/utils/snapshot_manager.h" + +namespace paimon { + +/// `WriteRestore` to restore files directly from file system. +class FileSystemWriteRestore : public WriteRestore { + public: + FileSystemWriteRestore(const CoreOptions& options, + const std::shared_ptr& snapshot_manager, + std::unique_ptr&& scan) + : snapshot_manager_(snapshot_manager), scan_(std::move(scan)) { + // TODO(yonghao.fyh): support index file handler + } + + Result LatestCommittedIdentifier(const std::string& user) const override { + // TODO(yonghao.fyh): in java paimon is LatestSnapshotOfUserFromFileSystem + PAIMON_ASSIGN_OR_RAISE(std::optional latest_snapshot, + snapshot_manager_->LatestSnapshotOfUser(user)); + if (latest_snapshot) { + return latest_snapshot.value().CommitIdentifier(); + } + return std::numeric_limits::min(); + } + + Result> GetRestoreFiles() const override { + // TODO(yonghao.fyh): java paimon doesn't use snapshot_manager.LatestSnapshot() here, + // because they don't want to flood the catalog with high concurrency + PAIMON_ASSIGN_OR_RAISE(std::optional snapshot, + snapshot_manager_->LatestSnapshot()); + if (snapshot == std::nullopt) { + return RestoreFiles::Empty(); + } + + // Plan scan + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr plan, + scan_->WithSnapshot(snapshot.value())->CreatePlan()); + std::vector entries = plan->Files(); + std::vector> restore_files; + PAIMON_ASSIGN_OR_RAISE(std::optional total_buckets, + WriteRestore::ExtractDataFiles(entries, &restore_files)); + + std::shared_ptr dynamic_bucket_index; + std::vector> delete_vectors_index; + + return std::make_shared( + snapshot, total_buckets, restore_files, + /*dynamic_bucket_index=*/nullptr, + /*delete_vectors_index=*/std::vector>{}); + } + + private: + std::shared_ptr snapshot_manager_; + std::unique_ptr scan_; +}; + +} // namespace paimon 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 f4821f00..5771a585 100644 --- a/src/paimon/core/operation/key_value_file_store_write.cpp +++ b/src/paimon/core/operation/key_value_file_store_write.cpp @@ -27,6 +27,7 @@ #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" +#include "paimon/core/operation/restore_files.h" #include "paimon/core/schema/table_schema.h" #include "paimon/core/snapshot.h" #include "paimon/core/utils/file_store_path_factory.h" @@ -92,14 +93,17 @@ Result>> KeyValueFileStoreWrite: partition.ToString().c_str(), bucket); PAIMON_ASSIGN_OR_RAISE(std::optional latest_snapshot, snapshot_manager_->LatestSnapshot()); - std::vector> restore_files; int32_t total_buckets = GetDefaultBucketNum(); + std::vector> restore_data_files; if (!ignore_previous_files && latest_snapshot != std::nullopt) { - PAIMON_ASSIGN_OR_RAISE( - total_buckets, - ScanExistingFileMetas(latest_snapshot.value(), partition, bucket, &restore_files)); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, + ScanExistingFileMetas(latest_snapshot.value(), partition, bucket)); + restore_data_files = restore_files->DataFiles(); + if (restore_files->TotalBuckets()) { + total_buckets = restore_files->TotalBuckets().value(); + } } - int64_t max_sequence_number = DataFileMeta::GetMaxSequenceNumber(restore_files); + int64_t max_sequence_number = DataFileMeta::GetMaxSequenceNumber(restore_data_files); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr data_file_path_factory, file_store_path_factory_->CreateDataFilePathFactory(partition, bucket)); PAIMON_ASSIGN_OR_RAISE(std::vector trimmed_primary_keys, diff --git a/src/paimon/core/operation/raw_file_split_read.cpp b/src/paimon/core/operation/raw_file_split_read.cpp index 286c0775..cbc72111 100644 --- a/src/paimon/core/operation/raw_file_split_read.cpp +++ b/src/paimon/core/operation/raw_file_split_read.cpp @@ -66,15 +66,23 @@ Result> RawFileSplitRead::CreateReader( if (!data_split) { return Status::Invalid("cannot cast split to data_split in RawFileSplitRead"); } - auto deletion_file_map = CreateDeletionFileMap(*data_split); + return CreateReader(data_split->Partition(), data_split->Bucket(), data_split->DataFiles(), + data_split->DeletionFiles()); +} + +Result> RawFileSplitRead::CreateReader( + const BinaryRow& partition, int32_t bucket, + const std::vector>& data_files, + const std::vector>& deletion_files) { const auto& predicate = context_->GetPredicate(); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr data_file_path_factory, + path_factory_->CreateDataFilePathFactory(partition, bucket)); + + auto deletion_file_map = CreateDeletionFileMap(data_files, deletion_files); PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr data_file_path_factory, - path_factory_->CreateDataFilePathFactory(data_split->Partition(), data_split->Bucket())); - PAIMON_ASSIGN_OR_RAISE(std::vector> raw_file_readers, - CreateRawFileReaders(data_split->Partition(), data_split->DataFiles(), - raw_read_schema_, predicate, deletion_file_map, - /*row_ranges=*/{}, data_file_path_factory)); + std::vector> raw_file_readers, + CreateRawFileReaders(partition, data_files, raw_read_schema_, predicate, deletion_file_map, + /*row_ranges=*/{}, data_file_path_factory)); auto concat_batch_reader = std::make_unique(std::move(raw_file_readers), pool_); PAIMON_ASSIGN_OR_RAISE(std::unique_ptr batch_reader, diff --git a/src/paimon/core/operation/raw_file_split_read.h b/src/paimon/core/operation/raw_file_split_read.h index ed017907..7c822c65 100644 --- a/src/paimon/core/operation/raw_file_split_read.h +++ b/src/paimon/core/operation/raw_file_split_read.h @@ -62,6 +62,10 @@ class RawFileSplitRead : public AbstractSplitRead { const std::shared_ptr& executor); Result> CreateReader(const std::shared_ptr& split) override; + Result> CreateReader( + const BinaryRow& partition, int32_t bucket, + const std::vector>& files, + const std::vector>& deletion_files); Result Match(const std::shared_ptr& split, bool force_keep_delete) const override; diff --git a/src/paimon/core/operation/restore_files.h b/src/paimon/core/operation/restore_files.h new file mode 100644 index 00000000..a2d20b07 --- /dev/null +++ b/src/paimon/core/operation/restore_files.h @@ -0,0 +1,74 @@ +/* + * 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/index/index_file_meta.h" +#include "paimon/core/io/data_file_meta.h" +#include "paimon/core/snapshot.h" + +namespace paimon { + +/// Restored files with snapshot and total buckets. +class RestoreFiles { + public: + RestoreFiles() = default; + + RestoreFiles(const std::optional& snapshot, + const std::optional& total_buckets, + const std::vector>& data_files, + const std::shared_ptr& dynamic_bucket_index, + const std::vector>& delete_vectors_index) + : snapshot_(snapshot), + total_buckets_(total_buckets), + data_files_(data_files), + dynamic_bucket_index_(dynamic_bucket_index), + delete_vectors_index_(delete_vectors_index) {} + + std::optional GetSnapshot() const { + return snapshot_; + } + std::optional TotalBuckets() const { + return total_buckets_; + } + std::vector> DataFiles() const { + return data_files_; + } + std::shared_ptr DynamicBucketIndex() const { + return dynamic_bucket_index_; + } + std::vector> DeleteVectorsIndex() const { + return delete_vectors_index_; + } + + static std::shared_ptr Empty() { + return std::make_shared(); + } + + private: + std::optional snapshot_; + std::optional total_buckets_; + std::vector> data_files_; + std::shared_ptr dynamic_bucket_index_; + std::vector> delete_vectors_index_; +}; + +} // namespace paimon diff --git a/src/paimon/core/operation/write_restore.cpp b/src/paimon/core/operation/write_restore.cpp new file mode 100644 index 00000000..56e7f748 --- /dev/null +++ b/src/paimon/core/operation/write_restore.cpp @@ -0,0 +1,42 @@ +/* + * Copyright 2026-present Alibaba Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "paimon/core/operation/write_restore.h" + +#include +#include +#include + +namespace paimon { + +Result> WriteRestore::ExtractDataFiles( + const std::vector& entries, + std::vector>* data_files) { + std::optional total_buckets; + for (const auto& entry : entries) { + if (total_buckets.has_value() && total_buckets.value() != entry.TotalBuckets()) { + return Status::Invalid(fmt::format( + "Bucket data files has different total bucket number, {} vs {}, this should " + "be a bug.", + total_buckets.value(), entry.TotalBuckets())); + } + total_buckets = entry.TotalBuckets(); + data_files->push_back(entry.File()); + } + return total_buckets; +} + +} // namespace paimon diff --git a/src/paimon/core/operation/write_restore.h b/src/paimon/core/operation/write_restore.h new file mode 100644 index 00000000..698d31ac --- /dev/null +++ b/src/paimon/core/operation/write_restore.h @@ -0,0 +1,46 @@ +/* + * 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 "fmt/format.h" +#include "paimon/common/data/binary_row.h" +#include "paimon/core/manifest/manifest_entry.h" +#include "paimon/core/operation/restore_files.h" +#include "paimon/result.h" + +namespace paimon { + +/// Restore for write to restore data files by partition and bucket from file system. +class WriteRestore { + public: + static Result> ExtractDataFiles( + const std::vector& entries, + std::vector>* data_files); + + virtual ~WriteRestore() = default; + + virtual Result LatestCommittedIdentifier(const std::string& user) const = 0; + + virtual Result> GetRestoreFiles() const = 0; +}; + +} // namespace paimon diff --git a/src/paimon/core/postpone/postpone_bucket_writer.cpp b/src/paimon/core/postpone/postpone_bucket_writer.cpp index b40b27fa..24b5a78f 100644 --- a/src/paimon/core/postpone/postpone_bucket_writer.cpp +++ b/src/paimon/core/postpone/postpone_bucket_writer.cpp @@ -281,7 +281,7 @@ Result PostponeBucketWriter::DrainIncrement() { DataIncrement data_increment(std::move(new_files_), /*deleted_files=*/{}, {}); CompactIncrement compact_increment({}, {}, {}); new_files_.clear(); - return CommitIncrement(data_increment, compact_increment); + return CommitIncrement(data_increment, compact_increment, /*compact_deletion_file=*/nullptr); } } // namespace paimon diff --git a/src/paimon/core/postpone/postpone_bucket_writer.h b/src/paimon/core/postpone/postpone_bucket_writer.h index 2c845aec..f6519e5b 100644 --- a/src/paimon/core/postpone/postpone_bucket_writer.h +++ b/src/paimon/core/postpone/postpone_bucket_writer.h @@ -60,6 +60,18 @@ class PostponeBucketWriter : public BatchWriter { Status Write(std::unique_ptr&& batch) override; + Status Compact(bool full_compaction) override { + return Status::NotImplemented("not implemented"); + } + + Result CompactNotCompleted() override { + return Status::NotImplemented("not implemented"); + } + + Status Sync() override { + return Status::NotImplemented("not implemented"); + } + Result PrepareCommit(bool wait_compaction) override; bool IsCompacting() const override { diff --git a/src/paimon/core/utils/batch_writer.h b/src/paimon/core/utils/batch_writer.h index 741ef18f..5098eb19 100644 --- a/src/paimon/core/utils/batch_writer.h +++ b/src/paimon/core/utils/batch_writer.h @@ -37,6 +37,13 @@ class BatchWriter { /// Add a record batch to the writer. virtual Status Write(std::unique_ptr&& batch) = 0; + + /// Compact files related to the writer. Note that compaction process is only submitted and may + /// not be completed when the method returns. + /// + /// @param full_compaction whether to trigger full compaction or just normal compaction + virtual Status Compact(bool full_compaction) = 0; + /// Prepare for a commit. /// /// @param wait_compaction if this method need to wait for current compaction to complete @@ -44,6 +51,15 @@ class BatchWriter { virtual Result PrepareCommit(bool wait_compaction) = 0; /// Check if a compaction is in progress, or if a compaction result remains to be fetched. virtual bool IsCompacting() const = 0; + + /// Check if a compaction is in progress, or if a compaction result remains to be fetched, or if + /// a compaction should be triggered later. + virtual Result CompactNotCompleted() = 0; + + /// Sync the writer. The structure related to file reading and writing is thread unsafe, there + /// are asynchronous threads inside the writer, which should be synced before reading data. + virtual Status Sync() = 0; + /// Close this writer, the call will delete newly generated but not committed files. virtual Status Close() = 0; diff --git a/src/paimon/core/utils/commit_increment.h b/src/paimon/core/utils/commit_increment.h index e8f5a508..89a72c4e 100644 --- a/src/paimon/core/utils/commit_increment.h +++ b/src/paimon/core/utils/commit_increment.h @@ -16,6 +16,9 @@ #pragma once +#include + +#include "paimon/core/compact/compact_deletion_file.h" #include "paimon/core/io/compact_increment.h" #include "paimon/core/io/data_increment.h" @@ -24,8 +27,11 @@ namespace paimon { // Changes to commit. class CommitIncrement { public: - CommitIncrement(const DataIncrement& data_increment, const CompactIncrement& compact_increment) - : data_increment_(data_increment), compact_increment_(compact_increment) {} + CommitIncrement(const DataIncrement& data_increment, const CompactIncrement& compact_increment, + const std::shared_ptr& compact_deletion_file) + : data_increment_(data_increment), + compact_increment_(compact_increment), + compact_deletion_file_(compact_deletion_file) {} const DataIncrement& GetNewFilesIncrement() const { return data_increment_; @@ -35,9 +41,14 @@ class CommitIncrement { return compact_increment_; } + std::shared_ptr GetCompactDeletionFile() const { + return compact_deletion_file_; + } + private: DataIncrement data_increment_; CompactIncrement compact_increment_; + std::shared_ptr compact_deletion_file_; }; } // namespace paimon diff --git a/test/inte/CMakeLists.txt b/test/inte/CMakeLists.txt index be193dd2..2eda120a 100644 --- a/test/inte/CMakeLists.txt +++ b/test/inte/CMakeLists.txt @@ -82,4 +82,12 @@ if(PAIMON_BUILD_TESTS) ${TEST_STATIC_LINK_LIBS} test_utils_static ${GTEST_LINK_TOOLCHAIN}) + + add_paimon_test(compaction_inte_test + STATIC_LINK_LIBS + paimon_shared + ${TEST_STATIC_LINK_LIBS} + test_utils_static + ${GTEST_LINK_TOOLCHAIN}) + endif() diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp new file mode 100644 index 00000000..ea9d2eb8 --- /dev/null +++ b/test/inte/compaction_inte_test.cpp @@ -0,0 +1,135 @@ +/* + * 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 "arrow/c/bridge.h" +#include "gtest/gtest.h" +#include "paimon/commit_context.h" +#include "paimon/common/data/binary_row.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/table/sink/commit_message_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/testharness.h" +#include "paimon/write_context.h" + +namespace paimon::test { + +class CompactionTest : public testing::Test { + public: + std::shared_ptr MakeFileMeta(const std::string& file_name) { + return std::make_shared( + file_name, /*file_size=*/1, /*row_count=*/1, /*min_key=*/BinaryRow::EmptyRow(), + /*max_key=*/BinaryRow::EmptyRow(), /*key_stats=*/SimpleStats::EmptyStats(), + /*value_stats=*/SimpleStats::EmptyStats(), /*min_sequence_number=*/0, + /*max_sequence_number=*/0, /*schema_id=*/0, + /*level=*/0, + /*extra_files=*/std::vector>(), + /*creation_time=*/Timestamp(1721643142472ll, 0), /*delete_row_count=*/0, + /*embedded_index=*/nullptr, FileSource::Append(), + /*value_stats_cols=*/std::nullopt, + /*external_path=*/std::nullopt, /*first_row_id=*/std::nullopt, + /*write_cols=*/std::nullopt); + } + + arrow::Result> PrepareData( + const arrow::FieldVector& fields) { + arrow::StringBuilder f0_builder; + arrow::Int32Builder f1_builder; + arrow::Int32Builder f2_builder; + arrow::DoubleBuilder f3_builder; + + std::vector> data = {{"Lily", 10, 0, 17.1}}; + + for (const auto& row : data) { + ARROW_RETURN_NOT_OK(f0_builder.Append(std::get<0>(row))); + ARROW_RETURN_NOT_OK(f1_builder.Append(std::get<1>(row))); + ARROW_RETURN_NOT_OK(f2_builder.Append(std::get<2>(row))); + ARROW_RETURN_NOT_OK(f3_builder.Append(std::get<3>(row))); + } + + std::shared_ptr f0_array, f1_array, f2_array, f3_array; + ARROW_RETURN_NOT_OK(f0_builder.Finish(&f0_array)); + ARROW_RETURN_NOT_OK(f1_builder.Finish(&f1_array)); + ARROW_RETURN_NOT_OK(f2_builder.Finish(&f2_array)); + ARROW_RETURN_NOT_OK(f3_builder.Finish(&f3_array)); + + std::vector> children = {f0_array, f1_array, f2_array, + f3_array}; + auto struct_type = arrow::struct_(fields); + return std::make_shared(struct_type, f0_array->length(), children); + } +}; + +TEST_F(CompactionTest, TestCompaction) { + std::string path = paimon::test::GetDataDir() + "/" + "orc" + "/append_09.db/append_09"; + auto dir = UniqueTestDirectory::Create(); + std::string table_path = dir->Str(); + auto pool = GetDefaultPool(); + ASSERT_TRUE(TestUtil::CopyDirectory(path, table_path)); + WriteContextBuilder context_builder(table_path, "commit_user_1"); + + ASSERT_OK_AND_ASSIGN(std::unique_ptr write_context, + context_builder.WithStreamingMode(true).Finish()); + ASSERT_OK_AND_ASSIGN(std::unique_ptr write, + FileStoreWrite::Create(std::move(write_context))); + auto append_write = dynamic_cast(write.get()); + ASSERT_TRUE(append_write); + + BinaryRow partition(1); + BinaryRowWriter binary_row_writer(&partition, 0, pool.get()); + binary_row_writer.WriteInt(0, 10); + binary_row_writer.Complete(); + + { + arrow::FieldVector fields = { + arrow::field("f0", arrow::utf8()), arrow::field("f1", arrow::int32()), + arrow::field("f2", arrow::int32()), arrow::field("f3", arrow::float64())}; + auto struct_array = PrepareData(fields); + ASSERT_TRUE(struct_array.ok()); + ::ArrowArray arrow_array; + auto arrow_status = arrow::ExportArray(*struct_array.ValueUnsafe(), &arrow_array); + ASSERT_TRUE(arrow_status.ok()); + paimon::RecordBatchBuilder batch_builder(&arrow_array); + ASSERT_OK_AND_ASSIGN(std::unique_ptr record_batch, + batch_builder.SetPartition({{"f1", "10"}}).SetBucket(1).Finish()); + + ASSERT_OK(append_write->Write(std::move(record_batch))); + ASSERT_OK(append_write->Compact({{"f1", "10"}}, 1, /*full_compaction=*/true)); + } + + ASSERT_OK_AND_ASSIGN(auto msgs, append_write->PrepareCommit(/*wait_compaction=*/true, 1)); + ASSERT_EQ(msgs.size(), 1); + auto msg_impl = dynamic_cast(msgs[0].get()); + auto compact_increment = msg_impl->GetCompactIncrement(); + ASSERT_EQ(compact_increment.CompactBefore().size(), 4); + ASSERT_EQ(compact_increment.CompactAfter().size(), 1); + std::cout << compact_increment.CompactAfter()[0]->file_name << std::endl; + CommitContextBuilder commit_context_builder(table_path, "commit_user_1"); + ASSERT_OK_AND_ASSIGN(std::unique_ptr commit_context, + commit_context_builder.IgnoreEmptyCommit(false).Finish()); + ASSERT_OK_AND_ASSIGN(std::unique_ptr commit, + FileStoreCommit::Create(std::move(commit_context))); + ASSERT_OK(commit->Commit(msgs, 2)); +} + +} // namespace paimon::test From 49a054c2e78f98e7ac500dfd8051d749289f9c1e Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 6 Mar 2026 17:11:52 +0800 Subject: [PATCH 02/21] fix --- include/paimon/file_store_write.h | 8 + .../append/bucketed_append_compact_manager.h | 1 + src/paimon/testing/utils/data_generator.cpp | 8 +- test/inte/compaction_inte_test.cpp | 253 ++++++++++++------ 4 files changed, 179 insertions(+), 91 deletions(-) diff --git a/include/paimon/file_store_write.h b/include/paimon/file_store_write.h index 6721a660..9e5ead81 100644 --- a/include/paimon/file_store_write.h +++ b/include/paimon/file_store_write.h @@ -52,6 +52,14 @@ class PAIMON_EXPORT FileStoreWrite { /// the corresponding array in `batch` must have zero null entries. virtual Status Write(std::unique_ptr&& batch) = 0; + /// Compact data stored in given partition and bucket. Note that compaction process is only + /// submitted and may not be completed when the method returns. + /// + /// @param partition the partition to compact + /// @param bucket the bucket to compact + /// @param full_compaction whether to trigger full compaction or just normal compaction + /// + /// @return status for compacting the records virtual Status Compact(const std::map& partition, int32_t bucket, bool full_compaction) = 0; diff --git a/src/paimon/core/append/bucketed_append_compact_manager.h b/src/paimon/core/append/bucketed_append_compact_manager.h index c76c7c9f..1b64ced1 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.h +++ b/src/paimon/core/append/bucketed_append_compact_manager.h @@ -24,6 +24,7 @@ #include #include "paimon/common/executor/future.h" +#include "paimon/core/compact/compact_deletion_file.h" #include "paimon/core/compact/compact_future_manager.h" #include "paimon/core/compact/compact_task.h" #include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" diff --git a/src/paimon/testing/utils/data_generator.cpp b/src/paimon/testing/utils/data_generator.cpp index 5aeaaa58..1e9883c2 100644 --- a/src/paimon/testing/utils/data_generator.cpp +++ b/src/paimon/testing/utils/data_generator.cpp @@ -269,8 +269,12 @@ Result>> DataGenerator::SplitArrayByPar auto struct_builder = struct_builder_iter->second; PAIMON_RETURN_NOT_OK_FROM_ARROW(struct_builder->Append()); for (size_t i = 0; i < fields.size(); i++) { - PAIMON_RETURN_NOT_OK( - AppendValue(binary_row, i, fields[i].Type(), struct_builder.get())); + if (binary_row.IsNullAt(i)) { + PAIMON_RETURN_NOT_OK_FROM_ARROW(struct_builder->field_builder(i)->AppendNull()); + } else { + PAIMON_RETURN_NOT_OK( + AppendValue(binary_row, i, fields[i].Type(), struct_builder.get())); + } } auto row_kinds_iter = row_kinds_holder.find({partition_map, bucket_id}); diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index ea9d2eb8..fb2ab336 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -25,111 +25,186 @@ #include "paimon/core/io/data_file_meta.h" #include "paimon/core/operation/append_only_file_store_write.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/test_helper.h" #include "paimon/testing/utils/testharness.h" #include "paimon/write_context.h" namespace paimon::test { -class CompactionTest : public testing::Test { +class CompactionInteTest : public testing::Test, public ::testing::WithParamInterface { public: - std::shared_ptr MakeFileMeta(const std::string& file_name) { - return std::make_shared( - file_name, /*file_size=*/1, /*row_count=*/1, /*min_key=*/BinaryRow::EmptyRow(), - /*max_key=*/BinaryRow::EmptyRow(), /*key_stats=*/SimpleStats::EmptyStats(), - /*value_stats=*/SimpleStats::EmptyStats(), /*min_sequence_number=*/0, - /*max_sequence_number=*/0, /*schema_id=*/0, - /*level=*/0, - /*extra_files=*/std::vector>(), - /*creation_time=*/Timestamp(1721643142472ll, 0), /*delete_row_count=*/0, - /*embedded_index=*/nullptr, FileSource::Append(), - /*value_stats_cols=*/std::nullopt, - /*external_path=*/std::nullopt, /*first_row_id=*/std::nullopt, - /*write_cols=*/std::nullopt); + void SetUp() override { + pool_ = GetDefaultPool(); } - arrow::Result> PrepareData( - const arrow::FieldVector& fields) { - arrow::StringBuilder f0_builder; - arrow::Int32Builder f1_builder; - arrow::Int32Builder f2_builder; - arrow::DoubleBuilder f3_builder; - - std::vector> data = {{"Lily", 10, 0, 17.1}}; - - for (const auto& row : data) { - ARROW_RETURN_NOT_OK(f0_builder.Append(std::get<0>(row))); - ARROW_RETURN_NOT_OK(f1_builder.Append(std::get<1>(row))); - ARROW_RETURN_NOT_OK(f2_builder.Append(std::get<2>(row))); - ARROW_RETURN_NOT_OK(f3_builder.Append(std::get<3>(row))); - } - - std::shared_ptr f0_array, f1_array, f2_array, f3_array; - ARROW_RETURN_NOT_OK(f0_builder.Finish(&f0_array)); - ARROW_RETURN_NOT_OK(f1_builder.Finish(&f1_array)); - ARROW_RETURN_NOT_OK(f2_builder.Finish(&f2_array)); - ARROW_RETURN_NOT_OK(f3_builder.Finish(&f3_array)); - - std::vector> children = {f0_array, f1_array, f2_array, - f3_array}; - auto struct_type = arrow::struct_(fields); - return std::make_shared(struct_type, f0_array->length(), children); - } + private: + std::shared_ptr pool_; }; -TEST_F(CompactionTest, TestCompaction) { - std::string path = paimon::test::GetDataDir() + "/" + "orc" + "/append_09.db/append_09"; +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, CompactionInteTest, + ::testing::ValuesIn(GetTestValuesForCompactionInteTest())); + +TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompaction) { auto dir = UniqueTestDirectory::Create(); - std::string table_path = dir->Str(); - auto pool = GetDefaultPool(); - ASSERT_TRUE(TestUtil::CopyDirectory(path, table_path)); - WriteContextBuilder context_builder(table_path, "commit_user_1"); - - ASSERT_OK_AND_ASSIGN(std::unique_ptr write_context, - context_builder.WithStreamingMode(true).Finish()); - ASSERT_OK_AND_ASSIGN(std::unique_ptr write, - FileStoreWrite::Create(std::move(write_context))); - auto append_write = dynamic_cast(write.get()); - ASSERT_TRUE(append_write); - - BinaryRow partition(1); - BinaryRowWriter binary_row_writer(&partition, 0, pool.get()); - binary_row_writer.WriteInt(0, 10); - binary_row_writer.Complete(); - - { - arrow::FieldVector fields = { - arrow::field("f0", arrow::utf8()), arrow::field("f1", arrow::int32()), - arrow::field("f2", arrow::int32()), arrow::field("f3", arrow::float64())}; - auto struct_array = PrepareData(fields); - ASSERT_TRUE(struct_array.ok()); - ::ArrowArray arrow_array; - auto arrow_status = arrow::ExportArray(*struct_array.ValueUnsafe(), &arrow_array); - ASSERT_TRUE(arrow_status.ok()); - paimon::RecordBatchBuilder batch_builder(&arrow_array); - ASSERT_OK_AND_ASSIGN(std::unique_ptr record_batch, - batch_builder.SetPartition({{"f1", "10"}}).SetBucket(1).Finish()); - - ASSERT_OK(append_write->Write(std::move(record_batch))); - ASSERT_OK(append_write->Compact({{"f1", "10"}}, 1, /*full_compaction=*/true)); + 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 = {}; + std::vector partition_keys = {"f1"}; + auto file_format = GetParam(); + std::map options = { + {Options::FILE_FORMAT, file_format}, + {Options::BUCKET, "2"}, + {Options::BUCKET_KEY, "f2"}, + {Options::FILE_SYSTEM, "local"}, + }; + ASSERT_OK_AND_ASSIGN( + auto helper, TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, + /*is_streaming_mode=*/true)); + int64_t commit_identifier = 0; + ASSERT_OK_AND_ASSIGN(std::optional> table_schema, + helper->LatestSchema()); + ASSERT_TRUE(table_schema); + + DataGenerator gen(table_schema.value(), pool_); + 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()); + + 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(11, snapshot5.value().TotalRecordCount().value()); + ASSERT_EQ(0, 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, "Bob", 10, 0, 12.1], +[0, "Emily", 10, 0, 13.1], +[0, "Tony", 10, 0, 14.1], +[0, "Emily", 10, 0, 15.1], +[0, "Bob", 10, 0, 12.1], +[0, "Alex", 10, 0, 16.1], +[0, "David", 10, 0, 17.1], +[0, "Lily", 10, 0, 17.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); } - - ASSERT_OK_AND_ASSIGN(auto msgs, append_write->PrepareCommit(/*wait_compaction=*/true, 1)); - ASSERT_EQ(msgs.size(), 1); - auto msg_impl = dynamic_cast(msgs[0].get()); - auto compact_increment = msg_impl->GetCompactIncrement(); - ASSERT_EQ(compact_increment.CompactBefore().size(), 4); - ASSERT_EQ(compact_increment.CompactAfter().size(), 1); - std::cout << compact_increment.CompactAfter()[0]->file_name << std::endl; - CommitContextBuilder commit_context_builder(table_path, "commit_user_1"); - ASSERT_OK_AND_ASSIGN(std::unique_ptr commit_context, - commit_context_builder.IgnoreEmptyCommit(false).Finish()); - ASSERT_OK_AND_ASSIGN(std::unique_ptr commit, - FileStoreCommit::Create(std::move(commit_context))); - ASSERT_OK(commit->Commit(msgs, 2)); } } // namespace paimon::test From 9d403d937ec784a54f9a29c4fead024e318e2d01 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 6 Mar 2026 17:58:39 +0800 Subject: [PATCH 03/21] fix --- .../append_only_file_store_write.cpp | 4 + test/inte/compaction_inte_test.cpp | 295 ++++++++++++++---- 2 files changed, 242 insertions(+), 57 deletions(-) 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 9a711a9e..3d68fd0b 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -122,6 +122,9 @@ Result>> AppendOnlyFileStoreWrite::Com if (rewriter) { (void)rewriter->Close(); } + if (reader) { + reader->Close(); + } }); while (true) { @@ -145,6 +148,7 @@ Result>> AppendOnlyFileStoreWrite::Com } guard.Release(); PAIMON_RETURN_NOT_OK(rewriter->Close()); + reader->Close(); return rewriter->GetResult(); } diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index fb2ab336..94d8c3e0 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -44,13 +44,73 @@ class CompactionInteTest : public testing::Test, public ::testing::WithParamInte pool_ = GetDefaultPool(); } + void PrepareSimpleAppendData(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"); + values.emplace_back("parquet"); #ifdef PAIMON_ENABLE_ORC values.emplace_back("orc"); #endif @@ -66,7 +126,7 @@ std::vector GetTestValuesForCompactionInteTest() { INSTANTIATE_TEST_SUITE_P(FileFormat, CompactionInteTest, ::testing::ValuesIn(GetTestValuesForCompactionInteTest())); -TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompaction) { +TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompaction) { auto dir = UniqueTestDirectory::Create(); ASSERT_TRUE(dir); arrow::FieldVector fields = { @@ -86,72 +146,193 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompaction) { ASSERT_OK_AND_ASSIGN( auto helper, TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, /*is_streaming_mode=*/true)); - int64_t commit_identifier = 0; 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; + PrepareSimpleAppendData(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()); - DataGenerator gen(table_schema.value(), pool_); - 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(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( - 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()); + 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(11, snapshot5.value().TotalRecordCount().value()); + ASSERT_EQ(0, 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, "Bob", 10, 0, 12.1], +[0, "Emily", 10, 0, 13.1], +[0, "Tony", 10, 0, 14.1], +[0, "Emily", 10, 0, 15.1], +[0, "Bob", 10, 0, 12.1], +[0, "Alex", 10, 0, 16.1], +[0, "David", 10, 0, 17.1], +[0, "Lily", 10, 0, 17.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); + } +} + +TEST_P(CompactionInteTest, TestAppendTableStreamWriteBestEffortCompaction) { + 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 = {}; + std::vector partition_keys = {"f1"}; + auto file_format = GetParam(); + std::map options = { + {Options::FILE_FORMAT, file_format}, + {Options::BUCKET, "2"}, + {Options::BUCKET_KEY, "f2"}, + {Options::FILE_SYSTEM, "local"}, + {Options::COMPACTION_MIN_FILE_NUM, "3"}, + }; 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()); + 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; + PrepareSimpleAppendData(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=*/false)); 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()); + 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(11, snapshot5.value().TotalRecordCount().value()); + ASSERT_EQ(0, 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, "Bob", 10, 0, 12.1], +[0, "Emily", 10, 0, 13.1], +[0, "Tony", 10, 0, 14.1], +[0, "Emily", 10, 0, 15.1], +[0, "Bob", 10, 0, 12.1], +[0, "Alex", 10, 0, 16.1], +[0, "David", 10, 0, 17.1], +[0, "Lily", 10, 0, 17.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); + } +} + +TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + auto external_dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(external_dir); + std::string external_test_dir = "FILE://" + external_dir->Str(); + std::cout << dir->Str() << std::endl; + + 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 = {}; + std::vector partition_keys = {"f1"}; + auto file_format = GetParam(); + std::map options = { + {Options::FILE_FORMAT, file_format}, + {Options::BUCKET, "2"}, + {Options::BUCKET_KEY, "f2"}, + {Options::FILE_SYSTEM, "local"}, + {Options::DATA_FILE_EXTERNAL_PATHS, external_test_dir}, + {Options::DATA_FILE_EXTERNAL_PATHS_STRATEGY, "round-robin"}}; + 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; + PrepareSimpleAppendData(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_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]))); From 2329476fa52dd8c4cf2537bf38968e0de29cd40c Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 9 Mar 2026 10:33:40 +0800 Subject: [PATCH 04/21] fix --- .../operation/abstract_file_store_write.cpp | 4 - .../append_only_file_store_write_test.cpp | 2 +- test/inte/compaction_inte_test.cpp | 85 +++++++++++++++++++ 3 files changed, 86 insertions(+), 5 deletions(-) diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index f2d8ca4f..e9af4a70 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -269,10 +269,6 @@ Result> AbstractFileStoreWrite::ScanExistingFileMe /*vector_search=*/nullptr); PAIMON_ASSIGN_OR_RAISE(std::unique_ptr scan, CreateFileStoreScan(scan_filter)); - // PAIMON_ASSIGN_OR_RAISE(std::shared_ptr plan, - // scan->WithSnapshot(snapshot)->CreatePlan()); - // std::vector entries = plan->Files(); - // TODO(yonghao.fyh): create index file handler FileSystemWriteRestore restore(options_, snapshot_manager_, std::move(scan)); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, restore.GetRestoreFiles()); diff --git a/src/paimon/core/operation/append_only_file_store_write_test.cpp b/src/paimon/core/operation/append_only_file_store_write_test.cpp index 573f89e9..68df47c9 100644 --- a/src/paimon/core/operation/append_only_file_store_write_test.cpp +++ b/src/paimon/core/operation/append_only_file_store_write_test.cpp @@ -169,7 +169,7 @@ TEST_F(AppendOnlyFileStoreWriteTest, TestGetMaxSequenceNumberFromMultiPartition) ASSERT_OK_AND_ASSIGN(std::shared_ptr restore_files, write->ScanExistingFileMetas(latest_snapshot.value(), partition, /*bucket=*/1)); - ASSERT_EQ(-1, restore_files->TotalBuckets().value()); + ASSERT_EQ(std::nullopt, restore_files->TotalBuckets()); ASSERT_EQ(-1, DataFileMeta::GetMaxSequenceNumber(restore_files->DataFiles())); } } diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index 94d8c3e0..3ec6fd9a 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -388,4 +388,89 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) } } +TEST_F(CompactionInteTest, TestAppendTableWriteAlterTableWithCompaction) { + std::string test_data_path = + paimon::test::GetDataDir() + + "/orc/append_table_with_alter_table.db/append_table_with_alter_table/"; + auto dir = UniqueTestDirectory::Create(); + std::string table_path = dir->Str(); + std::cout << "[alter]" << table_path << std::endl; + ASSERT_TRUE(TestUtil::CopyDirectory(test_data_path, table_path)); + arrow::FieldVector fields = { + arrow::field("key0", arrow::int32()), arrow::field("key1", arrow::int32()), + arrow::field("k", arrow::int32()), arrow::field("c", arrow::int32()), + arrow::field("d", arrow::int32()), arrow::field("a", arrow::int32()), + arrow::field("e", arrow::int32()), + }; + std::map options = {{Options::FILE_FORMAT, "orc"}, + {Options::MANIFEST_FORMAT, "orc"}, + {Options::TARGET_FILE_SIZE, "1024"}, + {Options::FILE_SYSTEM, "local"}}; + ASSERT_OK_AND_ASSIGN(auto helper, + TestHelper::Create(table_path, options, /*is_streaming_mode=*/true)); + // scan with empty split + ASSERT_OK_AND_ASSIGN(std::vector> empty_splits, + helper->NewScan(StartupMode::Latest(), /*snapshot_id=*/std::nullopt)); + ASSERT_TRUE(empty_splits.empty()); + + int64_t commit_identifier = 0; + auto data_type = arrow::struct_(fields); + std::string data = R"([[1, 1, 116, 113, 567, 115, 668]])"; + ASSERT_OK_AND_ASSIGN( + std::unique_ptr batch, + TestHelper::MakeRecordBatch(data_type, data, {{"key0", "1"}, {"key1", "1"}}, /*bucket=*/0, + /*row_kinds=*/{})); + // for append only unaware bucket table, previous files will be ignored + auto file_meta = std::make_shared( + "data-xxx.xxx", /*file_size=*/543, /*row_count=*/1, + /*min_key=*/BinaryRow::EmptyRow(), /*max_key=*/BinaryRow::EmptyRow(), + /*key_stats=*/SimpleStats::EmptyStats(), + BinaryRowGenerator::GenerateStats({1, 1, 116, 113, 567, 115, 668}, + {1, 1, 116, 113, 567, 115, 668}, {0, 0, 0, 0, 0, 0, 0}, + pool_.get()), + /*min_sequence_number=*/0, /*max_sequence_number=*/0, /*schema_id=*/1, + /*level=*/0, /*extra_files=*/std::vector>(), + /*creation_time=*/Timestamp(1724090888706ll, 0), + /*delete_row_count=*/0, /*embedded_index=*/nullptr, FileSource::Append(), + /*value_stats_cols=*/std::nullopt, /*external_path=*/std::nullopt, + /*first_row_id=*/std::nullopt, + /*write_cols=*/std::nullopt); + DataIncrement data_increment({file_meta}, {}, {}); + std::shared_ptr expected_commit_message = std::make_shared( + BinaryRowGenerator::GenerateRow({1, 1}, pool_.get()), /*bucket=*/0, + /*total_bucket=*/-1, data_increment, CompactIncrement({}, {}, {})); + std::vector> expected_commit_messages = { + expected_commit_message}; + ASSERT_OK( + helper->WriteAndCommit(std::move(batch), commit_identifier++, expected_commit_messages)); + + ASSERT_OK(helper->write_->Compact(/*partition=*/{{"key0", "0"}, {"key1", "1"}}, /*bucket=*/0, + /*full_compaction=*/true)); + ASSERT_OK(helper->write_->Compact(/*partition=*/{{"key0", "1"}, {"key1", "1"}}, /*bucket=*/0, + /*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 snapshot, helper->LatestSnapshot()); + ASSERT_TRUE(snapshot); + + ASSERT_EQ(1, snapshot.value().SchemaId()); + ASSERT_EQ(4, snapshot.value().Id()); + + // read + ASSERT_OK_AND_ASSIGN(std::vector> data_splits, helper->Scan()); + ASSERT_EQ(data_splits.size(), 1); + + 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_with_row_kind = arrow::struct_(fields_with_row_kind); + std::string expected_data = R"([[0, 1, 1, 116, 113, 567, 115, 668]])"; + ASSERT_OK_AND_ASSIGN(bool success, helper->ReadAndCheckResult(data_type_with_row_kind, + data_splits, expected_data)); + ASSERT_TRUE(success); +} + } // namespace paimon::test From 0af04bff5b622defe0542eab7af862d2f29b2e79 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 9 Mar 2026 14:00:24 +0800 Subject: [PATCH 05/21] fix --- .../bucketed_append_compact_manager_test.cpp | 122 +++++++++++++++++- 1 file changed, 121 insertions(+), 1 deletion(-) 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 dd6a523c..4f58f4d6 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager_test.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager_test.cpp @@ -25,13 +25,16 @@ #include "paimon/core/io/data_file_meta.h" #include "paimon/core/manifest/file_source.h" #include "paimon/core/stats/simple_stats.h" +#include "paimon/executor.h" #include "paimon/result.h" namespace paimon::test { class BucketedAppendCompactManagerTest : public testing::Test { public: - void SetUp() override {} + void SetUp() override { + executor_ = CreateDefaultExecutor(); + } std::vector> GenerateDataFileMeta() { std::vector> metas; @@ -57,6 +60,61 @@ class BucketedAppendCompactManagerTest : public testing::Test { } private: + void InnerTest(const std::vector>& to_compact_before_pick, + bool expected_present, + const std::vector>& expected_compact_before, + const std::vector>& to_compact_after_pick) { + int min_file_num = 4; + int64_t target_file_size = 1024; + int64_t threshold = target_file_size / 10 * 7; + BucketedAppendCompactManager manager( + executor_, to_compact_before_pick, + /*dv_maintainer=*/nullptr, min_file_num, target_file_size, threshold, + /*force_rewrite_all_files=*/false, /*compact_rewriter=*/nullptr); + auto actual = manager.PickCompactBefore(); + if (expected_present) { + ASSERT_TRUE(actual.has_value()); + ExpectVectorsEqual(actual.value(), expected_compact_before); + } else { + ASSERT_FALSE(actual.has_value()); + } + auto pq = manager.GetToCompact(); + std::vector> to_compact; + while (!pq.empty()) { + to_compact.push_back(pq.top()); + pq.pop(); + } + ExpectVectorsEqual(to_compact, to_compact_after_pick); + } + + std::shared_ptr NewFile(int64_t min_sequence_number, + int64_t max_sequence_number) { + return std::make_shared( + /*file_name=*/"", /*file_size=*/max_sequence_number - min_sequence_number + 1, + /*row_count=*/0, + /*min_key=*/BinaryRow::EmptyRow(), /*max_key=*/BinaryRow::EmptyRow(), + /*key_stats=*/SimpleStats::EmptyStats(), + /*value_stats=*/SimpleStats::EmptyStats(), min_sequence_number, max_sequence_number, + /*schema_id=*/0, + /*level=*/DataFileMeta::DUMMY_LEVEL, + /*extra_files=*/std::vector>(), + /*creation_time=*/Timestamp(1724090888706ll, 0), + /*delete_row_count=*/max_sequence_number - min_sequence_number + 1, + /*embedded_index=*/nullptr, FileSource::Append(), + /*value_stats_cols=*/std::nullopt, /*external_path=*/std::nullopt, + /*first_row_id=*/std::nullopt, + /*write_cols=*/std::nullopt); + } + + void ExpectVectorsEqual(const std::vector>& actual, + const std::vector>& expected) { + EXPECT_EQ(actual.size(), expected.size()); + for (size_t i = 0; i < actual.size(); ++i) { + EXPECT_EQ(*actual[i], *expected[i]); + } + } + + std::shared_ptr executor_; }; TEST_F(BucketedAppendCompactManagerTest, TestFileComparatorWithoutOverlap) { @@ -96,4 +154,66 @@ TEST_F(BucketedAppendCompactManagerTest, TestIsOverlap) { EXPECT_FALSE(BucketedAppendCompactManager::IsOverlap(file2, file3)); } +TEST_F(BucketedAppendCompactManagerTest, TestPickEmptyAndNotRelease) { + // 1~50 is small enough, so hold it + std::vector> to_compact = {NewFile(1, 50)}; + InnerTest(to_compact, /*expected_present=*/false, /*expected_compact_before=*/{}, to_compact); +} + +TEST_F(BucketedAppendCompactManagerTest, TestPickEmptyAndRelease) { + // large file, release + InnerTest(/*to_compact_before_pick=*/{NewFile(1, 2048)}, /*expected_present=*/false, + /*expected_compact_before=*/{}, /*to_compact_after_pick=*/{}); + + // small file at last, release previous + InnerTest(/*to_compact_before_pick=*/{NewFile(1, 2048), NewFile(2049, 2100)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, /*to_compact_after_pick=*/{NewFile(2049, 2100)}); + InnerTest( + /*to_compact_before_pick=*/{NewFile(1, 2048), NewFile(2049, 2100), NewFile(2101, 2110)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, + /*to_compact_after_pick=*/{NewFile(2049, 2100), NewFile(2101, 2110)}); + InnerTest( + /*to_compact_before_pick=*/{NewFile(1, 2048), NewFile(2049, 4096), NewFile(4097, 5000)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, /*to_compact_after_pick=*/{NewFile(4097, 5000)}); + InnerTest( + /*to_compact_before_pick=*/{NewFile(1, 1024), NewFile(1025, 2049), NewFile(2050, 2500), + NewFile(2501, 4096), NewFile(4097, 6000), NewFile(6001, 7000), + NewFile(7001, 7600)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, + /*to_compact_after_pick=*/{NewFile(6001, 7000), NewFile(7001, 7600)}); + + // ignore single small file (in the middle) + InnerTest( + /*to_compact_before_pick=*/{NewFile(1, 2048), NewFile(2049, 4096), NewFile(4097, 4100), + NewFile(4101, 6150)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, + /*to_compact_after_pick=*/{NewFile(4101, 6150)}); + InnerTest( + /*to_compact_before_pick=*/{NewFile(1, 2048), NewFile(2049, 4096), NewFile(4097, 5000), + NewFile(5001, 6144), NewFile(6145, 7048)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, /*to_compact_after_pick=*/{NewFile(6145, 7048)}); + + // wait for more file + InnerTest(/*to_compact_before_pick=*/{NewFile(1, 500), NewFile(501, 1000)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, + /*to_compact_after_pick=*/{NewFile(1, 500), NewFile(501, 1000)}); + + InnerTest(/*to_compact_before_pick=*/{NewFile(1, 500), NewFile(501, 1000), NewFile(1001, 2048)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, + /*to_compact_after_pick=*/{NewFile(501, 1000), NewFile(1001, 2048)}); + InnerTest( + /*to_compact_before_pick=*/{NewFile(1, 2050), NewFile(2051, 2100), NewFile(2101, 2110)}, + /*expected_present=*/false, + /*expected_compact_before=*/{}, + /*to_compact_after_pick=*/{NewFile(2051, 2100), NewFile(2101, 2110)}); +} + } // namespace paimon::test From 6034e02abf2edcb25b859f87aacf91e4612c9689 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 9 Mar 2026 14:38:41 +0800 Subject: [PATCH 06/21] fix --- src/paimon/core/operation/abstract_file_store_write.cpp | 4 ++-- src/paimon/core/operation/abstract_file_store_write.h | 3 +-- .../core/operation/append_only_file_store_write.cpp | 6 ++---- .../core/operation/append_only_file_store_write_test.cpp | 8 +++----- src/paimon/core/operation/file_system_write_restore.h | 3 +-- src/paimon/core/operation/key_value_file_store_write.cpp | 6 ++---- 6 files changed, 11 insertions(+), 19 deletions(-) diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index e9af4a70..72aac8fe 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -253,7 +253,7 @@ int32_t AbstractFileStoreWrite::GetDefaultBucketNum() const { } Result> AbstractFileStoreWrite::ScanExistingFileMetas( - const Snapshot& snapshot, const BinaryRow& partition, int32_t bucket) const { + const BinaryRow& partition, int32_t bucket) const { PAIMON_ASSIGN_OR_RAISE(auto part_values, file_store_path_factory_->GeneratePartitionVector(partition)); std::map part_values_map; @@ -270,7 +270,7 @@ Result> AbstractFileStoreWrite::ScanExistingFileMe PAIMON_ASSIGN_OR_RAISE(std::unique_ptr scan, CreateFileStoreScan(scan_filter)); // TODO(yonghao.fyh): create index file handler - FileSystemWriteRestore restore(options_, snapshot_manager_, std::move(scan)); + FileSystemWriteRestore restore(snapshot_manager_, std::move(scan)); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, restore.GetRestoreFiles()); std::optional restored_total_buckets = restore_files->TotalBuckets(); diff --git a/src/paimon/core/operation/abstract_file_store_write.h b/src/paimon/core/operation/abstract_file_store_write.h index d76b82c6..da11843f 100644 --- a/src/paimon/core/operation/abstract_file_store_write.h +++ b/src/paimon/core/operation/abstract_file_store_write.h @@ -108,8 +108,7 @@ class AbstractFileStoreWrite : public FileStoreWrite { virtual Result> CreateFileStoreScan( const std::shared_ptr& filter) const = 0; - Result> ScanExistingFileMetas(const Snapshot& snapshot, - const BinaryRow& partition, + Result> ScanExistingFileMetas(const BinaryRow& partition, int32_t bucket) const; int32_t GetDefaultBucketNum() 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 3d68fd0b..4873462c 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -156,13 +156,11 @@ Result>> AppendOnlyFileStoreWrit const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) { PAIMON_LOG_DEBUG(logger_, "Creating append only writer for partition %s, bucket %d", partition.ToString().c_str(), bucket); - PAIMON_ASSIGN_OR_RAISE(std::optional latest_snapshot, - snapshot_manager_->LatestSnapshot()); int32_t total_buckets = GetDefaultBucketNum(); std::vector> restore_data_files; - if (!ignore_previous_files && latest_snapshot != std::nullopt) { + if (!ignore_previous_files) { PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, - ScanExistingFileMetas(latest_snapshot.value(), partition, bucket)); + ScanExistingFileMetas(partition, bucket)); restore_data_files = restore_files->DataFiles(); if (restore_files->TotalBuckets()) { total_buckets = restore_files->TotalBuckets().value(); diff --git a/src/paimon/core/operation/append_only_file_store_write_test.cpp b/src/paimon/core/operation/append_only_file_store_write_test.cpp index 68df47c9..539de36f 100644 --- a/src/paimon/core/operation/append_only_file_store_write_test.cpp +++ b/src/paimon/core/operation/append_only_file_store_write_test.cpp @@ -136,8 +136,6 @@ TEST_F(AppendOnlyFileStoreWriteTest, TestGetMaxSequenceNumberFromMultiPartition) builder.AddOption("file.format", "orc").AddOption("manifest.format", "orc").Finish()); ASSERT_OK_AND_ASSIGN(auto file_store_write, FileStoreWrite::Create(std::move(write_context))); auto write = dynamic_cast(file_store_write.get()); - ASSERT_OK_AND_ASSIGN(std::optional latest_snapshot, - write->snapshot_manager_->LatestSnapshot()); auto pool = GetDefaultPool(); { BinaryRow partition(2); @@ -145,7 +143,7 @@ TEST_F(AppendOnlyFileStoreWriteTest, TestGetMaxSequenceNumberFromMultiPartition) writer.WriteInt(0, 20); writer.WriteInt(1, 1); ASSERT_OK_AND_ASSIGN(std::shared_ptr restore_files, - write->ScanExistingFileMetas(latest_snapshot.value(), partition, + write->ScanExistingFileMetas(partition, /*bucket=*/0)); ASSERT_EQ(-1, restore_files->TotalBuckets().value()); ASSERT_EQ(0, DataFileMeta::GetMaxSequenceNumber(restore_files->DataFiles())); @@ -156,7 +154,7 @@ TEST_F(AppendOnlyFileStoreWriteTest, TestGetMaxSequenceNumberFromMultiPartition) writer.WriteInt(0, 10); writer.WriteInt(1, 0); ASSERT_OK_AND_ASSIGN(std::shared_ptr restore_files, - write->ScanExistingFileMetas(latest_snapshot.value(), partition, + write->ScanExistingFileMetas(partition, /*bucket=*/0)); ASSERT_EQ(-1, restore_files->TotalBuckets().value()); ASSERT_EQ(2, DataFileMeta::GetMaxSequenceNumber(restore_files->DataFiles())); @@ -167,7 +165,7 @@ TEST_F(AppendOnlyFileStoreWriteTest, TestGetMaxSequenceNumberFromMultiPartition) writer.WriteInt(0, 10); writer.WriteInt(1, 0); ASSERT_OK_AND_ASSIGN(std::shared_ptr restore_files, - write->ScanExistingFileMetas(latest_snapshot.value(), partition, + write->ScanExistingFileMetas(partition, /*bucket=*/1)); ASSERT_EQ(std::nullopt, restore_files->TotalBuckets()); ASSERT_EQ(-1, DataFileMeta::GetMaxSequenceNumber(restore_files->DataFiles())); diff --git a/src/paimon/core/operation/file_system_write_restore.h b/src/paimon/core/operation/file_system_write_restore.h index 449d2743..815da118 100644 --- a/src/paimon/core/operation/file_system_write_restore.h +++ b/src/paimon/core/operation/file_system_write_restore.h @@ -27,8 +27,7 @@ namespace paimon { /// `WriteRestore` to restore files directly from file system. class FileSystemWriteRestore : public WriteRestore { public: - FileSystemWriteRestore(const CoreOptions& options, - const std::shared_ptr& snapshot_manager, + FileSystemWriteRestore(const std::shared_ptr& snapshot_manager, std::unique_ptr&& scan) : snapshot_manager_(snapshot_manager), scan_(std::move(scan)) { // TODO(yonghao.fyh): support index file handler 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 5771a585..e07d1188 100644 --- a/src/paimon/core/operation/key_value_file_store_write.cpp +++ b/src/paimon/core/operation/key_value_file_store_write.cpp @@ -91,13 +91,11 @@ Result>> KeyValueFileStoreWrite: const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) { PAIMON_LOG_DEBUG(logger_, "Creating key value writer for partition %s, bucket %d", partition.ToString().c_str(), bucket); - PAIMON_ASSIGN_OR_RAISE(std::optional latest_snapshot, - snapshot_manager_->LatestSnapshot()); int32_t total_buckets = GetDefaultBucketNum(); std::vector> restore_data_files; - if (!ignore_previous_files && latest_snapshot != std::nullopt) { + if (!ignore_previous_files) { PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, - ScanExistingFileMetas(latest_snapshot.value(), partition, bucket)); + ScanExistingFileMetas(partition, bucket)); restore_data_files = restore_files->DataFiles(); if (restore_files->TotalBuckets()) { total_buckets = restore_files->TotalBuckets().value(); From f8ee3b6b1b6e29181c47e3c27e97b374f9e30374 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 9 Mar 2026 14:40:18 +0800 Subject: [PATCH 07/21] fix --- src/paimon/core/append/append_only_writer.cpp | 2 +- src/paimon/core/append/bucketed_append_compact_manager_test.cpp | 2 +- src/paimon/core/compact/noop_compact_manager.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/paimon/core/append/append_only_writer.cpp b/src/paimon/core/append/append_only_writer.cpp index 936a37fc..4902cc6e 100644 --- a/src/paimon/core/append/append_only_writer.cpp +++ b/src/paimon/core/append/append_only_writer.cpp @@ -86,7 +86,7 @@ Status AppendOnlyWriter::Write(std::unique_ptr&& batch) { Result AppendOnlyWriter::PrepareCommit(bool wait_compaction) { PAIMON_RETURN_NOT_OK( - Flush(/*wait_for_latest_compaction=*/false, /*force_full_compaction=*/false)); + Flush(/*wait_for_latest_compaction=*/false, /*forced_full_compaction=*/false)); PAIMON_RETURN_NOT_OK(TrySyncLatestCompaction(wait_compaction || options_.CommitForceCompact())); return DrainIncrement(); } 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 4f58f4d6..2418e87e 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager_test.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager_test.cpp @@ -70,7 +70,7 @@ class BucketedAppendCompactManagerTest : public testing::Test { BucketedAppendCompactManager manager( executor_, to_compact_before_pick, /*dv_maintainer=*/nullptr, min_file_num, target_file_size, threshold, - /*force_rewrite_all_files=*/false, /*compact_rewriter=*/nullptr); + /*force_rewrite_all_files=*/false, /*rewriter=*/nullptr); auto actual = manager.PickCompactBefore(); if (expected_present) { ASSERT_TRUE(actual.has_value()); diff --git a/src/paimon/core/compact/noop_compact_manager.h b/src/paimon/core/compact/noop_compact_manager.h index 337abb42..46819128 100644 --- a/src/paimon/core/compact/noop_compact_manager.h +++ b/src/paimon/core/compact/noop_compact_manager.h @@ -30,7 +30,7 @@ namespace paimon { class NoopCompactManager : public CompactManager { public: NoopCompactManager() = default; - ~NoopCompactManager() override {} + ~NoopCompactManager() override = default; void AddNewFile(const std::shared_ptr& file) override {} From ab54e6f1f267248d724de011d9b5774ef0a93111 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 10 Mar 2026 08:57:23 +0800 Subject: [PATCH 08/21] fix --- include/paimon/metrics.h | 15 + src/paimon/common/metrics/metrics_impl.cpp | 26 ++ src/paimon/common/metrics/metrics_impl.h | 7 + .../bucketed_append_compact_manager.cpp | 12 +- .../append/bucketed_append_compact_manager.h | 19 +- .../bucketed_append_compact_manager_test.cpp | 2 +- src/paimon/core/compact/compact_task.h | 44 ++- .../operation/abstract_file_store_write.cpp | 5 +- .../operation/abstract_file_store_write.h | 2 + .../append_only_file_store_write.cpp | 3 +- .../operation/metrics/compaction_metrics.h | 281 ++++++++++++++++++ 11 files changed, 400 insertions(+), 16 deletions(-) create mode 100644 src/paimon/core/operation/metrics/compaction_metrics.h diff --git a/include/paimon/metrics.h b/include/paimon/metrics.h index 42352701..3bde1c07 100644 --- a/include/paimon/metrics.h +++ b/include/paimon/metrics.h @@ -77,6 +77,21 @@ class PAIMON_EXPORT Metrics { /// Get all histogram statistics snapshots. virtual std::map GetAllHistogramStats() const = 0; + /// Set the value of a specific gauge metric (current state metric). + /// @param metric_name The name/key of the gauge metric to set. + /// @param metric_value The value to set for this gauge metric (double, not just integer). + virtual void SetGauge(const std::string& metric_name, double metric_value) = 0; + + /// Get the current value of a specific gauge metric (current state metric). + /// @param metric_name The name/key of the gauge metric to retrieve. + /// @return The current value of the gauge metric, or `Status::KeyError` if the metric doesn't + /// exist. + virtual Result GetGauge(const std::string& metric_name) const = 0; + + /// Get all gauge metrics as a map. + /// @return A map containing all gauge metric names and their current values. + virtual std::map GetAllGauges() const = 0; + /// Merge metrics from another Metrics instance into this one. /// /// For metrics that exist in both instances, the values are added together. diff --git a/src/paimon/common/metrics/metrics_impl.cpp b/src/paimon/common/metrics/metrics_impl.cpp index db6382aa..afa3c9c0 100644 --- a/src/paimon/common/metrics/metrics_impl.cpp +++ b/src/paimon/common/metrics/metrics_impl.cpp @@ -92,6 +92,25 @@ std::map MetricsImpl::GetAllHistogramStats() const return res; } +void MetricsImpl::SetGauge(const std::string& metric_name, double value) { + std::lock_guard lock(gauge_lock_); + gauges_[metric_name] = value; +} + +Result MetricsImpl::GetGauge(const std::string& metric_name) const { + std::lock_guard lock(gauge_lock_); + auto it = gauges_.find(metric_name); + if (it != gauges_.end()) { + return it->second; + } + return Status::KeyError(fmt::format("metric '{}' not found", metric_name)); +} + +std::map MetricsImpl::GetAllGauges() const { + std::lock_guard lock(gauge_lock_); + return gauges_; +} + void MetricsImpl::Merge(const std::shared_ptr& other) { if (other && this != other.get()) { { @@ -223,6 +242,13 @@ std::string MetricsImpl::ToString() const { stddev_val.SetDouble(s.stddev); doc.AddMember(rapidjson::Value(name + ".stddev", allocator), stddev_val, allocator); } + + std::map gauges = GetAllGauges(); + for (const auto& kv : gauges) { + doc.AddMember(rapidjson::Value(kv.first, allocator), rapidjson::Value(kv.second), + allocator); + } + rapidjson::StringBuffer s; RapidWriter writer(s); doc.Accept(writer); diff --git a/src/paimon/common/metrics/metrics_impl.h b/src/paimon/common/metrics/metrics_impl.h index 023b3508..8d4db44b 100644 --- a/src/paimon/common/metrics/metrics_impl.h +++ b/src/paimon/common/metrics/metrics_impl.h @@ -41,6 +41,10 @@ class PAIMON_EXPORT MetricsImpl : public Metrics { Result GetHistogramStats(const std::string& metric_name) const override; std::map GetAllHistogramStats() const override; + void SetGauge(const std::string& metric_name, double metric_value) override; + Result GetGauge(const std::string& metric_name) const override; + std::map GetAllGauges() const override; + void Merge(const std::shared_ptr& other) override; std::string ToString() const override; void Overwrite(const std::shared_ptr& metrics); @@ -66,6 +70,9 @@ class PAIMON_EXPORT MetricsImpl : public Metrics { mutable std::mutex histogram_lock_; std::map> histograms_; + + mutable std::mutex gauge_lock_; + std::map gauges_; }; } // namespace paimon diff --git a/src/paimon/core/append/bucketed_append_compact_manager.cpp b/src/paimon/core/append/bucketed_append_compact_manager.cpp index 6d25783a..7379d05c 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager.cpp @@ -25,7 +25,7 @@ BucketedAppendCompactManager::BucketedAppendCompactManager( 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) + CompactRewriter rewriter, const std::shared_ptr& reporter) : executor_(executor), dv_maintainer_(dv_maintainer), min_file_num_(min_file_num), @@ -33,6 +33,7 @@ BucketedAppendCompactManager::BucketedAppendCompactManager( compaction_file_size_(compaction_file_size), force_rewrite_all_files_(force_rewrite_all_files), rewriter_(rewriter), + reporter_(reporter), to_compact_( [](const std::shared_ptr& lhs, const std::shared_ptr& rhs) { return lhs->min_sequence_number > rhs->min_sequence_number; @@ -70,8 +71,9 @@ Status BucketedAppendCompactManager::TriggerFullCompaction() { compacting.push_back(to_compact_.top()); to_compact_.pop(); } - auto compact_task = std::make_shared( - dv_maintainer_, compacting, compaction_file_size_, force_rewrite_all_files_, rewriter_); + auto compact_task = std::make_shared(reporter_, dv_maintainer_, compacting, + compaction_file_size_, + force_rewrite_all_files_, rewriter_); task_future_ = Via(executor_.get(), [compact_task]() -> Result> { return compact_task->Execute(); }); @@ -86,8 +88,8 @@ Status BucketedAppendCompactManager::TriggerCompactionWithBestEffort() { std::optional>> picked = PickCompactBefore(); if (picked) { compacting_ = picked.value(); - auto compact_task = - std::make_shared(dv_maintainer_, compacting_.value(), rewriter_); + auto compact_task = std::make_shared(reporter_, dv_maintainer_, + compacting_.value(), rewriter_); task_future_ = Via(executor_.get(), [compact_task]() -> Result> { return compact_task->Execute(); diff --git a/src/paimon/core/append/bucketed_append_compact_manager.h b/src/paimon/core/append/bucketed_append_compact_manager.h index 1b64ced1..7c09b745 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.h +++ b/src/paimon/core/append/bucketed_append_compact_manager.h @@ -29,6 +29,7 @@ #include "paimon/core/compact/compact_task.h" #include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" #include "paimon/core/io/data_file_meta.h" +#include "paimon/core/operation/metrics/compaction_metrics.h" #include "paimon/executor.h" #include "paimon/logging.h" #include "paimon/result.h" @@ -70,7 +71,8 @@ class BucketedAppendCompactManager : public CompactFutureManager { 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); + CompactRewriter rewriter, + const std::shared_ptr& reporter); ~BucketedAppendCompactManager() override = default; Status TriggerCompaction(bool full_compaction) override; @@ -126,11 +128,13 @@ class BucketedAppendCompactManager : public CompactFutureManager { /// A `CompactTask` impl for full compaction of append-only table. class FullCompactTask : public CompactTask { public: - FullCompactTask(const std::shared_ptr& dv_maintainer, + FullCompactTask(const std::shared_ptr& reporter, + const std::shared_ptr& dv_maintainer, const std::vector>& inputs, int64_t compaction_file_size, bool force_rewrite_all_files, CompactRewriter rewriter) - : dv_maintainer_(dv_maintainer), + : CompactTask(reporter), + dv_maintainer_(dv_maintainer), to_compact_(inputs.begin(), inputs.end()), compaction_file_size_(compaction_file_size), force_rewrite_all_files_(force_rewrite_all_files), @@ -163,10 +167,14 @@ class BucketedAppendCompactManager : public CompactFutureManager { /// compaction. class AutoCompactTask : public CompactTask { public: - AutoCompactTask(const std::shared_ptr& dv_maintainer, + AutoCompactTask(const std::shared_ptr& reporter, + const std::shared_ptr& dv_maintainer, const std::vector>& to_compact, CompactRewriter rewriter) - : dv_maintainer_(dv_maintainer), to_compact_(to_compact), rewriter_(rewriter) {} + : CompactTask(reporter), + dv_maintainer_(dv_maintainer), + to_compact_(to_compact), + rewriter_(rewriter) {} protected: Result> DoCompact() override { @@ -194,6 +202,7 @@ class BucketedAppendCompactManager : public CompactFutureManager { int64_t compaction_file_size_; bool force_rewrite_all_files_; CompactRewriter rewriter_; + std::shared_ptr reporter_; std::optional>> compacting_; DataFileMetaPriorityQueue to_compact_; 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 2418e87e..fdc1a78b 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager_test.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager_test.cpp @@ -70,7 +70,7 @@ class BucketedAppendCompactManagerTest : public testing::Test { BucketedAppendCompactManager manager( executor_, to_compact_before_pick, /*dv_maintainer=*/nullptr, min_file_num, target_file_size, threshold, - /*force_rewrite_all_files=*/false, /*rewriter=*/nullptr); + /*force_rewrite_all_files=*/false, /*rewriter=*/nullptr, /*reporter=*/nullptr); auto actual = manager.PickCompactBefore(); if (expected_present) { ASSERT_TRUE(actual.has_value()); diff --git a/src/paimon/core/compact/compact_task.h b/src/paimon/core/compact/compact_task.h index b82c8427..ddf08aa9 100644 --- a/src/paimon/core/compact/compact_task.h +++ b/src/paimon/core/compact/compact_task.h @@ -16,22 +16,60 @@ #pragma once +#include +#include +#include + #include "paimon/core/compact/compact_result.h" +#include "paimon/core/io/data_file_meta.h" +#include "paimon/core/operation/metrics/compaction_metrics.h" +#include "paimon/core/utils/duration.h" +#include "paimon/logging.h" #include "paimon/result.h" namespace paimon { -/// Compact task. +/// Base Compact task for metrics. class CompactTask { public: + explicit CompactTask(const std::shared_ptr& reporter) + : reporter_(reporter), logger_(Logger::GetLogger("CompactTask")) {} + virtual ~CompactTask() = default; - // TODO(yonghao.fyh): support metrics + Result> Execute() { - return DoCompact(); + Duration duration; + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr result, DoCompact()); + reporter_->ReportCompactionTime(static_cast(duration.Get())); + reporter_->IncreaseCompactionsCompletedCount(); + reporter_->ReportCompactionInputSize(CollectRewriteSize(result->Before())); + reporter_->ReportCompactionOutputSize(CollectRewriteSize(result->After())); + PAIMON_LOG_DEBUG( + logger_, + "Done compacting %zu files to %zu files in %lldms. Rewrite input file size " + "= %lld, output file size = %lld", + result->Before().size(), result->After().size(), static_cast(duration.Get()), + CollectRewriteSize(result->Before()), CollectRewriteSize(result->After())); + return result; } protected: + /// Perform compaction. + /// + /// @return `CompactResult` of compact before and compact after files. virtual Result> DoCompact() = 0; + + private: + static int64_t CollectRewriteSize(const std::vector>& files) { + int64_t size = 0; + for (const auto& file : files) { + size += file->file_size; + } + return size; + } + + std::shared_ptr reporter_; + std::unique_ptr logger_; }; } // namespace paimon diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index 72aac8fe..d64ab330 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -27,6 +27,7 @@ #include "paimon/core/manifest/manifest_entry.h" #include "paimon/core/operation/file_store_scan.h" #include "paimon/core/operation/file_system_write_restore.h" +#include "paimon/core/operation/metrics/compaction_metrics.h" #include "paimon/core/operation/restore_files.h" #include "paimon/core/schema/table_schema.h" #include "paimon/core/snapshot.h" @@ -77,6 +78,7 @@ AbstractFileStoreWrite::AbstractFileStoreWrite( logger_(Logger::GetLogger("AbstractFileStoreWrite")) { // TODO(yonghao.fyh): support with compact_executor_ = CreateDefaultExecutor(4); + compaction_metrics_ = std::make_shared(); } Status AbstractFileStoreWrite::Write(std::unique_ptr&& batch) { @@ -172,7 +174,7 @@ Result>> AbstractFileStoreWrite::Prep } std::vector> result; - auto metrics = std::make_shared(); + auto metrics = compaction_metrics_->GetMetrics(); for (auto partition_iter = writers_.begin(); partition_iter != writers_.end();) { auto& partition = partition_iter->first; auto& buckets = partition_iter->second; @@ -229,6 +231,7 @@ Result>> AbstractFileStoreWrite::Prep ++partition_iter; } } + metrics_->Overwrite(metrics); return result; } diff --git a/src/paimon/core/operation/abstract_file_store_write.h b/src/paimon/core/operation/abstract_file_store_write.h index da11843f..8f965d18 100644 --- a/src/paimon/core/operation/abstract_file_store_write.h +++ b/src/paimon/core/operation/abstract_file_store_write.h @@ -45,6 +45,7 @@ namespace paimon { struct DataFileMeta; class BatchWriter; +class CompactionMetrics; class FileStoreScan; class FileStorePathFactory; class ScanFilter; @@ -125,6 +126,7 @@ class AbstractFileStoreWrite : public FileStoreWrite { std::shared_ptr partition_schema_; CoreOptions options_; std::shared_ptr compact_executor_; + std::shared_ptr compaction_metrics_; private: Result> GetWriter(const BinaryRow& partition, int32_t bucket); 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 4873462c..668c3bbf 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -187,7 +187,8 @@ Result>> AppendOnlyFileStoreWrit options_.GetCompactionMinFileNum(), options_.GetTargetFileSize(/*has_primary_key=*/false), options_.GetCompactionFileSize(/*has_primary_key=*/false), - options_.CompactionForceRewriteAllFiles(), rewriter); + options_.CompactionForceRewriteAllFiles(), rewriter, + compaction_metrics_->CreateReporter(partition, bucket)); } auto writer = std::make_shared( diff --git a/src/paimon/core/operation/metrics/compaction_metrics.h b/src/paimon/core/operation/metrics/compaction_metrics.h new file mode 100644 index 00000000..a21efe0a --- /dev/null +++ b/src/paimon/core/operation/metrics/compaction_metrics.h @@ -0,0 +1,281 @@ +/* + * 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/common/data/binary_row.h" +#include "paimon/common/metrics/metrics_impl.h" + +namespace paimon { +/// Metrics to measure a compaction. +class CompactionMetrics { + public: + static constexpr int32_t kCompactionTimeWindow = 100; + + static constexpr char MAX_LEVEL0_FILE_COUNT[] = "maxLevel0FileCount"; + static constexpr char AVG_LEVEL0_FILE_COUNT[] = "avgLevel0FileCount"; + static constexpr char AVG_COMPACTION_TIME[] = "avgCompactionTime"; + static constexpr char COMPACTION_COMPLETED_COUNT[] = "compactionCompletedCount"; + static constexpr char COMPACTION_TOTAL_COUNT[] = "compactionTotalCount"; + static constexpr char COMPACTION_QUEUED_COUNT[] = "compactionQueuedCount"; + static constexpr char MAX_COMPACTION_INPUT_SIZE[] = "maxCompactionInputSize"; + static constexpr char MAX_COMPACTION_OUTPUT_SIZE[] = "maxCompactionOutputSize"; + static constexpr char AVG_COMPACTION_INPUT_SIZE[] = "avgCompactionInputSize"; + static constexpr char AVG_COMPACTION_OUTPUT_SIZE[] = "avgCompactionOutputSize"; + static constexpr char MAX_TOTAL_FILE_SIZE[] = "maxTotalFileSize"; + static constexpr char AVG_TOTAL_FILE_SIZE[] = "avgTotalFileSize"; + + class Reporter { + public: + Reporter(CompactionMetrics* metrics, const BinaryRow& partition, int32_t bucket) + : metrics_(metrics), partition_(partition), bucket_(bucket) {} + + void ReportLevel0FileCount(int64_t count) { + level0_file_count_ = count; + } + void ReportCompactionInputSize(int64_t bytes) { + compaction_input_size_ = bytes; + } + void ReportCompactionOutputSize(int64_t bytes) { + compaction_output_size_ = bytes; + } + void ReportTotalFileSize(int64_t bytes) { + total_file_size_ = bytes; + } + void ReportCompactionTime(int64_t time) { + metrics_->ReportCompactionTime(time); + } + + void IncreaseCompactionsCompletedCount() { + metrics_->IncreaseCompactionsCompletedCount(); + } + void IncreaseCompactionsTotalCount() { + metrics_->IncreaseCompactionsTotalCount(); + } + void IncreaseCompactionsQueuedCount() { + metrics_->IncreaseCompactionsQueuedCount(); + } + void DecreaseCompactionsQueuedCount() { + metrics_->DecreaseCompactionsQueuedCount(); + } + void Unregister() { + metrics_->EraseReporter(partition_, bucket_); + } + + int64_t Level0FileCount() const { + return level0_file_count_; + } + + int64_t CompactionInputSize() const { + return compaction_input_size_; + } + + int64_t CompactionOutputSize() const { + return compaction_output_size_; + } + + int64_t TotalFileSize() const { + return total_file_size_; + } + + private: + CompactionMetrics* metrics_; + BinaryRow partition_; + int32_t bucket_; + + // Data fields for metrics. + int64_t level0_file_count_ = 0; + int64_t compaction_input_size_ = 0; + int64_t compaction_output_size_ = 0; + int64_t total_file_size_ = 0; + }; + + std::shared_ptr CreateReporter(const BinaryRow& partition, int32_t bucket) { + std::lock_guard lock(reporter_mutex_); + std::pair key(partition, bucket); + auto reporter = std::make_shared(this, partition, bucket); + reporters_[key] = reporter; + return reporter; + } + + void EraseReporter(const BinaryRow& partition, int32_t bucket) { + std::lock_guard lock(reporter_mutex_); + reporters_.erase(std::pair{partition, bucket}); + } + + void ReportCompactionTime(int64_t time) { + std::lock_guard lock(compaction_times_mutex_); + compaction_times_.push_back(time); + if (compaction_times_.size() > kCompactionTimeWindow) { + compaction_times_.erase(compaction_times_.begin()); + } + } + + double MaxLevel0FileCount() { + std::lock_guard lock(reporter_mutex_); + int64_t max_val = -1; + for (const auto& [_, reporter] : reporters_) { + max_val = std::max(max_val, reporter->Level0FileCount()); + } + return static_cast(max_val); + } + + double AvgLevel0FileCount() { + std::lock_guard lock(reporter_mutex_); + int64_t sum = 0; + size_t n = 0; + for (const auto& [_, reporter] : reporters_) { + sum += reporter->Level0FileCount(); + n++; + } + return n > 0 ? static_cast(sum) / n : -1; + } + + double MaxCompactionInputSize() { + std::lock_guard lock(reporter_mutex_); + int64_t max_val = -1; + for (const auto& [_, reporter] : reporters_) { + max_val = std::max(max_val, reporter->CompactionInputSize()); + } + return static_cast(max_val); + } + + double MaxCompactionOutputSize() { + std::lock_guard lock(reporter_mutex_); + int64_t max_val = -1; + for (const auto& [_, reporter] : reporters_) { + max_val = std::max(max_val, reporter->CompactionOutputSize()); + } + return static_cast(max_val); + } + + double AvgCompactionInputSize() { + std::lock_guard lock(reporter_mutex_); + int64_t sum = 0; + size_t n = 0; + for (const auto& [_, reporter] : reporters_) { + sum += reporter->CompactionInputSize(); + n++; + } + return n > 0 ? static_cast(sum) / n : -1; + } + + double AvgCompactionOutputSize() { + std::lock_guard lock(reporter_mutex_); + int64_t sum = 0; + size_t n = 0; + for (const auto& [_, reporter] : reporters_) { + sum += reporter->CompactionOutputSize(); + n++; + } + return n > 0 ? static_cast(sum) / n : -1; + } + + double AvgCompactionTime() { + std::lock_guard lock(compaction_times_mutex_); + if (compaction_times_.empty()) { + return 0.0; + } + int64_t sum = 0; + for (const auto& t : compaction_times_) { + sum += t; + } + return static_cast(sum) / compaction_times_.size(); + } + + double MaxTotalFileSize() { + std::lock_guard lock(reporter_mutex_); + int64_t max_val = -1; + for (const auto& [_, reporter] : reporters_) { + max_val = std::max(max_val, reporter->TotalFileSize()); + } + return static_cast(max_val); + } + + double AvgTotalFileSize() { + std::lock_guard lock(reporter_mutex_); + int64_t sum = 0; + size_t n = 0; + for (const auto& [_, reporter] : reporters_) { + sum += reporter->TotalFileSize(); + n++; + } + return n > 0 ? static_cast(sum) / n : -1; + } + + void IncreaseCompactionsCompletedCount() { + compactions_completed_count_++; + } + + void IncreaseCompactionsQueuedCount() { + compactions_queued_count_++; + } + + void DecreaseCompactionsQueuedCount() { + compactions_queued_count_--; + } + + void IncreaseCompactionsTotalCount() { + compactions_total_count_++; + } + + int64_t GetCompactionsCompletedCount() const { + return compactions_completed_count_; + } + + int64_t GetCompactionsTotalCount() const { + return compactions_total_count_; + } + + int64_t GetCompactionsQueuedCount() const { + return compactions_queued_count_; + } + + std::shared_ptr GetMetrics() { + auto metrics = std::make_shared(); + metrics->SetCounter(MAX_LEVEL0_FILE_COUNT, MaxLevel0FileCount()); + metrics->SetCounter(AVG_LEVEL0_FILE_COUNT, AvgLevel0FileCount()); + metrics->SetCounter(AVG_COMPACTION_TIME, AvgCompactionTime()); + metrics->SetCounter(COMPACTION_COMPLETED_COUNT, GetCompactionsCompletedCount()); + metrics->SetCounter(COMPACTION_TOTAL_COUNT, GetCompactionsTotalCount()); + metrics->SetCounter(COMPACTION_QUEUED_COUNT, GetCompactionsQueuedCount()); + metrics->SetGauge(MAX_COMPACTION_INPUT_SIZE, MaxCompactionInputSize()); + metrics->SetGauge(MAX_COMPACTION_OUTPUT_SIZE, MaxCompactionOutputSize()); + metrics->SetGauge(AVG_COMPACTION_INPUT_SIZE, AvgCompactionInputSize()); + metrics->SetGauge(AVG_COMPACTION_OUTPUT_SIZE, AvgCompactionOutputSize()); + metrics->SetGauge(MAX_TOTAL_FILE_SIZE, MaxTotalFileSize()); + metrics->SetGauge(AVG_TOTAL_FILE_SIZE, AvgTotalFileSize()); + return metrics; + } + + private: + std::unordered_map, std::shared_ptr> reporters_; + std::mutex reporter_mutex_; + + std::vector compaction_times_; + std::mutex compaction_times_mutex_; + + std::atomic compactions_completed_count_; + std::atomic compactions_total_count_; + std::atomic compactions_queued_count_; +}; + +} // namespace paimon From 764dfbfced289120fb7c50b2435b158176b63cb1 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 10 Mar 2026 09:09:34 +0800 Subject: [PATCH 09/21] fix --- src/paimon/core/operation/file_system_write_restore.h | 6 ++++++ test/inte/compaction_inte_test.cpp | 4 ++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/paimon/core/operation/file_system_write_restore.h b/src/paimon/core/operation/file_system_write_restore.h index 815da118..df0a7e95 100644 --- a/src/paimon/core/operation/file_system_write_restore.h +++ b/src/paimon/core/operation/file_system_write_restore.h @@ -16,6 +16,12 @@ #pragma once +#include +#include +#include +#include +#include + #include "paimon/core/core_options.h" #include "paimon/core/operation/file_store_scan.h" #include "paimon/core/operation/restore_files.h" diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index 3ec6fd9a..67d08f26 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -14,8 +14,10 @@ * limitations under the License. */ +#include #include #include +#include #include "arrow/c/bridge.h" #include "gtest/gtest.h" @@ -303,7 +305,6 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) auto external_dir = UniqueTestDirectory::Create(); ASSERT_TRUE(external_dir); std::string external_test_dir = "FILE://" + external_dir->Str(); - std::cout << dir->Str() << std::endl; arrow::FieldVector fields = { arrow::field("f0", arrow::utf8()), arrow::field("f1", arrow::int32()), @@ -394,7 +395,6 @@ TEST_F(CompactionInteTest, TestAppendTableWriteAlterTableWithCompaction) { "/orc/append_table_with_alter_table.db/append_table_with_alter_table/"; auto dir = UniqueTestDirectory::Create(); std::string table_path = dir->Str(); - std::cout << "[alter]" << table_path << std::endl; ASSERT_TRUE(TestUtil::CopyDirectory(test_data_path, table_path)); arrow::FieldVector fields = { arrow::field("key0", arrow::int32()), arrow::field("key1", arrow::int32()), From fd12531b7ae55af319eedf670020ea0523caded2 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 10 Mar 2026 09:32:17 +0800 Subject: [PATCH 10/21] fix --- .../core/operation/append_only_file_store_write.cpp | 1 - .../core/operation/metrics/compaction_metrics.h | 13 +++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) 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 668c3bbf..c8783d11 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -181,7 +181,6 @@ Result>> AppendOnlyFileStoreWrit -> Result>> { return CompactRewrite(partition, bucket, files); }; - std::vector> restored; compact_manager = std::make_shared( compact_executor_, restore_data_files, /*dv_maintainer=*/nullptr, options_.GetCompactionMinFileNum(), diff --git a/src/paimon/core/operation/metrics/compaction_metrics.h b/src/paimon/core/operation/metrics/compaction_metrics.h index a21efe0a..868f7371 100644 --- a/src/paimon/core/operation/metrics/compaction_metrics.h +++ b/src/paimon/core/operation/metrics/compaction_metrics.h @@ -17,6 +17,7 @@ #pragma once #include +#include #include #include #include @@ -251,12 +252,12 @@ class CompactionMetrics { std::shared_ptr GetMetrics() { auto metrics = std::make_shared(); - metrics->SetCounter(MAX_LEVEL0_FILE_COUNT, MaxLevel0FileCount()); - metrics->SetCounter(AVG_LEVEL0_FILE_COUNT, AvgLevel0FileCount()); - metrics->SetCounter(AVG_COMPACTION_TIME, AvgCompactionTime()); metrics->SetCounter(COMPACTION_COMPLETED_COUNT, GetCompactionsCompletedCount()); metrics->SetCounter(COMPACTION_TOTAL_COUNT, GetCompactionsTotalCount()); metrics->SetCounter(COMPACTION_QUEUED_COUNT, GetCompactionsQueuedCount()); + metrics->SetGauge(MAX_LEVEL0_FILE_COUNT, MaxLevel0FileCount()); + metrics->SetGauge(AVG_LEVEL0_FILE_COUNT, AvgLevel0FileCount()); + metrics->SetGauge(AVG_COMPACTION_TIME, AvgCompactionTime()); metrics->SetGauge(MAX_COMPACTION_INPUT_SIZE, MaxCompactionInputSize()); metrics->SetGauge(MAX_COMPACTION_OUTPUT_SIZE, MaxCompactionOutputSize()); metrics->SetGauge(AVG_COMPACTION_INPUT_SIZE, AvgCompactionInputSize()); @@ -273,9 +274,9 @@ class CompactionMetrics { std::vector compaction_times_; std::mutex compaction_times_mutex_; - std::atomic compactions_completed_count_; - std::atomic compactions_total_count_; - std::atomic compactions_queued_count_; + std::atomic compactions_completed_count_ = {0}; + std::atomic compactions_total_count_ = {0}; + std::atomic compactions_queued_count_ = {0}; }; } // namespace paimon From 16bf9ad93f6a45060087232b15c70b00e20d5153 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 10 Mar 2026 09:49:21 +0800 Subject: [PATCH 11/21] fix --- src/paimon/CMakeLists.txt | 1 + .../metrics/compaction_metrics_test.cpp | 150 ++++++++++++++++++ 2 files changed, 151 insertions(+) create mode 100644 src/paimon/core/operation/metrics/compaction_metrics_test.cpp diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index 4ed45a0d..886ab02d 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -564,6 +564,7 @@ if(PAIMON_BUILD_TESTS) core/mergetree/merge_tree_writer_test.cpp core/mergetree/sorted_run_test.cpp core/migrate/file_meta_utils_test.cpp + core/operation/metrics/compaction_metrics_test.cpp core/operation/data_evolution_file_store_scan_test.cpp core/operation/data_evolution_split_read_test.cpp core/operation/key_value_file_store_write_test.cpp diff --git a/src/paimon/core/operation/metrics/compaction_metrics_test.cpp b/src/paimon/core/operation/metrics/compaction_metrics_test.cpp new file mode 100644 index 00000000..a2be4f01 --- /dev/null +++ b/src/paimon/core/operation/metrics/compaction_metrics_test.cpp @@ -0,0 +1,150 @@ +/* + * 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/operation/metrics/compaction_metrics.h" + +#include + +#include "gtest/gtest.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::test { + +TEST(CompactionMetricsTest, TestReporterAggregationAndCounters) { + CompactionMetrics metrics; + + auto reporter1 = metrics.CreateReporter(BinaryRow::EmptyRow(), 0); + auto reporter2 = metrics.CreateReporter(BinaryRow::EmptyRow(), 1); + + reporter1->ReportLevel0FileCount(10); + reporter2->ReportLevel0FileCount(4); + reporter1->ReportCompactionInputSize(200); + reporter2->ReportCompactionInputSize(100); + reporter1->ReportCompactionOutputSize(150); + reporter2->ReportCompactionOutputSize(90); + reporter1->ReportTotalFileSize(500); + reporter2->ReportTotalFileSize(300); + + reporter1->ReportCompactionTime(50); + reporter2->ReportCompactionTime(150); + + reporter1->IncreaseCompactionsCompletedCount(); + reporter1->IncreaseCompactionsTotalCount(); + reporter2->IncreaseCompactionsTotalCount(); + reporter1->IncreaseCompactionsQueuedCount(); + reporter2->IncreaseCompactionsQueuedCount(); + reporter2->DecreaseCompactionsQueuedCount(); + + auto snapshot = metrics.GetMetrics(); + + ASSERT_OK_AND_ASSIGN(auto completed, + snapshot->GetCounter(CompactionMetrics::COMPACTION_COMPLETED_COUNT)); + EXPECT_EQ(1, completed); + ASSERT_OK_AND_ASSIGN(auto total, + snapshot->GetCounter(CompactionMetrics::COMPACTION_TOTAL_COUNT)); + EXPECT_EQ(2, total); + ASSERT_OK_AND_ASSIGN(auto queued, + snapshot->GetCounter(CompactionMetrics::COMPACTION_QUEUED_COUNT)); + EXPECT_EQ(1, queued); + + ASSERT_OK_AND_ASSIGN(auto max_l0, snapshot->GetGauge(CompactionMetrics::MAX_LEVEL0_FILE_COUNT)); + EXPECT_DOUBLE_EQ(10.0, max_l0); + ASSERT_OK_AND_ASSIGN(auto avg_l0, snapshot->GetGauge(CompactionMetrics::AVG_LEVEL0_FILE_COUNT)); + EXPECT_DOUBLE_EQ(7.0, avg_l0); + + ASSERT_OK_AND_ASSIGN(auto max_input, + snapshot->GetGauge(CompactionMetrics::MAX_COMPACTION_INPUT_SIZE)); + EXPECT_DOUBLE_EQ(200.0, max_input); + ASSERT_OK_AND_ASSIGN(auto avg_input, + snapshot->GetGauge(CompactionMetrics::AVG_COMPACTION_INPUT_SIZE)); + EXPECT_DOUBLE_EQ(150.0, avg_input); + + ASSERT_OK_AND_ASSIGN(auto max_output, + snapshot->GetGauge(CompactionMetrics::MAX_COMPACTION_OUTPUT_SIZE)); + EXPECT_DOUBLE_EQ(150.0, max_output); + ASSERT_OK_AND_ASSIGN(auto avg_output, + snapshot->GetGauge(CompactionMetrics::AVG_COMPACTION_OUTPUT_SIZE)); + EXPECT_DOUBLE_EQ(120.0, avg_output); + + ASSERT_OK_AND_ASSIGN(auto max_total, + snapshot->GetGauge(CompactionMetrics::MAX_TOTAL_FILE_SIZE)); + EXPECT_DOUBLE_EQ(500.0, max_total); + ASSERT_OK_AND_ASSIGN(auto avg_total, + snapshot->GetGauge(CompactionMetrics::AVG_TOTAL_FILE_SIZE)); + EXPECT_DOUBLE_EQ(400.0, avg_total); + + ASSERT_OK_AND_ASSIGN(auto avg_time, snapshot->GetGauge(CompactionMetrics::AVG_COMPACTION_TIME)); + EXPECT_DOUBLE_EQ(100.0, avg_time); +} + +TEST(CompactionMetricsTest, TestUnregisterAndEmptyDefaults) { + CompactionMetrics metrics; + + auto reporter = metrics.CreateReporter(BinaryRow::EmptyRow(), 7); + reporter->ReportLevel0FileCount(9); + reporter->ReportCompactionInputSize(123); + reporter->ReportCompactionOutputSize(45); + reporter->ReportTotalFileSize(999); + + reporter->Unregister(); + + auto snapshot = metrics.GetMetrics(); + + ASSERT_OK_AND_ASSIGN(auto max_l0, snapshot->GetGauge(CompactionMetrics::MAX_LEVEL0_FILE_COUNT)); + EXPECT_DOUBLE_EQ(-1.0, max_l0); + ASSERT_OK_AND_ASSIGN(auto avg_l0, snapshot->GetGauge(CompactionMetrics::AVG_LEVEL0_FILE_COUNT)); + EXPECT_DOUBLE_EQ(-1.0, avg_l0); + + ASSERT_OK_AND_ASSIGN(auto max_input, + snapshot->GetGauge(CompactionMetrics::MAX_COMPACTION_INPUT_SIZE)); + EXPECT_DOUBLE_EQ(-1.0, max_input); + ASSERT_OK_AND_ASSIGN(auto avg_input, + snapshot->GetGauge(CompactionMetrics::AVG_COMPACTION_INPUT_SIZE)); + EXPECT_DOUBLE_EQ(-1.0, avg_input); + + ASSERT_OK_AND_ASSIGN(auto max_output, + snapshot->GetGauge(CompactionMetrics::MAX_COMPACTION_OUTPUT_SIZE)); + EXPECT_DOUBLE_EQ(-1.0, max_output); + ASSERT_OK_AND_ASSIGN(auto avg_output, + snapshot->GetGauge(CompactionMetrics::AVG_COMPACTION_OUTPUT_SIZE)); + EXPECT_DOUBLE_EQ(-1.0, avg_output); + + ASSERT_OK_AND_ASSIGN(auto max_total, + snapshot->GetGauge(CompactionMetrics::MAX_TOTAL_FILE_SIZE)); + EXPECT_DOUBLE_EQ(-1.0, max_total); + ASSERT_OK_AND_ASSIGN(auto avg_total, + snapshot->GetGauge(CompactionMetrics::AVG_TOTAL_FILE_SIZE)); + EXPECT_DOUBLE_EQ(-1.0, avg_total); + + ASSERT_OK_AND_ASSIGN(auto avg_time, snapshot->GetGauge(CompactionMetrics::AVG_COMPACTION_TIME)); + EXPECT_DOUBLE_EQ(0.0, avg_time); +} + +TEST(CompactionMetricsTest, TestCompactionTimeWindow) { + CompactionMetrics metrics; + + for (int64_t t = 1; t <= CompactionMetrics::kCompactionTimeWindow + 10; ++t) { + metrics.ReportCompactionTime(t); + } + + auto snapshot = metrics.GetMetrics(); + ASSERT_OK_AND_ASSIGN(auto avg_time, snapshot->GetGauge(CompactionMetrics::AVG_COMPACTION_TIME)); + + // Only the last kCompactionTimeWindow values are kept. + EXPECT_DOUBLE_EQ(60.5, avg_time); +} + +} // namespace paimon::test From d1be0e3219cc797954ec50390cce7995f990dda2 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 10 Mar 2026 10:43:40 +0800 Subject: [PATCH 12/21] fix --- src/paimon/common/metrics/metrics_impl.cpp | 26 +++++++++++++--- .../common/metrics/metrics_impl_test.cpp | 31 ++++++++++++++++++- .../append_only_file_store_write.cpp | 1 + 3 files changed, 53 insertions(+), 5 deletions(-) diff --git a/src/paimon/common/metrics/metrics_impl.cpp b/src/paimon/common/metrics/metrics_impl.cpp index afa3c9c0..58d29dd9 100644 --- a/src/paimon/common/metrics/metrics_impl.cpp +++ b/src/paimon/common/metrics/metrics_impl.cpp @@ -113,8 +113,8 @@ std::map MetricsImpl::GetAllGauges() const { void MetricsImpl::Merge(const std::shared_ptr& other) { if (other && this != other.get()) { + std::map other_counters = other->GetAllCounters(); { - std::map other_counters = other->GetAllCounters(); std::lock_guard guard(counter_lock_); for (const auto& kv : other_counters) { auto iter = counters_.find(kv.first); @@ -125,7 +125,18 @@ void MetricsImpl::Merge(const std::shared_ptr& other) { } } } - + std::map other_gauges = other->GetAllGauges(); + { + std::lock_guard guard(gauge_lock_); + for (const auto& kv : other_gauges) { + auto iter = gauges_.find(kv.first); + if (iter == gauges_.end()) { + gauges_[kv.first] = kv.second; + } else { + gauges_[kv.first] += kv.second; + } + } + } auto other_impl = std::dynamic_pointer_cast(other); if (other_impl) { std::vector>> other_histograms; @@ -158,8 +169,15 @@ void MetricsImpl::Merge(const std::shared_ptr& other) { void MetricsImpl::Overwrite(const std::shared_ptr& other) { if (other && this != other.get()) { std::map other_counters = other->GetAllCounters(); - std::lock_guard guard(counter_lock_); - counters_.swap(other_counters); + { + std::lock_guard guard(counter_lock_); + counters_.swap(other_counters); + } + std::map other_gauges = other->GetAllGauges(); + { + std::lock_guard guard(gauge_lock_); + gauges_.swap(other_gauges); + } auto other_impl = std::dynamic_pointer_cast(other); std::map> new_histograms; diff --git a/src/paimon/common/metrics/metrics_impl_test.cpp b/src/paimon/common/metrics/metrics_impl_test.cpp index 83621362..908ad4ce 100644 --- a/src/paimon/common/metrics/metrics_impl_test.cpp +++ b/src/paimon/common/metrics/metrics_impl_test.cpp @@ -49,16 +49,45 @@ TEST(MetricsImplTest, TestSimple) { "Key error: metric 'some_metric' not found"); } +TEST(MetricsImplTest, TestGaugeMergeAndOverwrite) { + auto metrics = std::make_shared(); + metrics->SetGauge("g1", 1.5); + metrics->SetGauge("g2", 2.0); + + auto other = std::make_shared(); + other->SetGauge("g2", 3.25); + other->SetGauge("g3", 4.75); + + metrics->Merge(other); + + ASSERT_OK_AND_ASSIGN(double gauge, metrics->GetGauge("g1")); + EXPECT_DOUBLE_EQ(1.5, gauge); + ASSERT_OK_AND_ASSIGN(gauge, metrics->GetGauge("g2")); + EXPECT_DOUBLE_EQ(5.25, gauge); + ASSERT_OK_AND_ASSIGN(gauge, metrics->GetGauge("g3")); + EXPECT_DOUBLE_EQ(4.75, gauge); + + metrics->Overwrite(other); + + ASSERT_OK_AND_ASSIGN(gauge, metrics->GetGauge("g2")); + EXPECT_DOUBLE_EQ(3.25, gauge); + ASSERT_OK_AND_ASSIGN(gauge, metrics->GetGauge("g3")); + EXPECT_DOUBLE_EQ(4.75, gauge); + ASSERT_NOK_WITH_MSG(metrics->GetGauge("g1"), "Key error: metric 'g1' not found"); +} + TEST(MetricsImplTest, TestToString) { std::shared_ptr metrics1 = std::make_shared(); metrics1->SetCounter("k1", 1); metrics1->SetCounter("k2", 2); + metrics1->SetGauge("g1", 1.25); std::shared_ptr metrics2 = std::make_shared(); metrics2->SetCounter("m1", 3); metrics2->SetCounter("m2", 4); metrics2->SetCounter("k2", 5); + metrics2->SetGauge("g2", 2.5); metrics1->Merge(metrics2); - EXPECT_EQ(metrics1->ToString(), "{\"k1\":1,\"k2\":7,\"m1\":3,\"m2\":4}"); + EXPECT_EQ(metrics1->ToString(), "{\"k1\":1,\"k2\":7,\"m1\":3,\"m2\":4,\"g1\":1.25,\"g2\":2.5}"); } } // namespace paimon::test 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 c8783d11..f072e241 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -144,6 +144,7 @@ Result>> AppendOnlyFileStoreWrite::Com struct_array, SpecialFields::ValueKind().Name())); PAIMON_RETURN_NOT_OK_FROM_ARROW( arrow::ExportArray(*struct_array, c_array.get(), c_schema.get())); + ScopeGuard guard([schema = c_schema.get()]() { ArrowSchemaRelease(schema); }); PAIMON_RETURN_NOT_OK(rewriter->Write(c_array.get())); } guard.Release(); From 95fbc62e2c0cbb6cf6477e1b2bf8cc94601cfd67 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 10 Mar 2026 11:24:22 +0800 Subject: [PATCH 13/21] fix --- src/paimon/core/append/append_only_writer.h | 3 -- .../core/append/append_only_writer_test.cpp | 6 --- .../bucketed_append_compact_manager.cpp | 7 ++-- .../append/bucketed_append_compact_manager.h | 2 +- src/paimon/core/mergetree/merge_tree_writer.h | 6 +-- .../operation/abstract_file_store_write.cpp | 42 ++++++++++--------- .../core/postpone/postpone_bucket_writer.h | 6 +-- src/paimon/core/utils/batch_writer.h | 2 - test/inte/compaction_inte_test.cpp | 6 +-- 9 files changed, 31 insertions(+), 49 deletions(-) diff --git a/src/paimon/core/append/append_only_writer.h b/src/paimon/core/append/append_only_writer.h index b70bd117..d2d12e4b 100644 --- a/src/paimon/core/append/append_only_writer.h +++ b/src/paimon/core/append/append_only_writer.h @@ -75,9 +75,6 @@ class AppendOnlyWriter : public BatchWriter { } Status Sync() override; Status Close() override; - bool IsCompacting() const override { - return false; - } std::shared_ptr GetMetrics() const override { return metrics_; } diff --git a/src/paimon/core/append/append_only_writer_test.cpp b/src/paimon/core/append/append_only_writer_test.cpp index 562dc5a5..0bb272d2 100644 --- a/src/paimon/core/append/append_only_writer_test.cpp +++ b/src/paimon/core/append/append_only_writer_test.cpp @@ -84,7 +84,6 @@ TEST_F(AppendOnlyWriterTest, TestEmptyCommits) { AppendOnlyWriter writer(options, /*schema_id=*/0, schema, /*write_cols=*/std::nullopt, /*max_sequence_number=*/-1, path_factory, memory_pool_, compact_manager_); - ASSERT_FALSE(writer.IsCompacting()); for (int i = 0; i < 3; i++) { ASSERT_OK_AND_ASSIGN(CommitIncrement inc, writer.PrepareCommit(true)); ASSERT_TRUE(inc.GetNewFilesIncrement().IsEmpty()); @@ -114,7 +113,6 @@ TEST_F(AppendOnlyWriterTest, TestWriteAndPrepareCommit) { AppendOnlyWriter writer(options, /*schema_id=*/2, schema, /*write_cols=*/std::nullopt, /*max_sequence_number=*/-1, path_factory, memory_pool_, compact_manager_); - ASSERT_FALSE(writer.IsCompacting()); arrow::StringBuilder builder; for (size_t j = 0; j < 100; j++) { ASSERT_TRUE(builder.Append(std::to_string(j)).ok()); @@ -157,8 +155,6 @@ TEST_F(AppendOnlyWriterTest, TestWriteAndClose) { AppendOnlyWriter writer(options, /*schema_id=*/1, schema, /*write_cols=*/std::nullopt, /*max_sequence_number=*/-1, path_factory, memory_pool_, compact_manager_); - ASSERT_FALSE(writer.IsCompacting()); - auto struct_type = arrow::struct_(fields); arrow::StructBuilder struct_builder(struct_type, arrow::default_memory_pool(), {std::make_shared()}); @@ -203,8 +199,6 @@ TEST_F(AppendOnlyWriterTest, TestInvalidRowKind) { AppendOnlyWriter writer(options, /*schema_id=*/1, schema, /*write_cols=*/std::nullopt, /*max_sequence_number=*/-1, path_factory, memory_pool_, compact_manager_); - ASSERT_FALSE(writer.IsCompacting()); - auto struct_type = arrow::struct_(fields); arrow::StructBuilder struct_builder(struct_type, arrow::default_memory_pool(), {std::make_shared()}); diff --git a/src/paimon/core/append/bucketed_append_compact_manager.cpp b/src/paimon/core/append/bucketed_append_compact_manager.cpp index 7379d05c..6f59af8e 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager.cpp @@ -48,7 +48,7 @@ Status BucketedAppendCompactManager::TriggerCompaction(bool full_compaction) { if (full_compaction) { PAIMON_RETURN_NOT_OK(TriggerFullCompaction()); } else { - PAIMON_RETURN_NOT_OK(TriggerCompactionWithBestEffort()); + TriggerCompactionWithBestEffort(); } return Status::OK(); } @@ -81,9 +81,9 @@ Status BucketedAppendCompactManager::TriggerFullCompaction() { return Status::OK(); } -Status BucketedAppendCompactManager::TriggerCompactionWithBestEffort() { +void BucketedAppendCompactManager::TriggerCompactionWithBestEffort() { if (task_future_.valid()) { - return Status::OK(); + return; } std::optional>> picked = PickCompactBefore(); if (picked) { @@ -95,7 +95,6 @@ Status BucketedAppendCompactManager::TriggerCompactionWithBestEffort() { return compact_task->Execute(); }); } - return Status::OK(); } std::optional>> diff --git a/src/paimon/core/append/bucketed_append_compact_manager.h b/src/paimon/core/append/bucketed_append_compact_manager.h index 7c09b745..2f619493 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.h +++ b/src/paimon/core/append/bucketed_append_compact_manager.h @@ -193,7 +193,7 @@ class BucketedAppendCompactManager : public CompactFutureManager { std::optional>> PickCompactBefore(); Status TriggerFullCompaction(); - Status TriggerCompactionWithBestEffort(); + void TriggerCompactionWithBestEffort(); std::shared_ptr executor_; std::shared_ptr dv_maintainer_; diff --git a/src/paimon/core/mergetree/merge_tree_writer.h b/src/paimon/core/mergetree/merge_tree_writer.h index 2e94e64c..ce457aab 100644 --- a/src/paimon/core/mergetree/merge_tree_writer.h +++ b/src/paimon/core/mergetree/merge_tree_writer.h @@ -70,7 +70,7 @@ class MergeTreeWriter : public BatchWriter { } Result CompactNotCompleted() override { - return Status::NotImplemented("not implemented"); + return false; } Status Sync() override { @@ -79,10 +79,6 @@ class MergeTreeWriter : public BatchWriter { Result PrepareCommit(bool wait_compaction) override; - bool IsCompacting() const override { - return false; - } - Status Close() override { return DoClose(); } diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index d64ab330..6250953f 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -187,19 +187,25 @@ Result>> AbstractFileStoreWrite::Prep partition, bucket, writer_container.total_buckets, increment.GetNewFilesIncrement(), increment.GetCompactIncrement()); result.push_back(committable); - if (committable->IsEmpty()) { - // Condition 1: There is no more record waiting to be committed. Note that the - // condition is < (instead of <=), because each commit identifier may have - // multiple snapshots. We must make sure all snapshots of this identifier are - // committed. - // Condition 2: No compaction is in progress. That is, no more changelog will be - // produced. - // - // Condition 3: The writer has no postponed compaction like gentle lookup - // compaction. - if (writer_container.last_modified_commit_identifier < - latest_committed_identifier && - !writer_container.writer->IsCompacting()) { + if (!committable->IsEmpty()) { + writer_container.last_modified_commit_identifier = commit_identifier; + metrics->Merge(writer_container.writer->GetMetrics()); + ++bucket_iter; + continue; + } + // Condition 1: There is no more record waiting to be committed. Note that the + // condition is < (instead of <=), because each commit identifier may have + // multiple snapshots. We must make sure all snapshots of this identifier are + // committed. + // Condition 2: No compaction is in progress. That is, no more changelog will be + // produced. + // + // Condition 3: The writer has no postponed compaction like gentle lookup + // compaction. + if (writer_container.last_modified_commit_identifier < latest_committed_identifier) { + PAIMON_ASSIGN_OR_RAISE(bool has_pending_compaction, + writer_container.writer->CompactNotCompleted()); + if (!has_pending_compaction) { // Clear writer if no update, and if its latest modification has committed. // // We need a mechanism to clear writers, otherwise there will be more and @@ -214,15 +220,11 @@ Result>> AbstractFileStoreWrite::Prep latest_committed_identifier, commit_identifier); PAIMON_RETURN_NOT_OK(writer_container.writer->Close()); bucket_iter = buckets.erase(bucket_iter); - } else { - metrics->Merge(writer_container.writer->GetMetrics()); - ++bucket_iter; + continue; } - } else { - writer_container.last_modified_commit_identifier = commit_identifier; - metrics->Merge(writer_container.writer->GetMetrics()); - ++bucket_iter; } + metrics->Merge(writer_container.writer->GetMetrics()); + ++bucket_iter; } if (buckets.empty()) { diff --git a/src/paimon/core/postpone/postpone_bucket_writer.h b/src/paimon/core/postpone/postpone_bucket_writer.h index f6519e5b..c9fd55f5 100644 --- a/src/paimon/core/postpone/postpone_bucket_writer.h +++ b/src/paimon/core/postpone/postpone_bucket_writer.h @@ -65,7 +65,7 @@ class PostponeBucketWriter : public BatchWriter { } Result CompactNotCompleted() override { - return Status::NotImplemented("not implemented"); + return false; } Status Sync() override { @@ -74,10 +74,6 @@ class PostponeBucketWriter : public BatchWriter { Result PrepareCommit(bool wait_compaction) override; - bool IsCompacting() const override { - return false; - } - Status Close() override { return DoClose(); } diff --git a/src/paimon/core/utils/batch_writer.h b/src/paimon/core/utils/batch_writer.h index 5098eb19..7dbfd4f4 100644 --- a/src/paimon/core/utils/batch_writer.h +++ b/src/paimon/core/utils/batch_writer.h @@ -49,8 +49,6 @@ class BatchWriter { /// @param wait_compaction if this method need to wait for current compaction to complete /// @return Incremental files in this snapshot cycle virtual Result PrepareCommit(bool wait_compaction) = 0; - /// Check if a compaction is in progress, or if a compaction result remains to be fetched. - virtual bool IsCompacting() const = 0; /// Check if a compaction is in progress, or if a compaction result remains to be fetched, or if /// a compaction should be triggered later. diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index 67d08f26..a96f8bd8 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -251,7 +251,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteBestEffortCompaction) { /*full_compaction=*/false)); ASSERT_OK_AND_ASSIGN( std::vector> commit_messages, - helper->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier++)); + 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()); @@ -341,7 +341,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) /*full_compaction=*/true)); ASSERT_OK_AND_ASSIGN( std::vector> commit_messages, - helper->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier++)); + 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()); @@ -450,7 +450,7 @@ TEST_F(CompactionInteTest, TestAppendTableWriteAlterTableWithCompaction) { /*full_compaction=*/true)); ASSERT_OK_AND_ASSIGN( std::vector> commit_messages, - helper->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier++)); + helper->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier)); ASSERT_OK(helper->commit_->Commit(commit_messages, commit_identifier)); ASSERT_OK_AND_ASSIGN(std::optional snapshot, helper->LatestSnapshot()); From bdbfbb6ac8dd1706ac3a7c708cfe73b8f2788ebc Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 10 Mar 2026 15:41:45 +0800 Subject: [PATCH 14/21] fix --- src/paimon/CMakeLists.txt | 1 + .../common/executor/default_executor_test.cpp | 42 +++++++++ src/paimon/common/executor/executor.cpp | 10 +- .../common/reader/reader_utils_test.cpp | 42 +++++++++ .../bucketed_append_compact_manager.cpp | 3 +- .../bucketed_append_compact_manager_test.cpp | 44 +++++++++ .../operation/abstract_file_store_write.cpp | 2 +- .../append_only_file_store_write.cpp | 2 +- .../core/operation/write_restore_test.cpp | 94 +++++++++++++++++++ 9 files changed, 232 insertions(+), 8 deletions(-) create mode 100644 src/paimon/core/operation/write_restore_test.cpp diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index 886ab02d..34409236 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -585,6 +585,7 @@ if(PAIMON_BUILD_TESTS) core/operation/raw_file_split_read_test.cpp core/operation/read_context_test.cpp core/operation/scan_context_test.cpp + core/operation/write_restore_test.cpp core/operation/write_context_test.cpp core/partition/partition_statistics_test.cpp core/postpone/postpone_bucket_writer_test.cpp diff --git a/src/paimon/common/executor/default_executor_test.cpp b/src/paimon/common/executor/default_executor_test.cpp index 61457ab3..de20a5f7 100644 --- a/src/paimon/common/executor/default_executor_test.cpp +++ b/src/paimon/common/executor/default_executor_test.cpp @@ -15,9 +15,11 @@ */ #include +#include #include #include #include +#include #include #include @@ -80,4 +82,44 @@ TEST(DefaultExecutorTest, TestViaWithException) { ASSERT_THROW(future.get(), std::runtime_error); } +TEST(DefaultExecutorTest, TestShutdownNowDropsPendingTasks) { + auto executor = CreateDefaultExecutor(/*thread_count=*/1); + std::atomic first_started = false; + std::atomic executed_count = 0; + std::promise release_first_task; + auto release_future = release_first_task.get_future(); + executor->Add([&]() { + first_started.store(true); + release_future.wait(); + ++executed_count; + }); + + for (int32_t index = 0; index < 20; ++index) { + executor->Add([&]() { ++executed_count; }); + } + + for (int32_t retry = 0; retry < 100 && !first_started.load(); ++retry) { + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + ASSERT_TRUE(first_started.load()); + std::thread shutdown_thread([&]() { executor->ShutdownNow(); }); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); + release_first_task.set_value(); + shutdown_thread.join(); + + // Only the running task should complete. Pending tasks should be discarded. + ASSERT_EQ(executed_count.load(), 1); +} + +TEST(DefaultExecutorTest, TestAddTaskAfterShutdownNowIgnored) { + auto executor = CreateDefaultExecutor(/*thread_count=*/1); + std::atomic executed_count = 0; + + executor->ShutdownNow(); + executor->Add([&]() { ++executed_count; }); + + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + ASSERT_EQ(executed_count.load(), 0); +} + } // namespace paimon::test diff --git a/src/paimon/common/executor/executor.cpp b/src/paimon/common/executor/executor.cpp index 62e67f1c..f52a98fa 100644 --- a/src/paimon/common/executor/executor.cpp +++ b/src/paimon/common/executor/executor.cpp @@ -38,7 +38,7 @@ class DefaultExecutor : public Executor { private: void WorkerThread(); - void ShutdownInternal(bool drain_queue); + void ShutdownInternal(bool wait_for_pending_tasks); uint32_t thread_count_; std::vector workers_; @@ -55,14 +55,14 @@ DefaultExecutor::DefaultExecutor(uint32_t thread_count) : thread_count_(thread_c } } -void DefaultExecutor::ShutdownInternal(bool drain_queue) { +void DefaultExecutor::ShutdownInternal(bool wait_for_pending_tasks) { { std::unique_lock lock(queue_mutex_); if (stop_) { return; } stop_ = true; - if (!drain_queue) { + if (!wait_for_pending_tasks) { // Discard all pending tasks immediately. std::queue> empty; tasks_.swap(empty); @@ -78,12 +78,12 @@ void DefaultExecutor::ShutdownInternal(bool drain_queue) { DefaultExecutor::~DefaultExecutor() { // Graceful shutdown: wait for all pending tasks to complete. - ShutdownInternal(/*drain_queue=*/true); + ShutdownInternal(/*wait_for_pending_tasks=*/true); } void DefaultExecutor::ShutdownNow() { // Immediate shutdown: discard all pending tasks. - ShutdownInternal(/*drain_queue=*/false); + ShutdownInternal(/*wait_for_pending_tasks=*/false); } void DefaultExecutor::Add(std::function func) { diff --git a/src/paimon/common/reader/reader_utils_test.cpp b/src/paimon/common/reader/reader_utils_test.cpp index 040e32bb..c75f264f 100644 --- a/src/paimon/common/reader/reader_utils_test.cpp +++ b/src/paimon/common/reader/reader_utils_test.cpp @@ -100,4 +100,46 @@ TEST(ReaderUtilsTest, TestApplyBitmapToReadBatch) { "NextBatchWithBitmap should always return the result with at least one valid row " "except eof"); } + +TEST(ReaderUtilsTest, TestRemoveFieldFromStructArrayFieldNotFound) { + auto struct_type = + arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("b", arrow::utf8())}); + auto src_array = arrow::ipc::internal::json::ArrayFromJSON( + struct_type, R"([{"a":1,"b":"x"},{"a":2,"b":"y"},{"a":3,"b":"z"}])") + .ValueOrDie(); + auto src_struct_array = std::static_pointer_cast(src_array); + + ASSERT_OK_AND_ASSIGN(auto result, + ReaderUtils::RemoveFieldFromStructArray(src_struct_array, "missing")); + + ASSERT_TRUE(result->Equals(src_struct_array)); + ASSERT_EQ(result->type()->num_fields(), 2); +} + +TEST(ReaderUtilsTest, TestRemoveFieldFromStructArraySuccess) { + auto struct_type = + arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("b", arrow::utf8()), + arrow::field("c", arrow::int64())}); + auto src_array = + arrow::ipc::internal::json::ArrayFromJSON( + struct_type, + R"([{"a":1,"b":"x","c":10},{"a":2,"b":"y","c":20},{"a":3,"b":"z","c":30}])") + .ValueOrDie(); + auto src_struct_array = std::static_pointer_cast(src_array); + + ASSERT_OK_AND_ASSIGN(auto result, + ReaderUtils::RemoveFieldFromStructArray(src_struct_array, "b")); + + auto expected_type = + arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("c", arrow::int64())}); + auto expected_array = arrow::ipc::internal::json::ArrayFromJSON( + expected_type, R"([{"a":1,"c":10},{"a":2,"c":20},{"a":3,"c":30}])") + .ValueOrDie(); + auto expected_struct_array = std::static_pointer_cast(expected_array); + + ASSERT_EQ(result->type()->num_fields(), 2); + ASSERT_EQ(result->type()->field(0)->name(), "a"); + ASSERT_EQ(result->type()->field(1)->name(), "c"); + ASSERT_TRUE(result->Equals(expected_struct_array)); +} } // namespace paimon::test diff --git a/src/paimon/core/append/bucketed_append_compact_manager.cpp b/src/paimon/core/append/bucketed_append_compact_manager.cpp index 6f59af8e..819232cc 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager.cpp @@ -115,7 +115,8 @@ BucketedAppendCompactManager::PickCompactBefore() { if (file_num >= min_file_num_) { return std::vector>(candidates.begin(), candidates.end()); } else if (total_file_size >= target_file_size_ * 2) { - // let pointer shift one pos to right + // Shift the compaction window right and drop the oldest file so picked files stay + // contiguous, preserving append order during compaction. std::shared_ptr removed = candidates.front(); candidates.pop_front(); total_file_size -= removed->file_size; 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 fdc1a78b..caa02f5a 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager_test.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager_test.cpp @@ -160,6 +160,16 @@ TEST_F(BucketedAppendCompactManagerTest, TestPickEmptyAndNotRelease) { InnerTest(to_compact, /*expected_present=*/false, /*expected_compact_before=*/{}, to_compact); } +TEST_F(BucketedAppendCompactManagerTest, TestPickPresentWhenEnoughSmallFiles) { + // All four files are small and should be picked once min_file_num is reached. + std::vector> to_compact_before_pick = { + NewFile(1, 100), NewFile(101, 200), NewFile(201, 300), NewFile(301, 400)}; + InnerTest(to_compact_before_pick, + /*expected_present=*/true, + /*expected_compact_before=*/to_compact_before_pick, + /*to_compact_after_pick=*/{}); +} + TEST_F(BucketedAppendCompactManagerTest, TestPickEmptyAndRelease) { // large file, release InnerTest(/*to_compact_before_pick=*/{NewFile(1, 2048)}, /*expected_present=*/false, @@ -216,4 +226,38 @@ TEST_F(BucketedAppendCompactManagerTest, TestPickEmptyAndRelease) { /*to_compact_after_pick=*/{NewFile(2051, 2100), NewFile(2101, 2110)}); } +TEST_F(BucketedAppendCompactManagerTest, TestPick) { + // fileNum is 13 (which > 4) and totalFileSize is 130 (which < 1024) + InnerTest({NewFile(1, 10), NewFile(11, 20), NewFile(21, 30), NewFile(31, 40), NewFile(41, 50), + NewFile(51, 60), NewFile(61, 70), NewFile(71, 80), NewFile(81, 90), NewFile(91, 100), + NewFile(101, 110), NewFile(111, 120), NewFile(121, 130)}, + /*expected_present=*/true, /*expected_compact_before=*/ + {NewFile(1, 10), NewFile(11, 20), NewFile(21, 30), NewFile(31, 40)}, + /*to_compact_after_pick=*/ + {NewFile(41, 50), NewFile(51, 60), NewFile(61, 70), NewFile(71, 80), NewFile(81, 90), + NewFile(91, 100), NewFile(101, 110), NewFile(111, 120), NewFile(121, 130)}); + + // fileNum is 4 (which > 3) and totalFileSize is 1026 (which > 1024) + InnerTest({NewFile(1, 2), NewFile(3, 500), NewFile(501, 1000), NewFile(1001, 1025), + NewFile(1026, 1050)}, + /*expected_present=*/true, /*expected_compact_before=*/ + {NewFile(1, 2), NewFile(3, 500), NewFile(501, 1000), NewFile(1001, 1025)}, + /*to_compact_after_pick=*/{NewFile(1026, 1050)}); + + // The window shifts right after large files are dropped, then picks contiguous files. + InnerTest({NewFile(1, 1022), NewFile(1023, 1024), NewFile(1025, 2050), + // 2051~2510, ..., 2611~2620 + NewFile(2051, 2510), NewFile(2511, 2520), NewFile(2521, 2530), NewFile(2531, 2540), + NewFile(2541, 2550), NewFile(2551, 2560), NewFile(2561, 2570), NewFile(2571, 2580), + NewFile(2581, 2590), NewFile(2591, 2600), NewFile(2601, 2610), NewFile(2611, 2620), + NewFile(2621, 2630)}, + /*expected_present=*/true, + /*expected_compact_before=*/ + {NewFile(1023, 1024), NewFile(1025, 2050), NewFile(2051, 2510), NewFile(2511, 2520)}, + /*to_compact_after_pick=*/ + {NewFile(2521, 2530), NewFile(2531, 2540), NewFile(2541, 2550), NewFile(2551, 2560), + NewFile(2561, 2570), NewFile(2571, 2580), NewFile(2581, 2590), NewFile(2591, 2600), + NewFile(2601, 2610), NewFile(2611, 2620), NewFile(2621, 2630)}); +} + } // namespace paimon::test diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index 6250953f..2a8af44e 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -129,7 +129,7 @@ Status AbstractFileStoreWrite::Write(std::unique_ptr&& batch) { Status AbstractFileStoreWrite::Compact(const std::map& partition, int32_t bucket, bool full_compaction) { - PAIMON_ASSIGN_OR_RAISE(BinaryRow part, file_store_path_factory_->ToBinaryRow(partition)) + PAIMON_ASSIGN_OR_RAISE(BinaryRow part, file_store_path_factory_->ToBinaryRow(partition)); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr writer, GetWriter(part, bucket)); assert(writer); return writer->Compact(full_compaction); 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 f072e241..894d6b86 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -148,8 +148,8 @@ Result>> AppendOnlyFileStoreWrite::Com PAIMON_RETURN_NOT_OK(rewriter->Write(c_array.get())); } guard.Release(); - PAIMON_RETURN_NOT_OK(rewriter->Close()); reader->Close(); + PAIMON_RETURN_NOT_OK(rewriter->Close()); return rewriter->GetResult(); } diff --git a/src/paimon/core/operation/write_restore_test.cpp b/src/paimon/core/operation/write_restore_test.cpp new file mode 100644 index 00000000..87ed00dc --- /dev/null +++ b/src/paimon/core/operation/write_restore_test.cpp @@ -0,0 +1,94 @@ +/* + * 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/operation/write_restore.h" + +#include +#include +#include +#include + +#include "gtest/gtest.h" +#include "paimon/core/manifest/file_kind.h" + +namespace paimon::test { + +namespace { + +std::shared_ptr CreateDataFileMeta(const std::string& file_name) { + return std::make_shared( + file_name, /*file_size=*/128, /*row_count=*/10, DataFileMeta::EmptyMinKey(), + DataFileMeta::EmptyMaxKey(), SimpleStats::EmptyStats(), SimpleStats::EmptyStats(), + /*min_sequence_number=*/1, /*max_sequence_number=*/1, /*schema_id=*/1, + /*level=*/DataFileMeta::DUMMY_LEVEL, + /*extra_files=*/std::vector>(), + /*creation_time=*/Timestamp(0, 0), /*delete_row_count=*/std::nullopt, + /*embedded_index=*/nullptr, /*file_source=*/std::nullopt, + /*value_stats_cols=*/std::nullopt, /*external_path=*/std::nullopt, + /*first_row_id=*/std::nullopt, /*write_cols=*/std::nullopt); +} + +ManifestEntry CreateManifestEntry(int32_t total_buckets, const std::string& file_name) { + return ManifestEntry(FileKind::Add(), BinaryRow::EmptyRow(), /*bucket=*/0, total_buckets, + CreateDataFileMeta(file_name)); +} + +} // namespace + +TEST(WriteRestoreTest, ExtractDataFilesEmptyEntries) { + std::vector entries; + std::vector> data_files; + + auto result = WriteRestore::ExtractDataFiles(entries, &data_files); + + ASSERT_TRUE(result.ok()) << result.status().ToString(); + ASSERT_FALSE(result.value().has_value()); + ASSERT_TRUE(data_files.empty()); +} + +TEST(WriteRestoreTest, ExtractDataFilesConsistentTotalBuckets) { + std::vector entries = { + CreateManifestEntry(/*total_buckets=*/4, "file-1.parquet"), + CreateManifestEntry(/*total_buckets=*/4, "file-2.parquet"), + CreateManifestEntry(/*total_buckets=*/4, "file-3.parquet")}; + std::vector> data_files; + + auto result = WriteRestore::ExtractDataFiles(entries, &data_files); + + ASSERT_TRUE(result.ok()) << result.status().ToString(); + ASSERT_TRUE(result.value().has_value()); + ASSERT_EQ(result.value().value(), 4); + ASSERT_EQ(data_files.size(), 3); + ASSERT_EQ(data_files[0]->file_name, "file-1.parquet"); + ASSERT_EQ(data_files[1]->file_name, "file-2.parquet"); + ASSERT_EQ(data_files[2]->file_name, "file-3.parquet"); +} + +TEST(WriteRestoreTest, ExtractDataFilesInconsistentTotalBuckets) { + std::vector entries = { + CreateManifestEntry(/*total_buckets=*/2, "file-1.parquet"), + CreateManifestEntry(/*total_buckets=*/3, "file-2.parquet")}; + std::vector> data_files; + + auto result = WriteRestore::ExtractDataFiles(entries, &data_files); + + ASSERT_FALSE(result.ok()); + ASSERT_NE(result.status().ToString().find("different total bucket number"), std::string::npos); + ASSERT_EQ(data_files.size(), 1); + ASSERT_EQ(data_files[0]->file_name, "file-1.parquet"); +} + +} // namespace paimon::test From 93370419115b1fec8cb7fabb1c47867feaf2bfc8 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 10 Mar 2026 15:52:41 +0800 Subject: [PATCH 15/21] fix --- src/paimon/CMakeLists.txt | 1 + .../bucketed_dv_maintainer_test.cpp | 100 ++++++++++++++++++ 2 files changed, 101 insertions(+) create mode 100644 src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index 34409236..bfa73af4 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -503,6 +503,7 @@ if(PAIMON_BUILD_TESTS) core/core_options_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 core/deletionvectors/deletion_vector_test.cpp core/deletionvectors/deletion_vectors_index_file_test.cpp core/index/index_in_data_file_dir_path_factory_test.cpp diff --git a/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp b/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp new file mode 100644 index 00000000..c710aa99 --- /dev/null +++ b/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp @@ -0,0 +1,100 @@ +/* + * 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/deletionvectors/bucketed_dv_maintainer.h" + +#include +#include +#include + +#include "gtest/gtest.h" +#include "paimon/core/deletionvectors/bitmap_deletion_vector.h" +#include "paimon/fs/file_system_factory.h" +#include "paimon/testing/mock/mock_index_path_factory.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::test { + +namespace { + +std::shared_ptr CreateDvIndexFile(const std::string& root_path) { + auto memory_pool = GetDefaultPool(); + EXPECT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", root_path, {})); + auto path_factory = std::make_shared(root_path); + return std::make_shared(fs, path_factory, + /*target_size_per_index_file=*/1024 * 1024, + /*bitmap64=*/false, memory_pool); +} + +std::shared_ptr CreateDeletionVector(int32_t begin, int32_t end) { + RoaringBitmap32 bitmap; + for (int32_t value = begin; value < end; ++value) { + bitmap.Add(value); + } + return std::make_shared(bitmap); +} + +} // namespace + +TEST(BucketedDvMaintainerTest, TestDeletionVectorLookupAndIndexFileGetter) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + auto index_file = CreateDvIndexFile(dir->Str()); + std::map> deletion_vectors = { + {"file-a", CreateDeletionVector(0, 3)}, {"file-b", CreateDeletionVector(10, 12)}}; + + BucketedDvMaintainer maintainer(index_file, deletion_vectors); + + ASSERT_EQ(maintainer.DvIndexFile(), index_file); + auto lookup_hit = maintainer.DeletionVectorOf("file-a"); + ASSERT_TRUE(lookup_hit.has_value()); + auto lookup_miss = maintainer.DeletionVectorOf("missing"); + ASSERT_FALSE(lookup_miss.has_value()); +} + +TEST(BucketedDvMaintainerTest, TestWriteDeletionVectorsIndexOnlyWhenModified) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + auto index_file = CreateDvIndexFile(dir->Str()); + std::map> deletion_vectors = { + {"file-a", CreateDeletionVector(0, 3)}, + {"file-b", CreateDeletionVector(10, 14)}, + }; + + BucketedDvMaintainer maintainer(index_file, deletion_vectors); + + // No modification yet, so no index file should be written. + ASSERT_OK_AND_ASSIGN(auto not_modified_write, maintainer.WriteDeletionVectorsIndex()); + ASSERT_FALSE(not_modified_write.has_value()); + + // Removing a missing key should keep the maintainer unmodified. + maintainer.RemoveDeletionVectorOf("missing"); + ASSERT_OK_AND_ASSIGN(auto still_not_modified_write, maintainer.WriteDeletionVectorsIndex()); + ASSERT_FALSE(still_not_modified_write.has_value()); + + // Removing an existing key marks the maintainer modified and triggers one write. + maintainer.RemoveDeletionVectorOf("file-a"); + ASSERT_OK_AND_ASSIGN(auto modified_write, maintainer.WriteDeletionVectorsIndex()); + ASSERT_TRUE(modified_write.has_value()); + ASSERT_GT(modified_write.value()->FileSize(), 0); + + // Modification flag should be reset after a successful write. + ASSERT_OK_AND_ASSIGN(auto write_after_reset, maintainer.WriteDeletionVectorsIndex()); + ASSERT_FALSE(write_after_reset.has_value()); +} + +} // namespace paimon::test From f84ce84db8131de84215dc2fafd32127741090d7 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 11 Mar 2026 09:22:36 +0800 Subject: [PATCH 16/21] fix --- src/paimon/core/append/bucketed_append_compact_manager.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/paimon/core/append/bucketed_append_compact_manager.h b/src/paimon/core/append/bucketed_append_compact_manager.h index 2f619493..309d45ab 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.h +++ b/src/paimon/core/append/bucketed_append_compact_manager.h @@ -95,7 +95,10 @@ class BucketedAppendCompactManager : public CompactFutureManager { bool blocking) override; Status Close() override { - // TODO(yonghao.fyh): metrics reporter + if (reporter_) { + reporter_->Unregister(); + reporter_.reset(); + } return Status::OK(); } @@ -113,6 +116,7 @@ class BucketedAppendCompactManager : public CompactFutureManager { const std::vector>& to_compact, CompactRewriter rewriter) { PAIMON_ASSIGN_OR_RAISE(std::vector> rewrite, rewriter(to_compact)); + auto result = std::make_shared(to_compact, rewrite); if (dv_maintainer != nullptr) { for (const auto& file : to_compact) { From e72ce1d2489f875e12cae6c85ddc915f6eace7f8 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 11 Mar 2026 09:25:50 +0800 Subject: [PATCH 17/21] fix --- src/paimon/core/compact/compact_task.h | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/paimon/core/compact/compact_task.h b/src/paimon/core/compact/compact_task.h index ddf08aa9..2c281734 100644 --- a/src/paimon/core/compact/compact_task.h +++ b/src/paimon/core/compact/compact_task.h @@ -40,10 +40,12 @@ class CompactTask { Result> Execute() { Duration duration; PAIMON_ASSIGN_OR_RAISE(std::shared_ptr result, DoCompact()); - reporter_->ReportCompactionTime(static_cast(duration.Get())); - reporter_->IncreaseCompactionsCompletedCount(); - reporter_->ReportCompactionInputSize(CollectRewriteSize(result->Before())); - reporter_->ReportCompactionOutputSize(CollectRewriteSize(result->After())); + if (reporter_) { + reporter_->ReportCompactionTime(static_cast(duration.Get())); + reporter_->IncreaseCompactionsCompletedCount(); + reporter_->ReportCompactionInputSize(CollectRewriteSize(result->Before())); + reporter_->ReportCompactionOutputSize(CollectRewriteSize(result->After())); + } PAIMON_LOG_DEBUG( logger_, "Done compacting %zu files to %zu files in %lldms. Rewrite input file size " From ce671265bfc9afa0f12ad58d7cf66dbff6d57ac5 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 11 Mar 2026 09:36:31 +0800 Subject: [PATCH 18/21] fix --- test/inte/compaction_inte_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index a96f8bd8..24ddcef9 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -165,7 +165,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompaction) { /*full_compaction=*/true)); ASSERT_OK_AND_ASSIGN( std::vector> commit_messages, - helper->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier++)); + 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()); From 66050b3b7a9307947be9af6442317e18093c0e01 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 11 Mar 2026 15:13:00 +0800 Subject: [PATCH 19/21] fix --- src/paimon/common/reader/reader_utils.cpp | 22 ---------- src/paimon/common/reader/reader_utils.h | 3 -- .../common/reader/reader_utils_test.cpp | 41 ------------------ src/paimon/common/utils/arrow/arrow_utils.cpp | 28 +++++++++++++ src/paimon/common/utils/arrow/arrow_utils.h | 3 ++ .../common/utils/arrow/arrow_utils_test.cpp | 42 +++++++++++++++++++ src/paimon/core/append/append_only_writer.cpp | 6 +-- src/paimon/core/append/append_only_writer.h | 6 +-- .../core/append/append_only_writer_test.cpp | 16 +++---- .../bucketed_append_compact_manager.cpp | 18 ++++++++ .../append/bucketed_append_compact_manager.h | 16 +------ .../core/compact/compact_deletion_file.h | 1 - .../deletionvectors/bucketed_dv_maintainer.h | 3 +- .../append_only_file_store_write.cpp | 32 +++++++------- 14 files changed, 126 insertions(+), 111 deletions(-) diff --git a/src/paimon/common/reader/reader_utils.cpp b/src/paimon/common/reader/reader_utils.cpp index 13fc5595..3d8e3579 100644 --- a/src/paimon/common/reader/reader_utils.cpp +++ b/src/paimon/common/reader/reader_utils.cpp @@ -107,26 +107,4 @@ BatchReader::ReadBatchWithBitmap ReaderUtils::AddAllValidBitmap(BatchReader::Rea return std::make_pair(std::move(batch), std::move(all_valid)); } -Result> ReaderUtils::RemoveFieldFromStructArray( - const std::shared_ptr& struct_array, const std::string& field_name) { - auto struct_type = std::static_pointer_cast(struct_array->type()); - int32_t field_idx = struct_type->GetFieldIndex(field_name); - if (field_idx == -1) { - return struct_array; - } - std::vector> new_arrays; - std::vector> new_fields; - for (int32_t i = 0; i < struct_type->num_fields(); ++i) { - if (i != field_idx) { - new_arrays.emplace_back(struct_array->field(i)); - new_fields.emplace_back(struct_type->field(i)); - } - } - PAIMON_ASSIGN_OR_RAISE_FROM_ARROW( - std::shared_ptr array, - arrow::StructArray::Make(new_arrays, new_fields, struct_array->null_bitmap(), - struct_array->null_count())); - return array; -} - } // namespace paimon diff --git a/src/paimon/common/reader/reader_utils.h b/src/paimon/common/reader/reader_utils.h index b8cdb48c..1cb381a2 100644 --- a/src/paimon/common/reader/reader_utils.h +++ b/src/paimon/common/reader/reader_utils.h @@ -54,8 +54,5 @@ class ReaderUtils { /// Precondition: input bitmap is not empty static Result GenerateFilteredArrayVector( const std::shared_ptr& src_array, const RoaringBitmap32& bitmap); - - static Result> RemoveFieldFromStructArray( - const std::shared_ptr& struct_array, const std::string& field_name); }; } // namespace paimon diff --git a/src/paimon/common/reader/reader_utils_test.cpp b/src/paimon/common/reader/reader_utils_test.cpp index c75f264f..0fcdb9c4 100644 --- a/src/paimon/common/reader/reader_utils_test.cpp +++ b/src/paimon/common/reader/reader_utils_test.cpp @@ -101,45 +101,4 @@ TEST(ReaderUtilsTest, TestApplyBitmapToReadBatch) { "except eof"); } -TEST(ReaderUtilsTest, TestRemoveFieldFromStructArrayFieldNotFound) { - auto struct_type = - arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("b", arrow::utf8())}); - auto src_array = arrow::ipc::internal::json::ArrayFromJSON( - struct_type, R"([{"a":1,"b":"x"},{"a":2,"b":"y"},{"a":3,"b":"z"}])") - .ValueOrDie(); - auto src_struct_array = std::static_pointer_cast(src_array); - - ASSERT_OK_AND_ASSIGN(auto result, - ReaderUtils::RemoveFieldFromStructArray(src_struct_array, "missing")); - - ASSERT_TRUE(result->Equals(src_struct_array)); - ASSERT_EQ(result->type()->num_fields(), 2); -} - -TEST(ReaderUtilsTest, TestRemoveFieldFromStructArraySuccess) { - auto struct_type = - arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("b", arrow::utf8()), - arrow::field("c", arrow::int64())}); - auto src_array = - arrow::ipc::internal::json::ArrayFromJSON( - struct_type, - R"([{"a":1,"b":"x","c":10},{"a":2,"b":"y","c":20},{"a":3,"b":"z","c":30}])") - .ValueOrDie(); - auto src_struct_array = std::static_pointer_cast(src_array); - - ASSERT_OK_AND_ASSIGN(auto result, - ReaderUtils::RemoveFieldFromStructArray(src_struct_array, "b")); - - auto expected_type = - arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("c", arrow::int64())}); - auto expected_array = arrow::ipc::internal::json::ArrayFromJSON( - expected_type, R"([{"a":1,"c":10},{"a":2,"c":20},{"a":3,"c":30}])") - .ValueOrDie(); - auto expected_struct_array = std::static_pointer_cast(expected_array); - - ASSERT_EQ(result->type()->num_fields(), 2); - ASSERT_EQ(result->type()->field(0)->name(), "a"); - ASSERT_EQ(result->type()->field(1)->name(), "c"); - ASSERT_TRUE(result->Equals(expected_struct_array)); -} } // namespace paimon::test diff --git a/src/paimon/common/utils/arrow/arrow_utils.cpp b/src/paimon/common/utils/arrow/arrow_utils.cpp index 0db69fb9..5cbbce64 100644 --- a/src/paimon/common/utils/arrow/arrow_utils.cpp +++ b/src/paimon/common/utils/arrow/arrow_utils.cpp @@ -15,6 +15,11 @@ */ #include "paimon/common/utils/arrow/arrow_utils.h" + +#include "arrow/array/array_base.h" +#include "arrow/array/array_nested.h" +#include "paimon/common/utils/arrow/status_utils.h" + namespace paimon { Result> ArrowUtils::DataTypeToSchema( const std::shared_ptr& data_type) { @@ -115,4 +120,27 @@ Status ArrowUtils::InnerCheckNullabilityMatch(const std::shared_ptr> ArrowUtils::RemoveFieldFromStructArray( + const std::shared_ptr& struct_array, const std::string& field_name) { + auto struct_type = std::static_pointer_cast(struct_array->type()); + int32_t field_idx = struct_type->GetFieldIndex(field_name); + if (field_idx == -1) { + return struct_array; + } + std::vector> new_arrays; + std::vector> new_fields; + for (int32_t i = 0; i < struct_type->num_fields(); ++i) { + if (i != field_idx) { + new_arrays.emplace_back(struct_array->field(i)); + new_fields.emplace_back(struct_type->field(i)); + } + } + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW( + std::shared_ptr array, + arrow::StructArray::Make(new_arrays, new_fields, struct_array->null_bitmap(), + struct_array->null_count(), struct_array->offset())); + return array; +} + } // namespace paimon diff --git a/src/paimon/common/utils/arrow/arrow_utils.h b/src/paimon/common/utils/arrow/arrow_utils.h index 41b43e65..4e49c913 100644 --- a/src/paimon/common/utils/arrow/arrow_utils.h +++ b/src/paimon/common/utils/arrow/arrow_utils.h @@ -43,6 +43,9 @@ class PAIMON_EXPORT ArrowUtils { // avoid subsequent multi-threading problems. static void TraverseArray(const std::shared_ptr& array); + static Result> RemoveFieldFromStructArray( + const std::shared_ptr& struct_array, const std::string& field_name); + private: static Status InnerCheckNullabilityMatch(const std::shared_ptr& field, const std::shared_ptr& data); diff --git a/src/paimon/common/utils/arrow/arrow_utils_test.cpp b/src/paimon/common/utils/arrow/arrow_utils_test.cpp index 42ef9c4b..ba27cbcc 100644 --- a/src/paimon/common/utils/arrow/arrow_utils_test.cpp +++ b/src/paimon/common/utils/arrow/arrow_utils_test.cpp @@ -336,4 +336,46 @@ TEST(ArrowUtilsTest, TestCheckNullableMatchComplex) { } } +TEST(ArrowUtilsTest, TestRemoveFieldFromStructArrayFieldNotFound) { + auto struct_type = + arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("b", arrow::utf8())}); + auto src_array = arrow::ipc::internal::json::ArrayFromJSON( + struct_type, R"([{"a":1,"b":"x"},{"a":2,"b":"y"},{"a":3,"b":"z"}])") + .ValueOrDie(); + auto src_struct_array = std::static_pointer_cast(src_array); + + ASSERT_OK_AND_ASSIGN(auto result, + ArrowUtils::RemoveFieldFromStructArray(src_struct_array, "missing")); + + ASSERT_TRUE(result->Equals(src_struct_array)); + ASSERT_EQ(result->type()->num_fields(), 2); +} + +TEST(ArrowUtilsTest, TestRemoveFieldFromStructArraySuccess) { + auto struct_type = + arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("b", arrow::utf8()), + arrow::field("c", arrow::int64())}); + auto src_array = + arrow::ipc::internal::json::ArrayFromJSON( + struct_type, + R"([{"a":1,"b":"x","c":10},{"a":2,"b":"y","c":20},{"a":3,"b":"z","c":30}])") + .ValueOrDie(); + auto src_struct_array = std::static_pointer_cast(src_array); + + ASSERT_OK_AND_ASSIGN(auto result, + ArrowUtils::RemoveFieldFromStructArray(src_struct_array, "b")); + + auto expected_type = + arrow::struct_({arrow::field("a", arrow::int32()), arrow::field("c", arrow::int64())}); + auto expected_array = arrow::ipc::internal::json::ArrayFromJSON( + expected_type, R"([{"a":1,"c":10},{"a":2,"c":20},{"a":3,"c":30}])") + .ValueOrDie(); + auto expected_struct_array = std::static_pointer_cast(expected_array); + + ASSERT_EQ(result->type()->num_fields(), 2); + ASSERT_EQ(result->type()->field(0)->name(), "a"); + ASSERT_EQ(result->type()->field(1)->name(), "c"); + ASSERT_TRUE(result->Equals(expected_struct_array)); +} + } // namespace paimon::test diff --git a/src/paimon/core/append/append_only_writer.cpp b/src/paimon/core/append/append_only_writer.cpp index 4902cc6e..97fe575a 100644 --- a/src/paimon/core/append/append_only_writer.cpp +++ b/src/paimon/core/append/append_only_writer.cpp @@ -55,16 +55,16 @@ AppendOnlyWriter::AppendOnlyWriter(const CoreOptions& options, int64_t schema_id const std::optional>& write_cols, int64_t max_sequence_number, const std::shared_ptr& path_factory, - const std::shared_ptr& memory_pool, - const std::shared_ptr& compact_manager) + const std::shared_ptr& compact_manager, + const std::shared_ptr& memory_pool) : options_(options), schema_id_(schema_id), write_schema_(write_schema), write_cols_(write_cols), seq_num_counter_(std::make_shared(max_sequence_number + 1)), path_factory_(path_factory), - memory_pool_(memory_pool), compact_manager_(compact_manager), + memory_pool_(memory_pool), metrics_(std::make_shared()) {} AppendOnlyWriter::~AppendOnlyWriter() = default; diff --git a/src/paimon/core/append/append_only_writer.h b/src/paimon/core/append/append_only_writer.h index d2d12e4b..7c73c473 100644 --- a/src/paimon/core/append/append_only_writer.h +++ b/src/paimon/core/append/append_only_writer.h @@ -59,8 +59,8 @@ class AppendOnlyWriter : public BatchWriter { const std::optional>& write_cols, int64_t max_sequence_number, const std::shared_ptr& path_factory, - const std::shared_ptr& memory_pool, - const std::shared_ptr& compact_manager); + const std::shared_ptr& compact_manager, + const std::shared_ptr& memory_pool); ~AppendOnlyWriter() override; @@ -110,8 +110,8 @@ class AppendOnlyWriter : public BatchWriter { std::optional> write_cols_; std::shared_ptr seq_num_counter_; std::shared_ptr path_factory_; - std::shared_ptr memory_pool_; std::shared_ptr compact_manager_; + std::shared_ptr memory_pool_; std::shared_ptr metrics_; std::vector> new_files_; diff --git a/src/paimon/core/append/append_only_writer_test.cpp b/src/paimon/core/append/append_only_writer_test.cpp index 0bb272d2..7c77b8ca 100644 --- a/src/paimon/core/append/append_only_writer_test.cpp +++ b/src/paimon/core/append/append_only_writer_test.cpp @@ -82,8 +82,8 @@ TEST_F(AppendOnlyWriterTest, TestEmptyCommits) { ASSERT_OK(path_factory->Init(dir->Str(), "mock_format", options.DataFilePrefix(), nullptr)); AppendOnlyWriter writer(options, /*schema_id=*/0, schema, /*write_cols=*/std::nullopt, - /*max_sequence_number=*/-1, path_factory, memory_pool_, - compact_manager_); + /*max_sequence_number=*/-1, path_factory, compact_manager_, + memory_pool_); for (int i = 0; i < 3; i++) { ASSERT_OK_AND_ASSIGN(CommitIncrement inc, writer.PrepareCommit(true)); ASSERT_TRUE(inc.GetNewFilesIncrement().IsEmpty()); @@ -111,8 +111,8 @@ TEST_F(AppendOnlyWriterTest, TestWriteAndPrepareCommit) { auto path_factory = std::make_shared(); ASSERT_OK(path_factory->Init(dir->Str(), "mock_format", options.DataFilePrefix(), nullptr)); AppendOnlyWriter writer(options, /*schema_id=*/2, schema, /*write_cols=*/std::nullopt, - /*max_sequence_number=*/-1, path_factory, memory_pool_, - compact_manager_); + /*max_sequence_number=*/-1, path_factory, compact_manager_, + memory_pool_); arrow::StringBuilder builder; for (size_t j = 0; j < 100; j++) { ASSERT_TRUE(builder.Append(std::to_string(j)).ok()); @@ -153,8 +153,8 @@ TEST_F(AppendOnlyWriterTest, TestWriteAndClose) { auto path_factory = std::make_shared(); ASSERT_OK(path_factory->Init(dir->Str(), "orc", options.DataFilePrefix(), nullptr)); AppendOnlyWriter writer(options, /*schema_id=*/1, schema, /*write_cols=*/std::nullopt, - /*max_sequence_number=*/-1, path_factory, memory_pool_, - compact_manager_); + /*max_sequence_number=*/-1, path_factory, compact_manager_, + memory_pool_); auto struct_type = arrow::struct_(fields); arrow::StructBuilder struct_builder(struct_type, arrow::default_memory_pool(), {std::make_shared()}); @@ -197,8 +197,8 @@ TEST_F(AppendOnlyWriterTest, TestInvalidRowKind) { auto path_factory = std::make_shared(); ASSERT_OK(path_factory->Init(dir->Str(), "orc", options.DataFilePrefix(), nullptr)); AppendOnlyWriter writer(options, /*schema_id=*/1, schema, /*write_cols=*/std::nullopt, - /*max_sequence_number=*/-1, path_factory, memory_pool_, - compact_manager_); + /*max_sequence_number=*/-1, path_factory, compact_manager_, + memory_pool_); auto struct_type = arrow::struct_(fields); arrow::StructBuilder struct_builder(struct_type, arrow::default_memory_pool(), {std::make_shared()}); diff --git a/src/paimon/core/append/bucketed_append_compact_manager.cpp b/src/paimon/core/append/bucketed_append_compact_manager.cpp index 819232cc..d40dce16 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager.cpp @@ -161,6 +161,24 @@ BucketedAppendCompactManager::GetCompactionResult(bool blocking) { return result; } +Result> BucketedAppendCompactManager::Compact( + const std::shared_ptr& dv_maintainer, + const std::vector>& to_compact, CompactRewriter rewriter) { + PAIMON_ASSIGN_OR_RAISE(std::vector> rewrite, + rewriter(to_compact)); + + auto result = std::make_shared(to_compact, rewrite); + if (dv_maintainer != nullptr) { + for (const auto& file : to_compact) { + dv_maintainer->RemoveDeletionVectorOf(file->file_name); + } + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr deletion_file, + CompactDeletionFile::GenerateFiles(dv_maintainer)); + result->SetDeletionFile(deletion_file); + } + return result; +} + Result> BucketedAppendCompactManager::FullCompactTask::DoCompact() { // remove large files while (!force_rewrite_all_files_ && !to_compact_.empty()) { diff --git a/src/paimon/core/append/bucketed_append_compact_manager.h b/src/paimon/core/append/bucketed_append_compact_manager.h index 309d45ab..6076a08a 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.h +++ b/src/paimon/core/append/bucketed_append_compact_manager.h @@ -113,21 +113,7 @@ class BucketedAppendCompactManager : public CompactFutureManager { static Result> Compact( const std::shared_ptr& dv_maintainer, - const std::vector>& to_compact, CompactRewriter rewriter) { - PAIMON_ASSIGN_OR_RAISE(std::vector> rewrite, - rewriter(to_compact)); - - auto result = std::make_shared(to_compact, rewrite); - if (dv_maintainer != nullptr) { - for (const auto& file : to_compact) { - dv_maintainer->RemoveDeletionVectorOf(file->file_name); - } - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr deletion_file, - CompactDeletionFile::GenerateFiles(dv_maintainer)); - result->SetDeletionFile(deletion_file); - } - return result; - } + const std::vector>& to_compact, CompactRewriter rewriter); /// A `CompactTask` impl for full compaction of append-only table. class FullCompactTask : public CompactTask { diff --git a/src/paimon/core/compact/compact_deletion_file.h b/src/paimon/core/compact/compact_deletion_file.h index b13a7a59..82db8380 100644 --- a/src/paimon/core/compact/compact_deletion_file.h +++ b/src/paimon/core/compact/compact_deletion_file.h @@ -54,7 +54,6 @@ class GeneratedDeletionFile : public CompactDeletionFile, : std::nullopt; } - // TODO(yonghao.fyh): check logic Result> MergeOldFile( const std::shared_ptr& old) override { auto derived = dynamic_cast(old.get()); diff --git a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h index 53fb955b..9a5c6967 100644 --- a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h +++ b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h @@ -37,7 +37,8 @@ class BucketedDvMaintainer { deletion_vectors_(deletion_vectors), bitmap64_(dv_index_file->Bitmap64()) {} - std::optional> DeletionVectorOf(const std::string& file_name) { + std::optional> DeletionVectorOf( + const std::string& file_name) const { if (auto it = deletion_vectors_.find(file_name); it != deletion_vectors_.end()) { return it->second; } 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 894d6b86..cae9ab6d 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -19,8 +19,8 @@ #include #include "paimon/common/data/binary_row.h" -#include "paimon/common/reader/reader_utils.h" #include "paimon/common/table/special_fields.h" +#include "paimon/common/utils/arrow/arrow_utils.h" #include "paimon/core/append/append_only_writer.h" #include "paimon/core/append/bucketed_append_compact_manager.h" #include "paimon/core/compact/noop_compact_manager.h" @@ -111,22 +111,26 @@ Result>> AppendOnlyFileStoreWrite::Com return std::vector>{}; } + // TODO(yonghao.fyh): support dv factory + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr reader, + CreateFilesReader(partition, bucket, to_compact)); auto rewriter = std::make_unique>>( options_.GetTargetFileSize(/*has_primary_key=*/false), GetDataFileWriterCreator(partition, bucket, write_schema_, write_cols_, to_compact)); - // TODO(yonghao.fyh): support dv factory - PAIMON_ASSIGN_OR_RAISE(std::unique_ptr reader, - CreateFilesReader(partition, bucket, to_compact)); - ScopeGuard guard([&]() { - if (rewriter) { - (void)rewriter->Close(); - } + + ScopeGuard reader_guard([&]() { if (reader) { reader->Close(); } }); + ScopeGuard rewriter_guard([&]() { + if (rewriter) { + (void)rewriter->Close(); + } + }); + while (true) { PAIMON_ASSIGN_OR_RAISE(BatchReader::ReadBatch batch, reader->NextBatch()); if (BatchReader::IsEofBatch(batch)) { @@ -140,16 +144,17 @@ Result>> AppendOnlyFileStoreWrite::Com return Status::Invalid( "cannot cast array to StructArray in CompleteRowKindBatchReader"); } - PAIMON_ASSIGN_OR_RAISE(struct_array, ReaderUtils::RemoveFieldFromStructArray( + PAIMON_ASSIGN_OR_RAISE(struct_array, ArrowUtils::RemoveFieldFromStructArray( struct_array, SpecialFields::ValueKind().Name())); PAIMON_RETURN_NOT_OK_FROM_ARROW( arrow::ExportArray(*struct_array, c_array.get(), c_schema.get())); ScopeGuard guard([schema = c_schema.get()]() { ArrowSchemaRelease(schema); }); PAIMON_RETURN_NOT_OK(rewriter->Write(c_array.get())); } - guard.Release(); - reader->Close(); + rewriter_guard.Release(); PAIMON_RETURN_NOT_OK(rewriter->Close()); + reader_guard.Release(); + reader->Close(); return rewriter->GetResult(); } @@ -193,7 +198,7 @@ Result>> AppendOnlyFileStoreWrit auto writer = std::make_shared( options_, table_schema_->Id(), write_schema_, write_cols_, max_sequence_number, - data_file_path_factory, pool_, compact_manager); + data_file_path_factory, compact_manager, pool_); return std::pair>(total_buckets, writer); } @@ -221,7 +226,6 @@ AppendOnlyFileStoreWrite::GetDataFileWriterCreator( PAIMON_ASSIGN_OR_RAISE( std::shared_ptr data_file_path_factory, file_store_path_factory_->CreateDataFilePathFactory(partition, bucket)); - // TODO(yonghao.fyh): check sequence number auto writer = std::make_unique( options_.GetFileCompression(), std::function(), table_schema_->Id(), @@ -238,7 +242,7 @@ Result> AppendOnlyFileStoreWrite::CreateFilesReader const BinaryRow& partition, int32_t bucket, const std::vector>& files) const { ReadContextBuilder context_builder(root_path_); - context_builder.EnablePrefetch(false).SetPrefetchMaxParallelNum(1); + context_builder.EnablePrefetch(true).SetPrefetchMaxParallelNum(1); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr read_context, context_builder.Finish()); std::map map = options_.ToMap(); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr internal_read_context, From 4fe86b8a613219fe1e6aebefd8a24c2b2ec5972b Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 11 Mar 2026 15:32:20 +0800 Subject: [PATCH 20/21] fix --- src/paimon/core/compact/compact_deletion_file.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/paimon/core/compact/compact_deletion_file.h b/src/paimon/core/compact/compact_deletion_file.h index 82db8380..1a66de74 100644 --- a/src/paimon/core/compact/compact_deletion_file.h +++ b/src/paimon/core/compact/compact_deletion_file.h @@ -36,7 +36,7 @@ class CompactDeletionFile { virtual std::optional> GetOrCompute() = 0; virtual Result> MergeOldFile( - const std::shared_ptr& old) = 0; + const std::shared_ptr& old) const = 0; virtual void Clean() = 0; }; @@ -55,7 +55,7 @@ class GeneratedDeletionFile : public CompactDeletionFile, } Result> MergeOldFile( - const std::shared_ptr& old) override { + const std::shared_ptr& old) const override { auto derived = dynamic_cast(old.get()); if (derived == nullptr) { return Status::Invalid("old should be a GeneratedDeletionFile, but it is not"); From 3f706f2de6428acb1ac2a712b9020f1521ad2219 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 11 Mar 2026 15:36:25 +0800 Subject: [PATCH 21/21] fix --- src/paimon/core/compact/compact_deletion_file.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/paimon/core/compact/compact_deletion_file.h b/src/paimon/core/compact/compact_deletion_file.h index 1a66de74..82db8380 100644 --- a/src/paimon/core/compact/compact_deletion_file.h +++ b/src/paimon/core/compact/compact_deletion_file.h @@ -36,7 +36,7 @@ class CompactDeletionFile { virtual std::optional> GetOrCompute() = 0; virtual Result> MergeOldFile( - const std::shared_ptr& old) const = 0; + const std::shared_ptr& old) = 0; virtual void Clean() = 0; }; @@ -55,7 +55,7 @@ class GeneratedDeletionFile : public CompactDeletionFile, } Result> MergeOldFile( - const std::shared_ptr& old) const override { + const std::shared_ptr& old) override { auto derived = dynamic_cast(old.get()); if (derived == nullptr) { return Status::Invalid("old should be a GeneratedDeletionFile, but it is not");