From cb0a78398a0cedb266a89bc62862320d9a7425ef Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Thu, 12 Mar 2026 16:19:05 +0800 Subject: [PATCH 01/27] feat(compaction): support append table compaction with dv --- include/paimon/defs.h | 8 +++ src/paimon/common/defs.cpp | 3 + src/paimon/core/core_options.cpp | 15 ++++ src/paimon/core/core_options.h | 2 + src/paimon/core/core_options_test.cpp | 6 ++ .../apply_deletion_vector_batch_reader.h | 6 +- .../bitmap64_deletion_vector.h | 28 ++++++++ .../deletionvectors/bucketed_dv_maintainer.h | 32 +++++++++ .../core/deletionvectors/deletion_vector.cpp | 34 +++++++++ .../core/deletionvectors/deletion_vector.h | 8 +++ .../deletion_vectors_index_file.cpp | 43 +++++++++++ .../deletion_vectors_index_file.h | 16 +++++ .../global_index/global_index_scan_impl.cpp | 4 +- src/paimon/core/index/index_file_handler.cpp | 13 ++++ src/paimon/core/index/index_file_handler.h | 39 +++++++++- .../core/index/index_file_handler_test.cpp | 4 +- .../operation/abstract_file_store_write.cpp | 71 ++++++++++++------- .../operation/abstract_file_store_write.h | 34 +++++---- .../core/operation/abstract_split_read.cpp | 10 ++- .../core/operation/abstract_split_read.h | 10 ++- .../append_only_file_store_write.cpp | 63 ++++++++-------- .../operation/append_only_file_store_write.h | 37 +++++----- .../operation/data_evolution_split_read.cpp | 5 +- .../operation/data_evolution_split_read.h | 3 +- .../core/operation/file_store_write.cpp | 33 +++++++-- .../operation/file_system_write_restore.h | 31 +++++--- .../operation/key_value_file_store_write.cpp | 27 +++---- .../operation/key_value_file_store_write.h | 8 ++- .../core/operation/merge_file_split_read.cpp | 62 ++++++++++------ .../core/operation/merge_file_split_read.h | 11 ++- .../core/operation/raw_file_split_read.cpp | 35 ++++++--- .../core/operation/raw_file_split_read.h | 9 ++- src/paimon/core/operation/write_restore.h | 3 +- .../postpone_bucket_file_store_write.h | 32 +++++---- src/paimon/core/table/source/table_scan.cpp | 4 +- 35 files changed, 546 insertions(+), 203 deletions(-) create mode 100644 src/paimon/core/deletionvectors/bitmap64_deletion_vector.h diff --git a/include/paimon/defs.h b/include/paimon/defs.h index f542605c..b09a0496 100644 --- a/include/paimon/defs.h +++ b/include/paimon/defs.h @@ -232,6 +232,14 @@ struct PAIMON_EXPORT Options { /// Default value is false. static const char DELETION_VECTORS_ENABLED[]; + /// "deletion-vector.index-file.target-size" - The target size of deletion vector index file. + /// Default value is 2MB. + static const char DELETION_VECTOR_INDEX_FILE_TARGET_SIZE[]; + + /// "deletion-vectors.bitmap64" - Enable 64 bit bitmap implementation. Note that only 64 bit + /// bitmap implementation is compatible with Iceberg. Default value is "false". + static const char DELETION_VECTOR_BITMAP64[]; + /// @note `CHANGELOG_PRODUCER` currently only support `none` /// /// "changelog-producer" - Whether to double write to a changelog file. This changelog file diff --git a/src/paimon/common/defs.cpp b/src/paimon/common/defs.cpp index 91e1ee0e..b80452e3 100644 --- a/src/paimon/common/defs.cpp +++ b/src/paimon/common/defs.cpp @@ -63,6 +63,9 @@ const char Options::SORT_ENGINE[] = "sort-engine"; const char Options::IGNORE_DELETE[] = "ignore-delete"; const char Options::FIELDS_DEFAULT_AGG_FUNC[] = "fields.default-aggregate-function"; const char Options::DELETION_VECTORS_ENABLED[] = "deletion-vectors.enabled"; +const char Options::DELETION_VECTOR_INDEX_FILE_TARGET_SIZE[] = + "deletion-vector.index-file.target-size"; +const char Options::DELETION_VECTOR_BITMAP64[] = "deletion-vectors.bitmap64"; const char Options::CHANGELOG_PRODUCER[] = "changelog-producer"; const char Options::FORCE_LOOKUP[] = "force-lookup"; const char Options::PARTIAL_UPDATE_REMOVE_RECORD_ON_DELETE[] = diff --git a/src/paimon/core/core_options.cpp b/src/paimon/core/core_options.cpp index 150cce7f..5f9c26c1 100644 --- a/src/paimon/core/core_options.cpp +++ b/src/paimon/core/core_options.cpp @@ -264,6 +264,7 @@ struct CoreOptions::Impl { int64_t source_split_target_size = 128 * 1024 * 1024; int64_t source_split_open_file_cost = 4 * 1024 * 1024; int64_t manifest_target_file_size = 8 * 1024 * 1024; + int64_t deletion_vector_target_file_size = 2 * 1024 * 1024; int64_t manifest_full_compaction_file_size = 16 * 1024 * 1024; int64_t write_buffer_size = 256 * 1024 * 1024; int64_t commit_timeout = std::numeric_limits::max(); @@ -310,6 +311,7 @@ struct CoreOptions::Impl { bool ignore_delete = false; bool write_only = false; bool deletion_vectors_enabled = false; + bool deletion_vectors_bitmap64 = false; bool force_lookup = false; bool partial_update_remove_record_on_delete = false; bool file_index_read_enabled = true; @@ -448,6 +450,12 @@ Result CoreOptions::FromMap( PAIMON_RETURN_NOT_OK( parser.Parse(Options::DELETION_VECTORS_ENABLED, &impl->deletion_vectors_enabled)); PAIMON_RETURN_NOT_OK(parser.Parse(Options::FORCE_LOOKUP, &impl->force_lookup)); + + PAIMON_RETURN_NOT_OK(parser.ParseMemorySize(Options::DELETION_VECTOR_INDEX_FILE_TARGET_SIZE, + &impl->deletion_vector_target_file_size)); + PAIMON_RETURN_NOT_OK( + parser.Parse(Options::DELETION_VECTOR_BITMAP64, &impl->deletion_vectors_bitmap64)); + // Parse changelog producer PAIMON_RETURN_NOT_OK(parser.ParseChangelogProducer(&impl->changelog_producer)); @@ -783,6 +791,13 @@ bool CoreOptions::DeletionVectorsEnabled() const { return impl_->deletion_vectors_enabled; } +bool CoreOptions::DeletionVectorsBitmap64() const { + return impl_->deletion_vectors_bitmap64; +} +int64_t CoreOptions::DeletionVectorTargetFileSize() const { + return impl_->deletion_vector_target_file_size; +} + ChangelogProducer CoreOptions::GetChangelogProducer() const { return impl_->changelog_producer; } diff --git a/src/paimon/core/core_options.h b/src/paimon/core/core_options.h index 7765b53a..44796334 100644 --- a/src/paimon/core/core_options.h +++ b/src/paimon/core/core_options.h @@ -97,6 +97,8 @@ class PAIMON_EXPORT CoreOptions { Result> GetFieldAggFunc(const std::string& field_name) const; Result FieldAggIgnoreRetract(const std::string& field_name) const; bool DeletionVectorsEnabled() const; + bool DeletionVectorsBitmap64() const; + int64_t DeletionVectorTargetFileSize() const; ChangelogProducer GetChangelogProducer() const; bool NeedLookup() const; bool FileIndexReadEnabled() const; diff --git a/src/paimon/core/core_options_test.cpp b/src/paimon/core/core_options_test.cpp index 50ced4c0..01ec10d5 100644 --- a/src/paimon/core/core_options_test.cpp +++ b/src/paimon/core/core_options_test.cpp @@ -78,6 +78,8 @@ TEST(CoreOptionsTest, TestDefaultValue) { ASSERT_EQ(std::nullopt, core_options.GetFieldAggFunc("f0").value()); ASSERT_FALSE(core_options.FieldAggIgnoreRetract("f1").value()); ASSERT_FALSE(core_options.DeletionVectorsEnabled()); + ASSERT_FALSE(core_options.DeletionVectorsBitmap64()); + ASSERT_EQ(2 * 1024 * 1024, core_options.DeletionVectorTargetFileSize()); ASSERT_EQ(ChangelogProducer::NONE, core_options.GetChangelogProducer()); ASSERT_FALSE(core_options.NeedLookup()); ASSERT_TRUE(core_options.GetFieldsSequenceGroups().empty()); @@ -146,6 +148,8 @@ TEST(CoreOptionsTest, TestFromMap) { {"fields.f0.aggregate-function", "min"}, {"fields.f1.ignore-retract", "true"}, {Options::DELETION_VECTORS_ENABLED, "true"}, + {Options::DELETION_VECTOR_BITMAP64, "true"}, + {Options::DELETION_VECTOR_INDEX_FILE_TARGET_SIZE, "4MB"}, {Options::CHANGELOG_PRODUCER, "full-compaction"}, {Options::FORCE_LOOKUP, "true"}, {"fields.g_1,g_3.sequence-group", "c,d"}, @@ -227,6 +231,8 @@ TEST(CoreOptionsTest, TestFromMap) { ASSERT_TRUE(core_options.FieldAggIgnoreRetract("f1").value()); ASSERT_TRUE(core_options.FieldAggIgnoreRetract("f1").value()); ASSERT_TRUE(core_options.DeletionVectorsEnabled()); + ASSERT_TRUE(core_options.DeletionVectorsBitmap64()); + ASSERT_EQ(4 * 1024 * 1024, core_options.DeletionVectorTargetFileSize()); ASSERT_EQ(ChangelogProducer::FULL_COMPACTION, core_options.GetChangelogProducer()); ASSERT_TRUE(core_options.NeedLookup()); std::map seq_grp; diff --git a/src/paimon/core/deletionvectors/apply_deletion_vector_batch_reader.h b/src/paimon/core/deletionvectors/apply_deletion_vector_batch_reader.h index 724be9dc..5e2ecf27 100644 --- a/src/paimon/core/deletionvectors/apply_deletion_vector_batch_reader.h +++ b/src/paimon/core/deletionvectors/apply_deletion_vector_batch_reader.h @@ -37,8 +37,8 @@ class Metrics; class ApplyDeletionVectorBatchReader : public FileBatchReader { public: ApplyDeletionVectorBatchReader(std::unique_ptr&& reader, - PAIMON_UNIQUE_PTR&& deletion_vector) - : reader_(std::move(reader)), deletion_vector_(std::move(deletion_vector)) { + const std::shared_ptr& deletion_vector) + : reader_(std::move(reader)), deletion_vector_(deletion_vector) { assert(reader_); } @@ -103,6 +103,6 @@ class ApplyDeletionVectorBatchReader : public FileBatchReader { private: std::unique_ptr reader_; - PAIMON_UNIQUE_PTR deletion_vector_; + std::shared_ptr deletion_vector_; }; } // namespace paimon diff --git a/src/paimon/core/deletionvectors/bitmap64_deletion_vector.h b/src/paimon/core/deletionvectors/bitmap64_deletion_vector.h new file mode 100644 index 00000000..822da422 --- /dev/null +++ b/src/paimon/core/deletionvectors/bitmap64_deletion_vector.h @@ -0,0 +1,28 @@ +/* + * 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/deletionvectors/deletion_vector.h" + +namespace paimon { + +class Bitmap64DeletionVector : public DeletionVector { + public: + static constexpr int32_t MAGIC_NUMBER = 1681511377; +}; + +} // namespace paimon diff --git a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h index 9a5c6967..d7ccf112 100644 --- a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h +++ b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h @@ -23,6 +23,7 @@ #include "paimon/core/deletionvectors/deletion_vector.h" #include "paimon/core/deletionvectors/deletion_vectors_index_file.h" +#include "paimon/core/index/index_file_handler.h" #include "paimon/core/index/index_file_meta.h" namespace paimon { @@ -65,6 +66,37 @@ class BucketedDvMaintainer { return dv_index_file_; } + /// Factory to restore `BucketedDvMaintainer`. + class Factory { + public: + explicit Factory(const std::shared_ptr& index_file_handler) + : handler_(index_file_handler) {} + + std::shared_ptr GetIndexFileHandler() const { + return handler_; + } + + Result> Create( + const BinaryRow& partition, int32_t bucket, + const std::vector>& restored_files) { + std::map> deletion_vectors; + PAIMON_ASSIGN_OR_RAISE(deletion_vectors, handler_->ReadAllDeletionVectors( + partition, bucket, restored_files)); + return Create(partition, bucket, deletion_vectors); + } + + Result> Create( + const BinaryRow& partition, int32_t bucket, + const std::map>& deletion_vectors) { + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr dv_index_file, + handler_->DvIndex(partition, bucket)); + return std::make_unique(dv_index_file, deletion_vectors); + } + + private: + std::shared_ptr handler_; + }; + private: std::shared_ptr dv_index_file_; std::map> deletion_vectors_; diff --git a/src/paimon/core/deletionvectors/deletion_vector.cpp b/src/paimon/core/deletionvectors/deletion_vector.cpp index e11a125d..44e76382 100644 --- a/src/paimon/core/deletionvectors/deletion_vector.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector.cpp @@ -20,6 +20,7 @@ #include #include "fmt/format.h" +#include "paimon/core/deletionvectors/bitmap64_deletion_vector.h" #include "paimon/core/deletionvectors/bitmap_deletion_vector.h" #include "paimon/core/table/source/deletion_file.h" #include "paimon/fs/file_system.h" @@ -51,6 +52,39 @@ Result> DeletionVector::Read(const FileSystem* return DeserializeFromBytes(bytes.get(), pool); } +Result> DeletionVector::Read(DataInputStream* input_stream, + std::optional length, + MemoryPool* pool) { + PAIMON_ASSIGN_OR_RAISE(int32_t bitmap_length, input_stream->ReadValue()); + PAIMON_ASSIGN_OR_RAISE(int32_t magic_number, input_stream->ReadValue()); + + if (magic_number == BitmapDeletionVector::MAGIC_NUMBER) { + if (length.has_value() && bitmap_length != length.value()) { + return Status::Invalid(fmt::format("Size not match, actual size: {}, expected size: {}", + bitmap_length, length.value())); + } + + int32_t payload_length = bitmap_length - BitmapDeletionVector::MAGIC_NUMBER_SIZE_BYTES; + if (payload_length < 0) { + return Status::Invalid(fmt::format("Invalid bitmap length: {}", bitmap_length)); + } + + auto bytes = Bytes::AllocateBytes(payload_length, pool); + PAIMON_RETURN_NOT_OK(input_stream->ReadBytes(bytes.get())); + // skip crc (4 bytes) + PAIMON_ASSIGN_OR_RAISE([[maybe_unused]] int32_t unused_crc, + input_stream->ReadValue()); + + return BitmapDeletionVector::Deserialize(bytes->data(), bytes->size(), pool); + } else if (EndianSwapValue(magic_number) == Bitmap64DeletionVector::MAGIC_NUMBER) { + return Status::NotImplemented("bitmap64 dv is not implemented"); + } + + return Status::Invalid(fmt::format( + "Invalid magic number: {}, v1 dv magic number: {}, v2 magic number: {}", magic_number, + BitmapDeletionVector::MAGIC_NUMBER, Bitmap64DeletionVector::MAGIC_NUMBER)); +} + PAIMON_UNIQUE_PTR DeletionVector::FromPrimitiveArray( const std::vector& is_deleted, MemoryPool* pool) { RoaringBitmap32 roaring; diff --git a/src/paimon/core/deletionvectors/deletion_vector.h b/src/paimon/core/deletionvectors/deletion_vector.h index 48368ba6..91bcc67e 100644 --- a/src/paimon/core/deletionvectors/deletion_vector.h +++ b/src/paimon/core/deletionvectors/deletion_vector.h @@ -23,6 +23,7 @@ #include "paimon/core/io/data_file_meta.h" #include "paimon/core/table/source/deletion_file.h" +#include "paimon/io/data_input_stream.h" #include "paimon/memory/bytes.h" #include "paimon/memory/memory_pool.h" #include "paimon/result.h" @@ -38,6 +39,8 @@ struct DeletionFile; /// which can then be used to filter out deleted rows when processing the file. class DeletionVector { public: + using Factory = std::function>(const std::string&)>; + virtual ~DeletionVector() = default; /// Marks the row at the specified position as deleted. @@ -97,7 +100,12 @@ class DeletionVector { const DeletionFile& deletion_file, MemoryPool* pool); + static Result> Read(DataInputStream* input_stream, + std::optional length, + MemoryPool* pool); + static PAIMON_UNIQUE_PTR FromPrimitiveArray(const std::vector& is_deleted, MemoryPool* pool); }; + } // namespace paimon diff --git a/src/paimon/core/deletionvectors/deletion_vectors_index_file.cpp b/src/paimon/core/deletionvectors/deletion_vectors_index_file.cpp index a4323baa..d9265a07 100644 --- a/src/paimon/core/deletionvectors/deletion_vectors_index_file.cpp +++ b/src/paimon/core/deletionvectors/deletion_vectors_index_file.cpp @@ -16,6 +16,11 @@ #include "paimon/core/deletionvectors/deletion_vectors_index_file.h" +#include +#include +#include +#include + #include "paimon/core/deletionvectors/deletion_vector_index_file_writer.h" namespace paimon { @@ -30,4 +35,42 @@ std::shared_ptr DeletionVectorsIndexFile::CreateW target_size_per_index_file_, pool_); } +Result>> +DeletionVectorsIndexFile::ReadAllDeletionVectors( + const std::shared_ptr& file_meta) const { + std::optional> deletion_vector_metas = + file_meta->DvRanges(); + if (deletion_vector_metas == std::nullopt) { + return Status::Invalid("deletion vector metas is null"); + } + + std::map> deletion_vectors; + std::string file_path = path_factory_->ToPath(file_meta); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr input_stream, fs_->Open(file_path)); + auto data_input_stream = std::make_shared(input_stream); + for (const auto& [_, deletion_vector_meta] : deletion_vector_metas.value()) { + // TODO(yonghao.fyh): check deletion_vector_meta.length = -1? + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr dv, + DeletionVector::Read(data_input_stream.get(), + static_cast(deletion_vector_meta.length), pool_.get())); + deletion_vectors[deletion_vector_meta.data_file_name] = dv; + } + return deletion_vectors; +} + +Result>> +DeletionVectorsIndexFile::ReadAllDeletionVectors( + const std::vector>& index_files) const { + std::map> deletion_vectors; + for (const auto& index_file : index_files) { + std::map> partial_deletion_vectors; + PAIMON_ASSIGN_OR_RAISE(partial_deletion_vectors, ReadAllDeletionVectors(index_file)); + for (const auto& kv : partial_deletion_vectors) { + deletion_vectors[kv.first] = kv.second; + } + } + return deletion_vectors; +} + } // namespace paimon diff --git a/src/paimon/core/deletionvectors/deletion_vectors_index_file.h b/src/paimon/core/deletionvectors/deletion_vectors_index_file.h index 18b3da35..70912e73 100644 --- a/src/paimon/core/deletionvectors/deletion_vectors_index_file.h +++ b/src/paimon/core/deletionvectors/deletion_vectors_index_file.h @@ -52,9 +52,25 @@ class DeletionVectorsIndexFile : public IndexFile { Result> WriteSingleFile( const std::map>& input); + Result>> ReadAllDeletionVectors( + const std::shared_ptr& file_meta) const; + + Result>> ReadAllDeletionVectors( + const std::vector>& index_files) const; + private: std::shared_ptr CreateWriter() const; + static Status CheckVersion(const std::shared_ptr& in) { + PAIMON_ASSIGN_OR_RAISE(int8_t version, in->ReadValue()); + if (version != VERSION_ID_V1) { + return Status::Invalid( + fmt::format("Version not match, actual version: {}, expected version: {}", version, + VERSION_ID_V1)); + } + return Status::OK(); + } + const int64_t target_size_per_index_file_; const bool bitmap64_; std::shared_ptr pool_; diff --git a/src/paimon/core/global_index/global_index_scan_impl.cpp b/src/paimon/core/global_index/global_index_scan_impl.cpp index 2420108f..bfa3f966 100644 --- a/src/paimon/core/global_index/global_index_scan_impl.cpp +++ b/src/paimon/core/global_index/global_index_scan_impl.cpp @@ -119,7 +119,9 @@ Status GlobalIndexScanImpl::Scan() { options_.GetFileSystem(), options_.GetManifestFormat(), options_.GetManifestCompression(), path_factory_, pool_, options_)); auto index_file_handler = std::make_unique( - std::move(index_manifest_file), std::make_shared(path_factory_)); + options_.GetFileSystem(), std::move(index_manifest_file), + std::make_shared(path_factory_), + options_.DeletionVectorTargetFileSize(), options_.DeletionVectorsBitmap64()); PAIMON_ASSIGN_OR_RAISE(std::vector partition_fields, table_schema_->GetFields(table_schema_->PartitionKeys())); diff --git a/src/paimon/core/index/index_file_handler.cpp b/src/paimon/core/index/index_file_handler.cpp index c4ed9846..c9db7bc5 100644 --- a/src/paimon/core/index/index_file_handler.cpp +++ b/src/paimon/core/index/index_file_handler.cpp @@ -57,4 +57,17 @@ Result> IndexFileHandler::Scan( return index_entries; } +Result>> IndexFileHandler::Scan( + const Snapshot& snapshot, const std::string& index_type, const BinaryRow& partition, + int32_t bucket) const { + PAIMON_ASSIGN_OR_RAISE(IndexFileHandler::IndexFileMetaGroups index_file_meta_groups, + Scan(snapshot, index_type, {partition})); + std::pair key(partition, bucket); + auto iter = index_file_meta_groups.find(key); + if (iter != index_file_meta_groups.end()) { + return iter->second; + } + return std::vector>{}; +} + } // namespace paimon diff --git a/src/paimon/core/index/index_file_handler.h b/src/paimon/core/index/index_file_handler.h index 3757b894..fd221e13 100644 --- a/src/paimon/core/index/index_file_handler.h +++ b/src/paimon/core/index/index_file_handler.h @@ -15,6 +15,7 @@ */ #pragma once + #include #include #include @@ -24,6 +25,7 @@ #include #include "paimon/common/data/binary_row.h" +#include "paimon/core/deletionvectors/deletion_vectors_index_file.h" #include "paimon/core/index/index_file_meta.h" #include "paimon/core/index/index_path_factory.h" #include "paimon/core/manifest/index_manifest_entry.h" @@ -40,14 +42,25 @@ class IndexFileHandler { using IndexFileMetaGroups = std::unordered_map, std::vector>>; - IndexFileHandler(std::unique_ptr&& index_manifest_file, - const std::shared_ptr& path_factories) - : index_manifest_file_(std::move(index_manifest_file)), path_factories_(path_factories) {} + IndexFileHandler(const std::shared_ptr& fs, + std::unique_ptr&& index_manifest_file, + const std::shared_ptr& path_factories, + int64_t dv_target_file_size, bool dv_bitmap64) + : fs_(fs), + index_manifest_file_(std::move(index_manifest_file)), + path_factories_(path_factories), + dv_target_file_size_(dv_target_file_size), + dv_bitmap64_(dv_bitmap64) {} /// 1.Scan specified index_type index. 2.Cluster with partition & bucket. Result Scan(const Snapshot& snapshot, const std::string& index_type, const std::unordered_set& partitions) const; + Result>> Scan(const Snapshot& snapshot, + const std::string& index_type, + const BinaryRow& partition, + int32_t bucket) const; + /// Scan specified all typed index. Result> Scan( const Snapshot& snapshot, @@ -60,8 +73,28 @@ class IndexFileHandler { return factory->ToPath(file); } + Result> DvIndex(const BinaryRow& partition, + int32_t bucket) const { + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr index_path_factory, + path_factories_->Get(partition, bucket)); + return std::make_unique( + fs_, index_path_factory, dv_target_file_size_, dv_bitmap64_, pool_); + } + + Result>> ReadAllDeletionVectors( + const BinaryRow& partition, int32_t bucket, + const std::vector>& file_metas) { + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr dv_index, + DvIndex(partition, bucket)); + return dv_index->ReadAllDeletionVectors(file_metas); + } + private: + std::shared_ptr fs_; std::unique_ptr index_manifest_file_; std::shared_ptr path_factories_; + int64_t dv_target_file_size_; + bool dv_bitmap64_; + std::shared_ptr pool_; }; } // namespace paimon diff --git a/src/paimon/core/index/index_file_handler_test.cpp b/src/paimon/core/index/index_file_handler_test.cpp index 50d29188..4c713667 100644 --- a/src/paimon/core/index/index_file_handler_test.cpp +++ b/src/paimon/core/index/index_file_handler_test.cpp @@ -73,7 +73,9 @@ class IndexFileHandlerTest : public testing::Test { core_options.GetManifestCompression(), path_factory, memory_pool_, core_options)); auto path_factories = std::make_shared(path_factory); - return std::make_unique(std::move(index_manifest_file), path_factories); + return std::make_unique( + core_options.GetFileSystem(), std::move(index_manifest_file), path_factories, + core_options.DeletionVectorTargetFileSize(), core_options.DeletionVectorsBitmap64()); } std::shared_ptr memory_pool_; }; diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index 2a8af44e..565219e3 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -56,9 +56,11 @@ AbstractFileStoreWrite::AbstractFileStoreWrite( const std::string& root_path, const std::shared_ptr& table_schema, const std::shared_ptr& schema, const std::shared_ptr& write_schema, - const std::shared_ptr& partition_schema, const CoreOptions& options, - bool ignore_previous_files, bool is_streaming_mode, bool ignore_num_bucket_check, - const std::shared_ptr& executor, const std::shared_ptr& pool) + const std::shared_ptr& partition_schema, + const std::shared_ptr& dv_maintainer_factory, + const CoreOptions& options, bool ignore_previous_files, bool is_streaming_mode, + bool ignore_num_bucket_check, const std::shared_ptr& executor, + const std::shared_ptr& pool) : pool_(pool), executor_(executor), file_store_path_factory_(file_store_path_factory), @@ -70,6 +72,7 @@ AbstractFileStoreWrite::AbstractFileStoreWrite( write_schema_(write_schema), table_schema_(table_schema), partition_schema_(partition_schema), + dv_maintainer_factory_(dv_maintainer_factory), options_(options), ignore_previous_files_(ignore_previous_files), is_streaming_mode_(is_streaming_mode), @@ -274,9 +277,14 @@ Result> AbstractFileStoreWrite::ScanExistingFileMe /*vector_search=*/nullptr); PAIMON_ASSIGN_OR_RAISE(std::unique_ptr scan, CreateFileStoreScan(scan_filter)); - // TODO(yonghao.fyh): create index file handler - FileSystemWriteRestore restore(snapshot_manager_, std::move(scan)); - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, restore.GetRestoreFiles()); + std::shared_ptr index_file_handler; + if (dv_maintainer_factory_) { + index_file_handler = dv_maintainer_factory_->GetIndexFileHandler(); + } + FileSystemWriteRestore restore(snapshot_manager_, std::move(scan), index_file_handler); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr restore_files, + restore.GetRestoreFiles(partition, bucket, dv_maintainer_factory_ != nullptr)); std::optional restored_total_buckets = restore_files->TotalBuckets(); int32_t total_buckets = GetDefaultBucketNum(); @@ -296,30 +304,43 @@ Result> AbstractFileStoreWrite::ScanExistingFileMe Result> AbstractFileStoreWrite::GetWriter(const BinaryRow& partition, int32_t bucket) { - auto iter = writers_.find(partition); - if (PAIMON_UNLIKELY(iter == writers_.end())) { - PAIMON_ASSIGN_OR_RAISE(auto result, - CreateWriter(partition, bucket, ignore_previous_files_)); - int32_t total_buckets = result.first; - std::shared_ptr writer = result.second; + auto partition_iter = writers_.find(partition); + if (partition_iter != writers_.end()) { + auto& buckets = partition_iter->second; + auto bucket_iter = buckets.find(bucket); + if (PAIMON_LIKELY(bucket_iter != buckets.end())) { + return bucket_iter->second.writer; + } + } + + std::shared_ptr restored = RestoreFiles::Empty(); + if (!ignore_previous_files_) { + PAIMON_ASSIGN_OR_RAISE(restored, ScanExistingFileMetas(partition, bucket)); + } + + auto restore_data_files = restored->DataFiles(); + int64_t max_sequence_number = DataFileMeta::GetMaxSequenceNumber(restore_data_files); + std::shared_ptr dv_maintainer; + if (dv_maintainer_factory_) { + PAIMON_ASSIGN_OR_RAISE( + dv_maintainer, + dv_maintainer_factory_->Create(partition, bucket, restored->DeleteVectorsIndex())); + } + + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr writer, + CreateWriter(partition, bucket, restore_data_files, max_sequence_number, dv_maintainer)); + int32_t total_buckets = restored->TotalBuckets().value_or(GetDefaultBucketNum()); + + if (partition_iter == writers_.end()) { writers_.emplace(partition, std::unordered_map>( {{bucket, WriterContainer(writer, total_buckets)}})); - return writer; } else { - auto& buckets = iter->second; - auto iter = buckets.find(bucket); - if (PAIMON_LIKELY(iter != buckets.end())) { - return iter->second.writer; - } else { - PAIMON_ASSIGN_OR_RAISE(auto result, - CreateWriter(partition, bucket, ignore_previous_files_)); - int32_t total_buckets = result.first; - std::shared_ptr writer = result.second; - buckets.emplace(bucket, WriterContainer(writer, total_buckets)); - return writer; - } + partition_iter->second.emplace(bucket, WriterContainer(writer, total_buckets)); } + + return writer; } } // namespace paimon diff --git a/src/paimon/core/operation/abstract_file_store_write.h b/src/paimon/core/operation/abstract_file_store_write.h index 8f965d18..e1726617 100644 --- a/src/paimon/core/operation/abstract_file_store_write.h +++ b/src/paimon/core/operation/abstract_file_store_write.h @@ -28,6 +28,7 @@ #include "paimon/commit_message.h" #include "paimon/common/data/binary_row.h" #include "paimon/core/core_options.h" +#include "paimon/core/deletionvectors/bucketed_dv_maintainer.h" #include "paimon/file_store_write.h" #include "paimon/logging.h" #include "paimon/metrics.h" @@ -64,18 +65,18 @@ class AbstractFileStoreWrite : public FileStoreWrite { public: // schema indicates all fields in table schema, write_schema indicates actual write fields while // "data-evolution.enabled" is true - AbstractFileStoreWrite(const std::shared_ptr& file_store_path_factory, - const std::shared_ptr& snapshot_manager, - const std::shared_ptr& schema_manager, - const std::string& commit_user, const std::string& root_path, - const std::shared_ptr& table_schema, - const std::shared_ptr& schema, - const std::shared_ptr& write_schema, - const std::shared_ptr& partition_schema, - const CoreOptions& options, bool ignore_previous_files, - bool is_streaming_mode, bool ignore_num_bucket_check, - const std::shared_ptr& executor, - const std::shared_ptr& pool); + AbstractFileStoreWrite( + const std::shared_ptr& file_store_path_factory, + const std::shared_ptr& snapshot_manager, + const std::shared_ptr& schema_manager, const std::string& commit_user, + const std::string& root_path, const std::shared_ptr& table_schema, + const std::shared_ptr& schema, + const std::shared_ptr& write_schema, + const std::shared_ptr& partition_schema, + const std::shared_ptr& dv_maintainer_factory, + const CoreOptions& options, bool ignore_previous_files, bool is_streaming_mode, + bool ignore_num_bucket_check, const std::shared_ptr& executor, + const std::shared_ptr& pool); Status Write(std::unique_ptr&& batch) override; Status Compact(const std::map& partition, int32_t bucket, @@ -102,9 +103,11 @@ class AbstractFileStoreWrite : public FileStoreWrite { }; protected: - // return actual total bucket and writer in the specific partition - virtual Result>> CreateWriter( - const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) = 0; + virtual Result> CreateWriter( + const BinaryRow& partition, int32_t bucket, + const std::vector>& restore_data_files, + int64_t restore_max_seq_number, + const std::shared_ptr& dv_maintainer) = 0; virtual Result> CreateFileStoreScan( const std::shared_ptr& filter) const = 0; @@ -124,6 +127,7 @@ class AbstractFileStoreWrite : public FileStoreWrite { std::shared_ptr write_schema_; std::shared_ptr table_schema_; std::shared_ptr partition_schema_; + std::shared_ptr dv_maintainer_factory_; CoreOptions options_; std::shared_ptr compact_executor_; std::shared_ptr compaction_metrics_; diff --git a/src/paimon/core/operation/abstract_split_read.cpp b/src/paimon/core/operation/abstract_split_read.cpp index be0ea416..f5f37631 100644 --- a/src/paimon/core/operation/abstract_split_read.cpp +++ b/src/paimon/core/operation/abstract_split_read.cpp @@ -64,8 +64,7 @@ AbstractSplitRead::AbstractSplitRead(const std::shared_ptr Result>> AbstractSplitRead::CreateRawFileReaders( const BinaryRow& partition, const std::vector>& data_files, const std::shared_ptr& read_schema, const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, - const std::optional>& row_ranges, + DeletionVector::Factory dv_factory, const std::optional>& row_ranges, const std::shared_ptr& data_file_path_factory) const { if (data_files.empty()) { return std::vector>(); @@ -84,7 +83,7 @@ Result>> AbstractSplitRead::CreateR PAIMON_ASSIGN_OR_RAISE( std::unique_ptr file_reader, CreateFieldMappingReader(data_file_path, file, partition, reader_builder.get(), - field_mapping_builder.get(), deletion_file_map, row_ranges, + field_mapping_builder.get(), dv_factory, row_ranges, data_file_path_factory)); if (file_reader) { raw_file_readers.push_back(std::move(file_reader)); @@ -173,8 +172,7 @@ Result> AbstractSplitRead::CreateFileBatchReade Result> AbstractSplitRead::CreateFieldMappingReader( const std::string& data_file_path, const std::shared_ptr& file_meta, const BinaryRow& partition, const ReaderBuilder* reader_builder, - const FieldMappingBuilder* field_mapping_builder, - const std::unordered_map& deletion_file_map, + const FieldMappingBuilder* field_mapping_builder, DeletionVector::Factory dv_factory, const std::optional>& row_ranges, const std::shared_ptr& data_file_path_factory) const { std::shared_ptr data_schema; @@ -216,7 +214,7 @@ Result> AbstractSplitRead::CreateFieldMappingRe PAIMON_ASSIGN_OR_RAISE(std::unique_ptr final_reader, ApplyIndexAndDvReaderIfNeeded( std::move(file_reader), file_meta, all_data_schema, read_schema, - predicate, deletion_file_map, row_ranges, data_file_path_factory)); + predicate, dv_factory, row_ranges, data_file_path_factory)); if (!final_reader) { // file is skipped by index or dv return std::unique_ptr(); diff --git a/src/paimon/core/operation/abstract_split_read.h b/src/paimon/core/operation/abstract_split_read.h index 20db532b..98bc0efd 100644 --- a/src/paimon/core/operation/abstract_split_read.h +++ b/src/paimon/core/operation/abstract_split_read.h @@ -24,6 +24,7 @@ #include "arrow/type_fwd.h" #include "paimon/core/core_options.h" +#include "paimon/core/deletionvectors/deletion_vector.h" #include "paimon/core/io/field_mapping_reader.h" #include "paimon/core/operation/internal_read_context.h" #include "paimon/core/operation/split_read.h" @@ -62,8 +63,7 @@ class AbstractSplitRead : public SplitRead { Result>> CreateRawFileReaders( const BinaryRow& partition, const std::vector>& data_files, const std::shared_ptr& read_schema, - const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, + const std::shared_ptr& predicate, DeletionVector::Factory dv_factory, const std::optional>& row_ranges, const std::shared_ptr& data_file_path_factory) const; @@ -90,8 +90,7 @@ class AbstractSplitRead : public SplitRead { std::unique_ptr&& file_reader, const std::shared_ptr& file, const std::shared_ptr& data_schema, const std::shared_ptr& read_schema, - const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, + const std::shared_ptr& predicate, DeletionVector::Factory dv_factory, const std::optional>& row_ranges, const std::shared_ptr& data_file_path_factory) const = 0; @@ -114,8 +113,7 @@ class AbstractSplitRead : public SplitRead { Result> CreateFieldMappingReader( const std::string& data_file_path, const std::shared_ptr& file_meta, const BinaryRow& partition, const ReaderBuilder* reader_builder, - const FieldMappingBuilder* field_mapping_builder, - const std::unordered_map& deletion_file_map, + const FieldMappingBuilder* field_mapping_builder, DeletionVector::Factory dv_factory, const std::optional>& row_ranges, const std::shared_ptr& data_file_path_factory) 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 cae9ab6d..35d862d4 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -61,13 +61,15 @@ AppendOnlyFileStoreWrite::AppendOnlyFileStoreWrite( const std::string& root_path, const std::shared_ptr& table_schema, const std::shared_ptr& schema, const std::shared_ptr& write_schema, - const std::shared_ptr& partition_schema, const CoreOptions& options, - bool ignore_previous_files, bool is_streaming_mode, bool ignore_num_bucket_check, - const std::shared_ptr& executor, const std::shared_ptr& pool) + const std::shared_ptr& partition_schema, + const std::shared_ptr& dv_maintainer_factory, + const CoreOptions& options, bool ignore_previous_files, bool is_streaming_mode, + bool ignore_num_bucket_check, const std::shared_ptr& executor, + const std::shared_ptr& pool) : AbstractFileStoreWrite(file_store_path_factory, snapshot_manager, schema_manager, commit_user, root_path, table_schema, schema, write_schema, partition_schema, - options, ignore_previous_files, is_streaming_mode, - ignore_num_bucket_check, executor, pool), + dv_maintainer_factory, options, ignore_previous_files, + is_streaming_mode, ignore_num_bucket_check, executor, pool), logger_(Logger::GetLogger("AppendOnlyFileStoreWrite")) { write_cols_ = write_schema->field_names(); auto schemas = BlobUtils::SeparateBlobSchema(schema_); @@ -105,15 +107,14 @@ Result> AppendOnlyFileStoreWrite::CreateFileStore } Result>> AppendOnlyFileStoreWrite::CompactRewrite( - const BinaryRow& partition, int32_t bucket, + const BinaryRow& partition, int32_t bucket, DeletionVector::Factory dv_factory, const std::vector>& to_compact) { if (to_compact.empty()) { return std::vector>{}; } - // TODO(yonghao.fyh): support dv factory PAIMON_ASSIGN_OR_RAISE(std::unique_ptr reader, - CreateFilesReader(partition, bucket, to_compact)); + CreateFilesReader(partition, bucket, dv_factory, to_compact)); auto rewriter = std::make_unique>>( options_.GetTargetFileSize(/*has_primary_key=*/false), @@ -158,37 +159,38 @@ Result>> AppendOnlyFileStoreWrite::Com return rewriter->GetResult(); } -Result>> AppendOnlyFileStoreWrite::CreateWriter( - const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) { +Result> AppendOnlyFileStoreWrite::CreateWriter( + const BinaryRow& partition, int32_t bucket, + const std::vector>& restore_data_files, + int64_t restore_max_seq_number, const std::shared_ptr& dv_maintainer) { PAIMON_LOG_DEBUG(logger_, "Creating append only writer for partition %s, bucket %d", partition.ToString().c_str(), bucket); - int32_t total_buckets = GetDefaultBucketNum(); - std::vector> restore_data_files; - if (!ignore_previous_files) { - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, - ScanExistingFileMetas(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_data_files); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr data_file_path_factory, file_store_path_factory_->CreateDataFilePathFactory(partition, bucket)); std::shared_ptr compact_manager; auto schemas = BlobUtils::SeparateBlobSchema(write_schema_); - if (options_.WriteOnly() || with_blob_) { + if (options_.WriteOnly() || options_.GetBucket() == -1 || with_blob_) { // TODO(yonghao.fyh): check data evolution compact_manager = std::make_shared(); } else { - auto rewriter = [this, partition, - bucket](const std::vector>& files) + auto dv_factory = + [dv_maintainer]( + const std::string& file_name) -> Result> { + if (dv_maintainer) { + return dv_maintainer->DeletionVectorOf(file_name).value_or(nullptr); + } + return std::shared_ptr(); + }; + + auto rewriter = [this, partition, bucket, + dv_factory](const std::vector>& to_compact) -> Result>> { - return CompactRewrite(partition, bucket, files); + return CompactRewrite(partition, bucket, dv_factory, to_compact); }; + compact_manager = std::make_shared( - compact_executor_, restore_data_files, /*dv_maintainer=*/nullptr, + compact_executor_, restore_data_files, dv_maintainer, options_.GetCompactionMinFileNum(), options_.GetTargetFileSize(/*has_primary_key=*/false), options_.GetCompactionFileSize(/*has_primary_key=*/false), @@ -197,9 +199,9 @@ Result>> AppendOnlyFileStoreWrit } auto writer = std::make_shared( - options_, table_schema_->Id(), write_schema_, write_cols_, max_sequence_number, + options_, table_schema_->Id(), write_schema_, write_cols_, restore_max_seq_number, data_file_path_factory, compact_manager, pool_); - return std::pair>(total_buckets, writer); + return std::shared_ptr(writer); } AppendOnlyFileStoreWrite::SingleFileWriterCreator @@ -239,7 +241,7 @@ AppendOnlyFileStoreWrite::GetDataFileWriterCreator( } Result> AppendOnlyFileStoreWrite::CreateFilesReader( - const BinaryRow& partition, int32_t bucket, + const BinaryRow& partition, int32_t bucket, DeletionVector::Factory dv_factory, const std::vector>& files) const { ReadContextBuilder context_builder(root_path_); context_builder.EnablePrefetch(true).SetPrefetchMaxParallelNum(1); @@ -249,7 +251,8 @@ Result> AppendOnlyFileStoreWrite::CreateFilesReader 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, {}); + + return read->CreateReader(partition, bucket, files, dv_factory); } } // 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 ee3a475c..aabcd2e1 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/deletionvectors/deletion_vector.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" @@ -46,6 +47,7 @@ namespace paimon { struct DataFileMeta; class BatchWriter; +class BucketedDvMaintainer; class FileStorePathFactory; class FileStoreScan; class SnapshotManager; @@ -61,32 +63,35 @@ class TableSchema; class AppendOnlyFileStoreWrite : public AbstractFileStoreWrite { public: - AppendOnlyFileStoreWrite(const std::shared_ptr& file_store_path_factory, - const std::shared_ptr& snapshot_manager, - const std::shared_ptr& schema_manager, - const std::string& commit_user, const std::string& root_path, - const std::shared_ptr& table_schema, - const std::shared_ptr& schema, - const std::shared_ptr& write_schema, - const std::shared_ptr& partition_schema, - const CoreOptions& options, bool ignore_previous_files, - bool is_streaming_mode, bool ignore_num_bucket_check, - const std::shared_ptr& executor, - const std::shared_ptr& pool); + AppendOnlyFileStoreWrite( + const std::shared_ptr& file_store_path_factory, + const std::shared_ptr& snapshot_manager, + const std::shared_ptr& schema_manager, const std::string& commit_user, + const std::string& root_path, const std::shared_ptr& table_schema, + const std::shared_ptr& schema, + const std::shared_ptr& write_schema, + const std::shared_ptr& partition_schema, + const std::shared_ptr& dv_maintainer_factory, + const CoreOptions& options, bool ignore_previous_files, bool is_streaming_mode, + bool ignore_num_bucket_check, const std::shared_ptr& executor, + const std::shared_ptr& pool); ~AppendOnlyFileStoreWrite() override; private: using SingleFileWriterCreator = std::function< Result>>>()>; - Result>> CreateWriter( - const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) override; + Result> CreateWriter( + const BinaryRow& partition, int32_t bucket, + const std::vector>& restore_data_files, + int64_t restore_max_seq_number, + const std::shared_ptr& dv_maintainer) override; Result> CreateFileStoreScan( const std::shared_ptr& filter) const override; Result>> CompactRewrite( - const BinaryRow& partition, int32_t bucket, + const BinaryRow& partition, int32_t bucket, DeletionVector::Factory dv_factory, const std::vector>& to_compact); SingleFileWriterCreator GetDataFileWriterCreator( @@ -95,7 +100,7 @@ class AppendOnlyFileStoreWrite : public AbstractFileStoreWrite { const std::vector>& to_compact) const; Result> CreateFilesReader( - const BinaryRow& partition, int32_t bucket, + const BinaryRow& partition, int32_t bucket, DeletionVector::Factory dv_factory, const std::vector>& files) const; std::optional> write_cols_; diff --git a/src/paimon/core/operation/data_evolution_split_read.cpp b/src/paimon/core/operation/data_evolution_split_read.cpp index afda12bd..26bb006c 100644 --- a/src/paimon/core/operation/data_evolution_split_read.cpp +++ b/src/paimon/core/operation/data_evolution_split_read.cpp @@ -178,10 +178,9 @@ Result> DataEvolutionSplitRead::ApplyIndexAndDv std::unique_ptr&& file_reader, const std::shared_ptr& file, const std::shared_ptr& data_schema, const std::shared_ptr& read_schema, const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, - const std::optional>& row_ranges, + DeletionVector::Factory dv_factory, const std::optional>& row_ranges, const std::shared_ptr& data_file_path_factory) const { - if (!deletion_file_map.empty()) { + if (dv_factory) { return Status::Invalid("DataEvolutionSplitRead do not support deletion vector"); } if (predicate) { diff --git a/src/paimon/core/operation/data_evolution_split_read.h b/src/paimon/core/operation/data_evolution_split_read.h index bd3e8a4c..6f571f70 100644 --- a/src/paimon/core/operation/data_evolution_split_read.h +++ b/src/paimon/core/operation/data_evolution_split_read.h @@ -75,8 +75,7 @@ class DataEvolutionSplitRead : public AbstractSplitRead { std::unique_ptr&& file_reader, const std::shared_ptr& file, const std::shared_ptr& data_schema, const std::shared_ptr& read_schema, - const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, + const std::shared_ptr& predicate, DeletionVector::Factory dv_factory, const std::optional>& row_ranges, const std::shared_ptr& data_file_path_factory) const override; diff --git a/src/paimon/core/operation/file_store_write.cpp b/src/paimon/core/operation/file_store_write.cpp index 11d661ff..397b2c34 100644 --- a/src/paimon/core/operation/file_store_write.cpp +++ b/src/paimon/core/operation/file_store_write.cpp @@ -23,6 +23,7 @@ #include "fmt/format.h" #include "paimon/common/types/data_field.h" #include "paimon/core/core_options.h" +#include "paimon/core/manifest/index_manifest_file.h" #include "paimon/core/mergetree/compact/lookup_merge_function.h" #include "paimon/core/mergetree/compact/merge_function.h" #include "paimon/core/mergetree/compact/reducer_merge_function_wrapper.h" @@ -101,7 +102,9 @@ Result> FileStoreWrite::Create(std::unique_ptrIgnorePreviousFiles(); if (schema->PrimaryKeys().empty()) { // append table + bool need_dv_mantainer_factory = true; if (options.GetBucket() == -1) { + need_dv_mantainer_factory = false; ignore_previous_files = true; } else if (options.GetBucket() <= 0) { return Status::Invalid( @@ -123,11 +126,27 @@ Result> FileStoreWrite::Create(std::unique_ptr dv_maintainer_factory; + if (need_dv_mantainer_factory) { + PAIMON_ASSIGN_OR_RAISE( + std::unique_ptr index_manifest_file, + IndexManifestFile::Create(options.GetFileSystem(), options.GetManifestFormat(), + options.GetManifestCompression(), file_store_path_factory, + ctx->GetMemoryPool(), options)); + auto index_file_handler = std::make_shared( + options.GetFileSystem(), std::move(index_manifest_file), + std::make_shared(file_store_path_factory), + options.DeletionVectorTargetFileSize(), options.DeletionVectorsBitmap64()); + dv_maintainer_factory = + std::make_shared(index_file_handler); + } + return std::make_unique( file_store_path_factory, snapshot_manager, schema_manager, ctx->GetCommitUser(), - ctx->GetRootPath(), schema, arrow_schema, write_schema, partition_schema, options, - ignore_previous_files, ctx->IsStreamingMode(), ctx->IgnoreNumBucketCheck(), - ctx->GetExecutor(), ctx->GetMemoryPool()); + ctx->GetRootPath(), schema, arrow_schema, write_schema, partition_schema, + dv_maintainer_factory, options, ignore_previous_files, ctx->IsStreamingMode(), + ctx->IgnoreNumBucketCheck(), ctx->GetExecutor(), ctx->GetMemoryPool()); } else { // pk table if (options.GetBucket() == BucketModeDefine::POSTPONE_BUCKET) { @@ -165,10 +184,10 @@ Result> FileStoreWrite::Create(std::unique_ptrFields(), options)); return std::make_unique( file_store_path_factory, snapshot_manager, schema_manager, ctx->GetCommitUser(), - ctx->GetRootPath(), schema, arrow_schema, partition_schema, key_comparator, - sequence_fields_comparator, merge_function_wrapper, options, ignore_previous_files, - ctx->IsStreamingMode(), ctx->IgnoreNumBucketCheck(), ctx->GetExecutor(), - ctx->GetMemoryPool()); + ctx->GetRootPath(), schema, arrow_schema, partition_schema, + /*dv_maintainer_factory=*/nullptr, key_comparator, sequence_fields_comparator, + merge_function_wrapper, options, ignore_previous_files, ctx->IsStreamingMode(), + ctx->IgnoreNumBucketCheck(), ctx->GetExecutor(), ctx->GetMemoryPool()); } } diff --git a/src/paimon/core/operation/file_system_write_restore.h b/src/paimon/core/operation/file_system_write_restore.h index df0a7e95..9acf3f52 100644 --- a/src/paimon/core/operation/file_system_write_restore.h +++ b/src/paimon/core/operation/file_system_write_restore.h @@ -23,6 +23,7 @@ #include #include "paimon/core/core_options.h" +#include "paimon/core/index/index_file_handler.h" #include "paimon/core/operation/file_store_scan.h" #include "paimon/core/operation/restore_files.h" #include "paimon/core/operation/write_restore.h" @@ -34,10 +35,11 @@ namespace paimon { class FileSystemWriteRestore : public WriteRestore { public: 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 - } + std::unique_ptr&& scan, + const std::shared_ptr& index_file_handler) + : snapshot_manager_(snapshot_manager), + scan_(std::move(scan)), + index_file_handler_(index_file_handler) {} Result LatestCommittedIdentifier(const std::string& user) const override { // TODO(yonghao.fyh): in java paimon is LatestSnapshotOfUserFromFileSystem @@ -49,7 +51,8 @@ class FileSystemWriteRestore : public WriteRestore { return std::numeric_limits::min(); } - Result> GetRestoreFiles() const override { + Result> GetRestoreFiles( + const BinaryRow& partition, int32_t bucket, bool scan_delete_vectors_index) 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, @@ -66,18 +69,24 @@ class FileSystemWriteRestore : public WriteRestore { PAIMON_ASSIGN_OR_RAISE(std::optional total_buckets, WriteRestore::ExtractDataFiles(entries, &restore_files)); - std::shared_ptr dynamic_bucket_index; - std::vector> delete_vectors_index; + std::vector> deletion_vectors_index; + if (scan_delete_vectors_index) { + PAIMON_ASSIGN_OR_RAISE( + deletion_vectors_index, + index_file_handler_->Scan( + snapshot.value(), std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), + partition, bucket)); + } - return std::make_shared( - snapshot, total_buckets, restore_files, - /*dynamic_bucket_index=*/nullptr, - /*delete_vectors_index=*/std::vector>{}); + return std::make_shared(snapshot, total_buckets, restore_files, + /*dynamic_bucket_index=*/nullptr, + deletion_vectors_index); } private: std::shared_ptr snapshot_manager_; std::unique_ptr scan_; + std::shared_ptr index_file_handler_; }; } // 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 e07d1188..6f16590b 100644 --- a/src/paimon/core/operation/key_value_file_store_write.cpp +++ b/src/paimon/core/operation/key_value_file_store_write.cpp @@ -53,6 +53,7 @@ KeyValueFileStoreWrite::KeyValueFileStoreWrite( const std::string& root_path, const std::shared_ptr& table_schema, const std::shared_ptr& schema, const std::shared_ptr& partition_schema, + const std::shared_ptr& dv_maintainer_factory, const std::shared_ptr& key_comparator, const std::shared_ptr& user_defined_seq_comparator, const std::shared_ptr>& merge_function_wrapper, @@ -61,8 +62,9 @@ KeyValueFileStoreWrite::KeyValueFileStoreWrite( const std::shared_ptr& pool) : AbstractFileStoreWrite(file_store_path_factory, snapshot_manager, schema_manager, commit_user, root_path, table_schema, schema, /*write_schema=*/schema, - partition_schema, options, ignore_previous_files, is_streaming_mode, - ignore_num_bucket_check, executor, pool), + partition_schema, dv_maintainer_factory, options, + ignore_previous_files, is_streaming_mode, ignore_num_bucket_check, + executor, pool), key_comparator_(key_comparator), user_defined_seq_comparator_(user_defined_seq_comparator), merge_function_wrapper_(merge_function_wrapper), @@ -87,30 +89,21 @@ Result> KeyValueFileStoreWrite::CreateFileStoreSc return scan; } -Result>> KeyValueFileStoreWrite::CreateWriter( - const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) { +Result> KeyValueFileStoreWrite::CreateWriter( + const BinaryRow& partition, int32_t bucket, + const std::vector>& restore_data_files, + int64_t restore_max_seq_number, const std::shared_ptr& dv_maintainer) { PAIMON_LOG_DEBUG(logger_, "Creating key value writer for partition %s, bucket %d", partition.ToString().c_str(), bucket); - int32_t total_buckets = GetDefaultBucketNum(); - std::vector> restore_data_files; - if (!ignore_previous_files) { - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr restore_files, - ScanExistingFileMetas(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_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, table_schema_->TrimmedPrimaryKeys()); auto writer = std::make_shared( - max_sequence_number, trimmed_primary_keys, data_file_path_factory, key_comparator_, + restore_max_seq_number, trimmed_primary_keys, data_file_path_factory, key_comparator_, user_defined_seq_comparator_, merge_function_wrapper_, table_schema_->Id(), schema_, options_, pool_); - return std::pair>(total_buckets, writer); + return std::shared_ptr(writer); } } // namespace paimon diff --git a/src/paimon/core/operation/key_value_file_store_write.h b/src/paimon/core/operation/key_value_file_store_write.h index dd397333..d18c57d7 100644 --- a/src/paimon/core/operation/key_value_file_store_write.h +++ b/src/paimon/core/operation/key_value_file_store_write.h @@ -57,6 +57,7 @@ class KeyValueFileStoreWrite : public AbstractFileStoreWrite { const std::string& root_path, const std::shared_ptr& table_schema, const std::shared_ptr& schema, const std::shared_ptr& partition_schema, + const std::shared_ptr& dv_maintainer_factory, const std::shared_ptr& key_comparator, const std::shared_ptr& user_defined_seq_comparator, const std::shared_ptr>& merge_function_wrapper, @@ -65,8 +66,11 @@ class KeyValueFileStoreWrite : public AbstractFileStoreWrite { const std::shared_ptr& pool); private: - Result>> CreateWriter( - const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) override; + Result> CreateWriter( + const BinaryRow& partition, int32_t bucket, + const std::vector>& restore_data_files, + int64_t restore_max_seq_number, + const std::shared_ptr& dv_maintainer) override; Result> CreateFileStoreScan( const std::shared_ptr& filter) const override; diff --git a/src/paimon/core/operation/merge_file_split_read.cpp b/src/paimon/core/operation/merge_file_split_read.cpp index bffe3ca0..b124ea32 100644 --- a/src/paimon/core/operation/merge_file_split_read.cpp +++ b/src/paimon/core/operation/merge_file_split_read.cpp @@ -177,15 +177,12 @@ Result> MergeFileSplitRead::ApplyIndexAndDvRead std::unique_ptr&& file_reader, const std::shared_ptr& file, const std::shared_ptr& data_schema, const std::shared_ptr& read_schema, const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, - const std::optional>& ranges, + DeletionVector::Factory dv_factory, const std::optional>& ranges, const std::shared_ptr& data_file_path_factory) const { // merge read does not use index - PAIMON_UNIQUE_PTR deletion_vector; - auto dv_iter = deletion_file_map.find(file->file_name); - if (dv_iter != deletion_file_map.end()) { - PAIMON_ASSIGN_OR_RAISE(deletion_vector, DeletionVector::Read(options_.GetFileSystem().get(), - dv_iter->second, pool_.get())); + std::shared_ptr deletion_vector; + if (dv_factory) { + PAIMON_ASSIGN_OR_RAISE(deletion_vector, dv_factory(file->file_name)); } const RoaringBitmap32* deletion = nullptr; @@ -207,7 +204,7 @@ Result> MergeFileSplitRead::ApplyIndexAndDvRead if (!file_reader->SupportPreciseBitmapSelection() && actual_selection) { return std::make_unique(std::move(file_reader), - std::move(deletion_vector)); + deletion_vector); } if (deletion_vector && !deletion && !deletion_vector->IsEmpty()) { // TODO(xinyu.lxy): if deletion vector is bitmap64, use ApplyBitmapIndexBatchReader to @@ -221,6 +218,19 @@ Result> MergeFileSplitRead::CreateMergeReader( const std::shared_ptr& data_split, const std::shared_ptr& data_file_path_factory) { auto deletion_file_map = AbstractSplitRead::CreateDeletionFileMap(*data_split); + + auto dv_factory = [this, deletion_file_map]( + const std::string& file_name) -> Result> { + auto iter = deletion_file_map.find(file_name); + if (iter != deletion_file_map.end()) { + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr dv, + DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); + return dv; + } + return std::shared_ptr(); + }; + std::vector> sections = IntervalPartition(data_split->DataFiles(), interval_partition_comparator_).Partition(); std::vector> batch_readers; @@ -228,8 +238,8 @@ Result> MergeFileSplitRead::CreateMergeReader( // no overlap through multiple sections for (const auto& section : sections) { PAIMON_ASSIGN_OR_RAISE(std::unique_ptr projection_reader, - CreateReaderForSection(section, data_split->Partition(), - deletion_file_map, data_file_path_factory)); + CreateReaderForSection(section, data_split->Partition(), dv_factory, + data_file_path_factory)); batch_readers.push_back(std::move(projection_reader)); } auto concat_batch_reader = std::make_unique(std::move(batch_readers), pool_); @@ -241,6 +251,18 @@ Result> MergeFileSplitRead::CreateNoMergeReader( const std::shared_ptr& data_split, bool only_filter_key, const std::shared_ptr& data_file_path_factory) const { auto deletion_file_map = AbstractSplitRead::CreateDeletionFileMap(*data_split); + auto dv_factory = [this, deletion_file_map]( + const std::string& file_name) -> Result> { + auto iter = deletion_file_map.find(file_name); + if (iter != deletion_file_map.end()) { + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr dv, + DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); + return dv; + } + return std::shared_ptr(); + }; + // create read schema without extra fields (e.g., completed key, sequence fields) auto row_kind_field = DataField::ConvertDataFieldToArrowField(SpecialFields::ValueKind()); @@ -250,7 +272,7 @@ Result> MergeFileSplitRead::CreateNoMergeReader( std::vector> raw_file_readers, CreateRawFileReaders(data_split->Partition(), data_split->DataFiles(), read_schema, only_filter_key ? predicate_for_keys_ : context_->GetPredicate(), - deletion_file_map, /*row_ranges=*/{}, data_file_path_factory)); + dv_factory, /*row_ranges=*/{}, data_file_path_factory)); auto raw_readers = ObjectUtils::MoveVector>(std::move(raw_file_readers)); @@ -398,7 +420,7 @@ Result> MergeFileSplitRead::GenerateKeyPredicates( Result> MergeFileSplitRead::CreateReaderForSection( const std::vector& section, const BinaryRow& partition, - const std::unordered_map& deletion_file_map, + DeletionVector::Factory dv_factory, const std::shared_ptr& data_file_path_factory) { // with overlap in one section std::shared_ptr predicate; @@ -409,7 +431,7 @@ Result> MergeFileSplitRead::CreateReaderForSection( } PAIMON_ASSIGN_OR_RAISE( std::unique_ptr sort_merge_reader, - CreateSortMergeReaderForSection(section, partition, deletion_file_map, predicate, + CreateSortMergeReaderForSection(section, partition, dv_factory, predicate, data_file_path_factory, /*drop_delete=*/true)); // KeyValueProjectionReader converts KeyValue objects to arrow array according to projection if (!context_->EnableMultiThreadRowToBatch()) { @@ -425,17 +447,16 @@ Result> MergeFileSplitRead::CreateReaderForSection( Result> MergeFileSplitRead::CreateSortMergeReaderForSection( const std::vector& section, const BinaryRow& partition, - const std::unordered_map& deletion_file_map, - const std::shared_ptr& predicate, + DeletionVector::Factory dv_factory, const std::shared_ptr& predicate, const std::shared_ptr& data_file_path_factory, bool drop_delete) { // with overlap in one section std::vector> record_readers; record_readers.reserve(section.size()); for (const auto& run : section) { // no overlap in a run - PAIMON_ASSIGN_OR_RAISE(std::unique_ptr run_reader, - CreateReaderForRun(partition, run, deletion_file_map, predicate, - data_file_path_factory)); + PAIMON_ASSIGN_OR_RAISE( + std::unique_ptr run_reader, + CreateReaderForRun(partition, run, dv_factory, predicate, data_file_path_factory)); record_readers.emplace_back(std::move(run_reader)); } PAIMON_ASSIGN_OR_RAISE(std::unique_ptr sort_merge_reader, @@ -447,15 +468,14 @@ Result> MergeFileSplitRead::CreateSortMergeRead } Result> MergeFileSplitRead::CreateReaderForRun( - const BinaryRow& partition, const SortedRun& sorted_run, - const std::unordered_map& deletion_file_map, + const BinaryRow& partition, const SortedRun& sorted_run, DeletionVector::Factory dv_factory, const std::shared_ptr& predicate, const std::shared_ptr& data_file_path_factory) const { // no overlap in a run const auto& data_files = sorted_run.Files(); PAIMON_ASSIGN_OR_RAISE( std::vector> raw_file_readers, - CreateRawFileReaders(partition, data_files, read_schema_, predicate, deletion_file_map, + CreateRawFileReaders(partition, data_files, read_schema_, predicate, dv_factory, /*row_ranges=*/{}, data_file_path_factory)); assert(data_files.size() == raw_file_readers.size()); diff --git a/src/paimon/core/operation/merge_file_split_read.h b/src/paimon/core/operation/merge_file_split_read.h index a1d5e396..c2a79fc0 100644 --- a/src/paimon/core/operation/merge_file_split_read.h +++ b/src/paimon/core/operation/merge_file_split_read.h @@ -88,15 +88,13 @@ class MergeFileSplitRead : public AbstractSplitRead { std::unique_ptr&& file_reader, const std::shared_ptr& file, const std::shared_ptr& data_schema, const std::shared_ptr& read_schema, - const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, + const std::shared_ptr& predicate, DeletionVector::Factory dv_factory, const std::optional>& ranges, const std::shared_ptr& data_file_path_factory) const override; Result> CreateSortMergeReaderForSection( const std::vector& section, const BinaryRow& partition, - const std::unordered_map& deletion_file_map, - const std::shared_ptr& predicate, + DeletionVector::Factory dv_factory, const std::shared_ptr& predicate, const std::shared_ptr& data_file_path_factory, bool drop_delete); std::shared_ptr GetPathFactory() const { @@ -118,12 +116,11 @@ class MergeFileSplitRead : public AbstractSplitRead { Result> CreateReaderForSection( const std::vector& section, const BinaryRow& partition, - const std::unordered_map& deletion_file_map, + DeletionVector::Factory dv_factory, const std::shared_ptr& data_file_path_factory); Result> CreateReaderForRun( - const BinaryRow& partition, const SortedRun& sorted_run, - const std::unordered_map& deletion_file_map, + const BinaryRow& partition, const SortedRun& sorted_run, DeletionVector::Factory dv_factory, const std::shared_ptr& predicate, const std::shared_ptr& data_file_path_factory) const; diff --git a/src/paimon/core/operation/raw_file_split_read.cpp b/src/paimon/core/operation/raw_file_split_read.cpp index 415ef1e2..74ce21a2 100644 --- a/src/paimon/core/operation/raw_file_split_read.cpp +++ b/src/paimon/core/operation/raw_file_split_read.cpp @@ -74,15 +74,14 @@ Result> RawFileSplitRead::CreateReader( Result> RawFileSplitRead::CreateReader( const BinaryRow& partition, int32_t bucket, const std::vector>& data_files, - const std::vector>& deletion_files) { + DeletionVector::Factory dv_factory) { 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::vector> raw_file_readers, - CreateRawFileReaders(partition, data_files, raw_read_schema_, predicate, deletion_file_map, + CreateRawFileReaders(partition, data_files, raw_read_schema_, predicate, dv_factory, /*row_ranges=*/{}, data_file_path_factory)); auto raw_readers = @@ -93,6 +92,25 @@ Result> RawFileSplitRead::CreateReader( return std::make_unique(std::move(batch_reader), pool_); } +Result> RawFileSplitRead::CreateReader( + const BinaryRow& partition, int32_t bucket, + const std::vector>& data_files, + const std::vector>& deletion_files) { + auto deletion_file_map = CreateDeletionFileMap(data_files, deletion_files); + auto dv_factory = [this, deletion_file_map]( + const std::string& file_name) -> Result> { + auto iter = deletion_file_map.find(file_name); + if (iter != deletion_file_map.end()) { + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr dv, + DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); + return dv; + } + return std::shared_ptr(); + }; + return CreateReader(partition, bucket, data_files, dv_factory); +} + Result RawFileSplitRead::Match(const std::shared_ptr& split, bool force_keep_delete) const { auto split_impl = dynamic_cast(split.get()); @@ -122,8 +140,7 @@ Result> RawFileSplitRead::ApplyIndexAndDvReader std::unique_ptr&& file_reader, const std::shared_ptr& file, const std::shared_ptr& data_schema, const std::shared_ptr& read_schema, const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, - const std::optional>& ranges, + DeletionVector::Factory dv_factory, const std::optional>& ranges, const std::shared_ptr& data_file_path_factory) const { std::shared_ptr file_index_result; if (options_.FileIndexReadEnabled()) { @@ -143,11 +160,9 @@ Result> RawFileSplitRead::ApplyIndexAndDvReader } // prepare deletion bitmap for deletion vector - PAIMON_UNIQUE_PTR deletion_vector; - auto dv_iter = deletion_file_map.find(file->file_name); - if (dv_iter != deletion_file_map.end()) { - PAIMON_ASSIGN_OR_RAISE(deletion_vector, DeletionVector::Read(options_.GetFileSystem().get(), - dv_iter->second, pool_.get())); + std::shared_ptr deletion_vector; + if (dv_factory) { + PAIMON_ASSIGN_OR_RAISE(deletion_vector, dv_factory(file->file_name)); } const RoaringBitmap32* deletion = nullptr; if (auto* bitmap_dv = dynamic_cast(deletion_vector.get())) { diff --git a/src/paimon/core/operation/raw_file_split_read.h b/src/paimon/core/operation/raw_file_split_read.h index b7438f8a..2519aa85 100644 --- a/src/paimon/core/operation/raw_file_split_read.h +++ b/src/paimon/core/operation/raw_file_split_read.h @@ -22,6 +22,7 @@ #include #include "paimon/core/core_options.h" +#include "paimon/core/deletionvectors/deletion_vector.h" #include "paimon/core/io/data_file_meta.h" #include "paimon/core/operation/abstract_split_read.h" #include "paimon/core/schema/schema_manager.h" @@ -67,14 +68,18 @@ class RawFileSplitRead : public AbstractSplitRead { const std::vector>& files, const std::vector>& deletion_files); + Result> CreateReader( + const BinaryRow& partition, int32_t bucket, + const std::vector>& files, + DeletionVector::Factory dv_factory); + Result Match(const std::shared_ptr& split, bool force_keep_delete) const override; Result> ApplyIndexAndDvReaderIfNeeded( std::unique_ptr&& file_reader, const std::shared_ptr& file, const std::shared_ptr& data_schema, const std::shared_ptr& read_schema, - const std::shared_ptr& predicate, - const std::unordered_map& deletion_file_map, + const std::shared_ptr& predicate, DeletionVector::Factory dv_factory, const std::optional>& ranges, const std::shared_ptr& data_file_path_factory) const override; }; diff --git a/src/paimon/core/operation/write_restore.h b/src/paimon/core/operation/write_restore.h index 698d31ac..2dfdccd1 100644 --- a/src/paimon/core/operation/write_restore.h +++ b/src/paimon/core/operation/write_restore.h @@ -40,7 +40,8 @@ class WriteRestore { virtual Result LatestCommittedIdentifier(const std::string& user) const = 0; - virtual Result> GetRestoreFiles() const = 0; + virtual Result> GetRestoreFiles( + const BinaryRow& partition, int32_t bucket, bool scan_delete_vectors_index) const = 0; }; } // namespace paimon diff --git a/src/paimon/core/postpone/postpone_bucket_file_store_write.h b/src/paimon/core/postpone/postpone_bucket_file_store_write.h index d5818127..c070b472 100644 --- a/src/paimon/core/postpone/postpone_bucket_file_store_write.h +++ b/src/paimon/core/postpone/postpone_bucket_file_store_write.h @@ -86,8 +86,9 @@ class PostponeBucketFileStoreWrite : public AbstractFileStoreWrite { // Because there is no merging when reading, sequence id across files are useless. return std::unique_ptr(new PostponeBucketFileStoreWrite( file_store_path_factory, snapshot_manager, schema_manager, commit_user, root_path, - table_schema, schema, partition_schema, new_options, /*ignore_previous_files=*/true, - is_streaming_mode, ignore_num_bucket_check, executor, pool)); + table_schema, schema, partition_schema, /*dv_maintainer_factory=*/nullptr, new_options, + /*ignore_previous_files=*/true, is_streaming_mode, ignore_num_bucket_check, executor, + pool)); } private: @@ -97,16 +98,22 @@ class PostponeBucketFileStoreWrite : public AbstractFileStoreWrite { const std::shared_ptr& schema_manager, const std::string& commit_user, const std::string& root_path, const std::shared_ptr& table_schema, const std::shared_ptr& schema, - const std::shared_ptr& partition_schema, const CoreOptions& options, - bool ignore_previous_files, bool is_streaming_mode, bool ignore_num_bucket_check, - const std::shared_ptr& executor, const std::shared_ptr& pool) - : AbstractFileStoreWrite( - file_store_path_factory, snapshot_manager, schema_manager, commit_user, root_path, - table_schema, schema, /*write_schema=*/schema, partition_schema, options, - ignore_previous_files, is_streaming_mode, ignore_num_bucket_check, executor, pool) {} + const std::shared_ptr& partition_schema, + const std::shared_ptr& dv_maintainer_factory, + const CoreOptions& options, bool ignore_previous_files, bool is_streaming_mode, + bool ignore_num_bucket_check, const std::shared_ptr& executor, + const std::shared_ptr& pool) + : AbstractFileStoreWrite(file_store_path_factory, snapshot_manager, schema_manager, + commit_user, root_path, table_schema, schema, + /*write_schema=*/schema, partition_schema, dv_maintainer_factory, + options, ignore_previous_files, is_streaming_mode, + ignore_num_bucket_check, executor, pool) {} - Result>> CreateWriter( - const BinaryRow& partition, int32_t bucket, bool ignore_previous_files) override { + Result> CreateWriter( + const BinaryRow& partition, int32_t bucket, + const std::vector>& restore_data_files, + int64_t restore_max_seq_number, + const std::shared_ptr& dv_maintainer) override { PAIMON_RETURN_NOT_OK( Preconditions::CheckState(bucket == BucketModeDefine::POSTPONE_BUCKET, "bucket mode is supposed to be postpone bucket")); @@ -118,8 +125,7 @@ class PostponeBucketFileStoreWrite : public AbstractFileStoreWrite { auto writer = std::make_shared(trimmed_primary_keys, data_file_path_factory, table_schema_->Id(), schema_, options_, pool_); - int32_t total_buckets = GetDefaultBucketNum(); - return std::pair>(total_buckets, writer); + return std::shared_ptr(writer); } Result> CreateFileStoreScan( diff --git a/src/paimon/core/table/source/table_scan.cpp b/src/paimon/core/table/source/table_scan.cpp index b4cd3afa..03c6ea8a 100644 --- a/src/paimon/core/table/source/table_scan.cpp +++ b/src/paimon/core/table/source/table_scan.cpp @@ -145,7 +145,9 @@ class TableScanImpl { core_options.GetManifestCompression(), path_factory, memory_pool, core_options)); return std::make_unique( - std::move(index_manifest_file), std::make_shared(path_factory)); + core_options.GetFileSystem(), std::move(index_manifest_file), + std::make_shared(path_factory), + core_options.DeletionVectorTargetFileSize(), core_options.DeletionVectorsBitmap64()); } }; From c8092b589f6494e6c3dd5aad296f51cab111a6a2 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Thu, 12 Mar 2026 16:47:41 +0800 Subject: [PATCH 02/27] fix --- src/paimon/core/deletionvectors/deletion_vector.cpp | 5 ++++- src/paimon/core/global_index/global_index_scan_impl.cpp | 2 +- src/paimon/core/index/index_file_handler.h | 6 ++++-- src/paimon/core/index/index_file_handler_test.cpp | 3 ++- src/paimon/core/operation/file_store_write.cpp | 9 +++++---- src/paimon/core/table/source/table_scan.cpp | 3 ++- 6 files changed, 18 insertions(+), 10 deletions(-) diff --git a/src/paimon/core/deletionvectors/deletion_vector.cpp b/src/paimon/core/deletionvectors/deletion_vector.cpp index 44e76382..43999a10 100644 --- a/src/paimon/core/deletionvectors/deletion_vector.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector.cpp @@ -77,7 +77,10 @@ Result> DeletionVector::Read(DataInputStream* return BitmapDeletionVector::Deserialize(bytes->data(), bytes->size(), pool); } else if (EndianSwapValue(magic_number) == Bitmap64DeletionVector::MAGIC_NUMBER) { - return Status::NotImplemented("bitmap64 dv is not implemented"); + return Status::NotImplemented( + "bitmap64 deletion vectors are not supported in this version; " + "please use bitmap deletion vectors instead or upgrade to a version " + "that supports bitmap64."); } return Status::Invalid(fmt::format( diff --git a/src/paimon/core/global_index/global_index_scan_impl.cpp b/src/paimon/core/global_index/global_index_scan_impl.cpp index bfa3f966..f19bc872 100644 --- a/src/paimon/core/global_index/global_index_scan_impl.cpp +++ b/src/paimon/core/global_index/global_index_scan_impl.cpp @@ -121,7 +121,7 @@ Status GlobalIndexScanImpl::Scan() { auto index_file_handler = std::make_unique( options_.GetFileSystem(), std::move(index_manifest_file), std::make_shared(path_factory_), - options_.DeletionVectorTargetFileSize(), options_.DeletionVectorsBitmap64()); + options_.DeletionVectorTargetFileSize(), options_.DeletionVectorsBitmap64(), pool_); PAIMON_ASSIGN_OR_RAISE(std::vector partition_fields, table_schema_->GetFields(table_schema_->PartitionKeys())); diff --git a/src/paimon/core/index/index_file_handler.h b/src/paimon/core/index/index_file_handler.h index fd221e13..08859b08 100644 --- a/src/paimon/core/index/index_file_handler.h +++ b/src/paimon/core/index/index_file_handler.h @@ -45,12 +45,14 @@ class IndexFileHandler { IndexFileHandler(const std::shared_ptr& fs, std::unique_ptr&& index_manifest_file, const std::shared_ptr& path_factories, - int64_t dv_target_file_size, bool dv_bitmap64) + int64_t dv_target_file_size, bool dv_bitmap64, + const std::shared_ptr& pool) : fs_(fs), index_manifest_file_(std::move(index_manifest_file)), path_factories_(path_factories), dv_target_file_size_(dv_target_file_size), - dv_bitmap64_(dv_bitmap64) {} + dv_bitmap64_(dv_bitmap64), + pool_(pool) {} /// 1.Scan specified index_type index. 2.Cluster with partition & bucket. Result Scan(const Snapshot& snapshot, const std::string& index_type, diff --git a/src/paimon/core/index/index_file_handler_test.cpp b/src/paimon/core/index/index_file_handler_test.cpp index 4c713667..ff33ade3 100644 --- a/src/paimon/core/index/index_file_handler_test.cpp +++ b/src/paimon/core/index/index_file_handler_test.cpp @@ -75,7 +75,8 @@ class IndexFileHandlerTest : public testing::Test { auto path_factories = std::make_shared(path_factory); return std::make_unique( core_options.GetFileSystem(), std::move(index_manifest_file), path_factories, - core_options.DeletionVectorTargetFileSize(), core_options.DeletionVectorsBitmap64()); + core_options.DeletionVectorTargetFileSize(), core_options.DeletionVectorsBitmap64(), + memory_pool_); } std::shared_ptr memory_pool_; }; diff --git a/src/paimon/core/operation/file_store_write.cpp b/src/paimon/core/operation/file_store_write.cpp index 397b2c34..70816529 100644 --- a/src/paimon/core/operation/file_store_write.cpp +++ b/src/paimon/core/operation/file_store_write.cpp @@ -102,9 +102,9 @@ Result> FileStoreWrite::Create(std::unique_ptrIgnorePreviousFiles(); if (schema->PrimaryKeys().empty()) { // append table - bool need_dv_mantainer_factory = true; + bool need_dv_maintainer_factory = true; if (options.GetBucket() == -1) { - need_dv_mantainer_factory = false; + need_dv_maintainer_factory = false; ignore_previous_files = true; } else if (options.GetBucket() <= 0) { return Status::Invalid( @@ -128,7 +128,7 @@ Result> FileStoreWrite::Create(std::unique_ptr dv_maintainer_factory; - if (need_dv_mantainer_factory) { + if (need_dv_maintainer_factory) { PAIMON_ASSIGN_OR_RAISE( std::unique_ptr index_manifest_file, IndexManifestFile::Create(options.GetFileSystem(), options.GetManifestFormat(), @@ -137,7 +137,8 @@ Result> FileStoreWrite::Create(std::unique_ptr( options.GetFileSystem(), std::move(index_manifest_file), std::make_shared(file_store_path_factory), - options.DeletionVectorTargetFileSize(), options.DeletionVectorsBitmap64()); + options.DeletionVectorTargetFileSize(), options.DeletionVectorsBitmap64(), + ctx->GetMemoryPool()); dv_maintainer_factory = std::make_shared(index_file_handler); } diff --git a/src/paimon/core/table/source/table_scan.cpp b/src/paimon/core/table/source/table_scan.cpp index 03c6ea8a..e521bf63 100644 --- a/src/paimon/core/table/source/table_scan.cpp +++ b/src/paimon/core/table/source/table_scan.cpp @@ -147,7 +147,8 @@ class TableScanImpl { return std::make_unique( core_options.GetFileSystem(), std::move(index_manifest_file), std::make_shared(path_factory), - core_options.DeletionVectorTargetFileSize(), core_options.DeletionVectorsBitmap64()); + core_options.DeletionVectorTargetFileSize(), core_options.DeletionVectorsBitmap64(), + memory_pool); } }; From 06c6312a714a062e3a9ed46f46d2c60892c57ccd Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Thu, 12 Mar 2026 18:29:08 +0800 Subject: [PATCH 03/27] fix --- .../deletionvectors/bucketed_dv_maintainer.h | 4 +- .../deletion_vector_index_file_writer.h | 1 - .../deletion_vectors_index_file.cpp | 28 +++++++--- .../deletion_vectors_index_file.h | 12 +---- src/paimon/core/index/deletion_vector_meta.h | 51 +++++++++++++------ src/paimon/core/index/index_file_handler.h | 2 +- src/paimon/core/index/index_file_meta.h | 1 - .../core/index/index_file_meta_serializer.cpp | 10 ++-- .../table/source/snapshot/snapshot_reader.cpp | 8 +-- test/inte/compaction_inte_test.cpp | 2 +- 10 files changed, 70 insertions(+), 49 deletions(-) diff --git a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h index d7ccf112..3339d472 100644 --- a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h +++ b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h @@ -78,7 +78,7 @@ class BucketedDvMaintainer { Result> Create( const BinaryRow& partition, int32_t bucket, - const std::vector>& restored_files) { + const std::vector>& restored_files) const { std::map> deletion_vectors; PAIMON_ASSIGN_OR_RAISE(deletion_vectors, handler_->ReadAllDeletionVectors( partition, bucket, restored_files)); @@ -87,7 +87,7 @@ class BucketedDvMaintainer { Result> Create( const BinaryRow& partition, int32_t bucket, - const std::map>& deletion_vectors) { + const std::map>& deletion_vectors) const { PAIMON_ASSIGN_OR_RAISE(std::shared_ptr dv_index_file, handler_->DvIndex(partition, bucket)); return std::make_unique(dv_index_file, deletion_vectors); diff --git a/src/paimon/core/deletionvectors/deletion_vector_index_file_writer.h b/src/paimon/core/deletionvectors/deletion_vector_index_file_writer.h index 687d16e9..ac65110e 100644 --- a/src/paimon/core/deletionvectors/deletion_vector_index_file_writer.h +++ b/src/paimon/core/deletionvectors/deletion_vector_index_file_writer.h @@ -31,7 +31,6 @@ class DeletionVectorIndexFileWriter { public: DeletionVectorIndexFileWriter(const std::shared_ptr& fs, const std::shared_ptr& path_factory, - int64_t target_size_per_index_file, const std::shared_ptr& pool) : index_path_factory_(path_factory), fs_(fs), pool_(pool) {} diff --git a/src/paimon/core/deletionvectors/deletion_vectors_index_file.cpp b/src/paimon/core/deletionvectors/deletion_vectors_index_file.cpp index d9265a07..15164bc3 100644 --- a/src/paimon/core/deletionvectors/deletion_vectors_index_file.cpp +++ b/src/paimon/core/deletionvectors/deletion_vectors_index_file.cpp @@ -31,8 +31,7 @@ Result> DeletionVectorsIndexFile::WriteSingleFile } std::shared_ptr DeletionVectorsIndexFile::CreateWriter() const { - return std::make_shared(fs_, path_factory_, - target_size_per_index_file_, pool_); + return std::make_shared(fs_, path_factory_, pool_); } Result>> @@ -41,20 +40,24 @@ DeletionVectorsIndexFile::ReadAllDeletionVectors( std::optional> deletion_vector_metas = file_meta->DvRanges(); if (deletion_vector_metas == std::nullopt) { - return Status::Invalid("deletion vector metas is null"); + return Status::Invalid( + fmt::format("Read all deletion vectors failed from IndexFileMeta '{}'. Deletion vector " + "metas is null", + file_meta->FileName())); } std::map> deletion_vectors; std::string file_path = path_factory_->ToPath(file_meta); PAIMON_ASSIGN_OR_RAISE(std::shared_ptr input_stream, fs_->Open(file_path)); auto data_input_stream = std::make_shared(input_stream); + PAIMON_RETURN_NOT_OK(CheckVersion(data_input_stream)); for (const auto& [_, deletion_vector_meta] : deletion_vector_metas.value()) { - // TODO(yonghao.fyh): check deletion_vector_meta.length = -1? PAIMON_ASSIGN_OR_RAISE( std::shared_ptr dv, DeletionVector::Read(data_input_stream.get(), - static_cast(deletion_vector_meta.length), pool_.get())); - deletion_vectors[deletion_vector_meta.data_file_name] = dv; + static_cast(deletion_vector_meta.GetLength()), + pool_.get())); + deletion_vectors[deletion_vector_meta.GetDataFileName()] = dv; } return deletion_vectors; } @@ -66,11 +69,20 @@ DeletionVectorsIndexFile::ReadAllDeletionVectors( for (const auto& index_file : index_files) { std::map> partial_deletion_vectors; PAIMON_ASSIGN_OR_RAISE(partial_deletion_vectors, ReadAllDeletionVectors(index_file)); - for (const auto& kv : partial_deletion_vectors) { - deletion_vectors[kv.first] = kv.second; + for (const auto& [data_file_name, dv] : partial_deletion_vectors) { + deletion_vectors[data_file_name] = dv; } } return deletion_vectors; } +Status DeletionVectorsIndexFile::CheckVersion(const std::shared_ptr& in) { + PAIMON_ASSIGN_OR_RAISE(int8_t version, in->ReadValue()); + if (version != VERSION_ID_V1) { + return Status::Invalid(fmt::format( + "Version not match, actual version: {}, expected version: {}", version, VERSION_ID_V1)); + } + return Status::OK(); +} + } // namespace paimon diff --git a/src/paimon/core/deletionvectors/deletion_vectors_index_file.h b/src/paimon/core/deletionvectors/deletion_vectors_index_file.h index 70912e73..d5515f9d 100644 --- a/src/paimon/core/deletionvectors/deletion_vectors_index_file.h +++ b/src/paimon/core/deletionvectors/deletion_vectors_index_file.h @@ -59,17 +59,9 @@ class DeletionVectorsIndexFile : public IndexFile { const std::vector>& index_files) const; private: - std::shared_ptr CreateWriter() const; + static Status CheckVersion(const std::shared_ptr& in); - static Status CheckVersion(const std::shared_ptr& in) { - PAIMON_ASSIGN_OR_RAISE(int8_t version, in->ReadValue()); - if (version != VERSION_ID_V1) { - return Status::Invalid( - fmt::format("Version not match, actual version: {}, expected version: {}", version, - VERSION_ID_V1)); - } - return Status::OK(); - } + std::shared_ptr CreateWriter() const; const int64_t target_size_per_index_file_; const bool bitmap64_; diff --git a/src/paimon/core/index/deletion_vector_meta.h b/src/paimon/core/index/deletion_vector_meta.h index 715bfdd0..c0599015 100644 --- a/src/paimon/core/index/deletion_vector_meta.h +++ b/src/paimon/core/index/deletion_vector_meta.h @@ -29,7 +29,8 @@ namespace paimon { /// Indicates the deletion vector info of member data_file_name, e.g., the length of dv. /// * DeletionVectorMeta is used when serialize to manifest file. -struct DeletionVectorMeta { +class DeletionVectorMeta { + public: static const std::shared_ptr& DataType() { static std::shared_ptr schema = arrow::struct_( {arrow::field("f0", arrow::utf8(), false), arrow::field("f1", arrow::int32(), false), @@ -37,19 +38,19 @@ struct DeletionVectorMeta { arrow::field("_CARDINALITY", arrow::int64(), true)}); return schema; } - DeletionVectorMeta(const std::string& _data_file_name, int32_t _offset, int32_t _length, - const std::optional& _cardinality) - : data_file_name(_data_file_name), - offset(_offset), - length(_length), - cardinality(_cardinality) {} + DeletionVectorMeta(const std::string& data_file_name, int32_t offset, int32_t length, + const std::optional& cardinality) + : data_file_name_(data_file_name), + offset_(offset), + length_(length), + cardinality_(cardinality) {} bool operator==(const DeletionVectorMeta& other) const { if (this == &other) { return true; } - return data_file_name == other.data_file_name && offset == other.offset && - length == other.length && cardinality == other.cardinality; + return data_file_name_ == other.data_file_name_ && offset_ == other.offset_ && + length_ == other.length_ && cardinality_ == other.cardinality_; } bool TEST_Equal(const DeletionVectorMeta& other) const { @@ -57,19 +58,37 @@ struct DeletionVectorMeta { return true; } // ignore data_file_name - return offset == other.offset && length == other.length && cardinality == other.cardinality; + return offset_ == other.offset_ && length_ == other.length_ && + cardinality_ == other.cardinality_; } std::string ToString() const { return fmt::format( "DeletionVectorMeta{{data_file_name = {}, offset = {}, length = {}, cardinality = {}}}", - data_file_name, offset, length, - cardinality == std::nullopt ? "null" : std::to_string(cardinality.value())); + data_file_name_, offset_, length_, + cardinality_ == std::nullopt ? "null" : std::to_string(cardinality_.value())); } - std::string data_file_name = ""; - int32_t offset = -1; - int32_t length = -1; - std::optional cardinality; + const std::string& GetDataFileName() const { + return data_file_name_; + } + + int32_t GetOffset() const { + return offset_; + } + + int32_t GetLength() const { + return length_; + } + + std::optional GetCardinality() const { + return cardinality_; + } + + private: + std::string data_file_name_; + int32_t offset_; + int32_t length_; + std::optional cardinality_; }; } // namespace paimon diff --git a/src/paimon/core/index/index_file_handler.h b/src/paimon/core/index/index_file_handler.h index 08859b08..d7c4bf7b 100644 --- a/src/paimon/core/index/index_file_handler.h +++ b/src/paimon/core/index/index_file_handler.h @@ -85,7 +85,7 @@ class IndexFileHandler { Result>> ReadAllDeletionVectors( const BinaryRow& partition, int32_t bucket, - const std::vector>& file_metas) { + const std::vector>& file_metas) const { PAIMON_ASSIGN_OR_RAISE(std::unique_ptr dv_index, DvIndex(partition, bucket)); return dv_index->ReadAllDeletionVectors(file_metas); diff --git a/src/paimon/core/index/index_file_meta.h b/src/paimon/core/index/index_file_meta.h index 6c668429..a8484dc9 100644 --- a/src/paimon/core/index/index_file_meta.h +++ b/src/paimon/core/index/index_file_meta.h @@ -163,7 +163,6 @@ class IndexFileMeta { return result; } - private: std::string index_type_; std::string file_name_; int64_t file_size_ = 0; diff --git a/src/paimon/core/index/index_file_meta_serializer.cpp b/src/paimon/core/index/index_file_meta_serializer.cpp index e7cddfc9..a70c3f06 100644 --- a/src/paimon/core/index/index_file_meta_serializer.cpp +++ b/src/paimon/core/index/index_file_meta_serializer.cpp @@ -91,11 +91,11 @@ BinaryArray IndexFileMetaSerializer::DvRangesToRowArrayData( const auto& meta = dv_meta.second; BinaryRow dv_data(4); BinaryRowWriter writer(&dv_data, 1024, pool); - writer.WriteString(/*pos=*/0, BinaryString::FromString(meta.data_file_name, pool)); - writer.WriteInt(/*pos=*/1, meta.offset); - writer.WriteInt(/*pos=*/2, meta.length); - if (meta.cardinality) { - writer.WriteLong(/*pos=*/3, meta.cardinality.value()); + writer.WriteString(/*pos=*/0, BinaryString::FromString(meta.GetDataFileName(), pool)); + writer.WriteInt(/*pos=*/1, meta.GetOffset()); + writer.WriteInt(/*pos=*/2, meta.GetLength()); + if (meta.GetCardinality()) { + writer.WriteLong(/*pos=*/3, meta.GetCardinality().value()); } else { writer.SetNullAt(3); } diff --git a/src/paimon/core/table/source/snapshot/snapshot_reader.cpp b/src/paimon/core/table/source/snapshot/snapshot_reader.cpp index fac299ae..a0d8e420 100644 --- a/src/paimon/core/table/source/snapshot/snapshot_reader.cpp +++ b/src/paimon/core/table/source/snapshot/snapshot_reader.cpp @@ -121,7 +121,7 @@ Result>> SnapshotReader::GetDeletionFile for (const auto& dv_meta_iter : dv_metas.value()) { const auto& dv_meta = dv_meta_iter.second; data_file_to_index_file_meta.insert( - std::make_pair(dv_meta.data_file_name, index_file_meta)); + std::make_pair(dv_meta.GetDataFileName(), index_file_meta)); } } } @@ -139,9 +139,9 @@ Result>> SnapshotReader::GetDeletionFile PAIMON_ASSIGN_OR_RAISE( std::string index_file_path, index_file_handler_->FilePath(partition, bucket, index_file_meta_iter->second)); - deletion_files.emplace_back( - DeletionFile(index_file_path, dv_meta_iter->second.offset, - dv_meta_iter->second.length, dv_meta_iter->second.cardinality)); + deletion_files.emplace_back(DeletionFile( + index_file_path, dv_meta_iter->second.GetOffset(), + dv_meta_iter->second.GetLength(), dv_meta_iter->second.GetCardinality())); continue; } } diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index 24ddcef9..22ca2200 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -389,7 +389,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) } } -TEST_F(CompactionInteTest, TestAppendTableWriteAlterTableWithCompaction) { +TEST_F(CompactionInteTest, DISABLED_TestAppendTableWriteAlterTableWithCompaction) { std::string test_data_path = paimon::test::GetDataDir() + "/orc/append_table_with_alter_table.db/append_table_with_alter_table/"; From 4ff41652781d5bed53b679045add098213a692f7 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 13 Mar 2026 09:01:58 +0800 Subject: [PATCH 04/27] fix --- .../core/deletionvectors/bucketed_dv_maintainer_test.cpp | 1 - .../core/deletionvectors/deletion_vectors_index_file.h | 9 ++------- .../deletionvectors/deletion_vectors_index_file_test.cpp | 4 +--- src/paimon/core/global_index/global_index_scan_impl.cpp | 8 ++++---- src/paimon/core/index/index_file_handler.h | 9 +++------ src/paimon/core/index/index_file_handler_test.cpp | 3 +-- src/paimon/core/operation/file_store_write.cpp | 3 +-- src/paimon/core/table/source/table_scan.cpp | 3 +-- 8 files changed, 13 insertions(+), 27 deletions(-) diff --git a/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp b/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp index c710aa99..fc9bb204 100644 --- a/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp +++ b/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp @@ -36,7 +36,6 @@ std::shared_ptr CreateDvIndexFile(const std::string& r 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); } diff --git a/src/paimon/core/deletionvectors/deletion_vectors_index_file.h b/src/paimon/core/deletionvectors/deletion_vectors_index_file.h index d5515f9d..6a9cba77 100644 --- a/src/paimon/core/deletionvectors/deletion_vectors_index_file.h +++ b/src/paimon/core/deletionvectors/deletion_vectors_index_file.h @@ -35,13 +35,9 @@ class DeletionVectorsIndexFile : public IndexFile { static constexpr int8_t VERSION_ID_V1 = 1; DeletionVectorsIndexFile(const std::shared_ptr& fs, - const std::shared_ptr& path_factory, - int64_t target_size_per_index_file, bool bitmap64, + const std::shared_ptr& path_factory, bool bitmap64, const std::shared_ptr& pool) - : IndexFile(fs, path_factory), - target_size_per_index_file_(target_size_per_index_file), - bitmap64_(bitmap64), - pool_(pool) {} + : IndexFile(fs, path_factory), bitmap64_(bitmap64), pool_(pool) {} ~DeletionVectorsIndexFile() override = default; @@ -63,7 +59,6 @@ class DeletionVectorsIndexFile : public IndexFile { std::shared_ptr CreateWriter() const; - const int64_t target_size_per_index_file_; const bool bitmap64_; std::shared_ptr pool_; }; diff --git a/src/paimon/core/deletionvectors/deletion_vectors_index_file_test.cpp b/src/paimon/core/deletionvectors/deletion_vectors_index_file_test.cpp index 292d2585..f9de086e 100644 --- a/src/paimon/core/deletionvectors/deletion_vectors_index_file_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_vectors_index_file_test.cpp @@ -35,8 +35,7 @@ TEST(DeletionVectorsIndexFileTest, Basic) { FileSystemFactory::Get("local", dir->Str(), {})); auto path_factory = std::make_shared(dir->Str()); auto pool = GetDefaultPool(); - DeletionVectorsIndexFile index_file( - fs, path_factory, /*target_size_per_index_file=*/1024 * 1024, /*bitmap64=*/false, pool); + DeletionVectorsIndexFile index_file(fs, path_factory, /*bitmap64=*/false, pool); std::map> input; RoaringBitmap32 roaring_1; @@ -66,7 +65,6 @@ TEST(DeletionVectorsIndexFileTest, ExternalPathAndIndexFileMeta) { path_factory->SetExternal(true); auto pool = GetDefaultPool(); DeletionVectorsIndexFile index_file(fs, path_factory, - /*target_size_per_index_file=*/1024 * 1024, /*bitmap64=*/false, pool); std::map> input; diff --git a/src/paimon/core/global_index/global_index_scan_impl.cpp b/src/paimon/core/global_index/global_index_scan_impl.cpp index f19bc872..6b59a209 100644 --- a/src/paimon/core/global_index/global_index_scan_impl.cpp +++ b/src/paimon/core/global_index/global_index_scan_impl.cpp @@ -118,10 +118,10 @@ Status GlobalIndexScanImpl::Scan() { IndexManifestFile::Create( options_.GetFileSystem(), options_.GetManifestFormat(), options_.GetManifestCompression(), path_factory_, pool_, options_)); - auto index_file_handler = std::make_unique( - options_.GetFileSystem(), std::move(index_manifest_file), - std::make_shared(path_factory_), - options_.DeletionVectorTargetFileSize(), options_.DeletionVectorsBitmap64(), pool_); + auto index_file_handler = + std::make_unique(options_.GetFileSystem(), std::move(index_manifest_file), + std::make_shared(path_factory_), + options_.DeletionVectorsBitmap64(), pool_); PAIMON_ASSIGN_OR_RAISE(std::vector partition_fields, table_schema_->GetFields(table_schema_->PartitionKeys())); diff --git a/src/paimon/core/index/index_file_handler.h b/src/paimon/core/index/index_file_handler.h index d7c4bf7b..f0c11642 100644 --- a/src/paimon/core/index/index_file_handler.h +++ b/src/paimon/core/index/index_file_handler.h @@ -45,12 +45,10 @@ class IndexFileHandler { IndexFileHandler(const std::shared_ptr& fs, std::unique_ptr&& index_manifest_file, const std::shared_ptr& path_factories, - int64_t dv_target_file_size, bool dv_bitmap64, - const std::shared_ptr& pool) + bool dv_bitmap64, const std::shared_ptr& pool) : fs_(fs), index_manifest_file_(std::move(index_manifest_file)), path_factories_(path_factories), - dv_target_file_size_(dv_target_file_size), dv_bitmap64_(dv_bitmap64), pool_(pool) {} @@ -79,8 +77,8 @@ class IndexFileHandler { int32_t bucket) const { PAIMON_ASSIGN_OR_RAISE(std::shared_ptr index_path_factory, path_factories_->Get(partition, bucket)); - return std::make_unique( - fs_, index_path_factory, dv_target_file_size_, dv_bitmap64_, pool_); + return std::make_unique(fs_, index_path_factory, dv_bitmap64_, + pool_); } Result>> ReadAllDeletionVectors( @@ -95,7 +93,6 @@ class IndexFileHandler { std::shared_ptr fs_; std::unique_ptr index_manifest_file_; std::shared_ptr path_factories_; - int64_t dv_target_file_size_; bool dv_bitmap64_; std::shared_ptr pool_; }; diff --git a/src/paimon/core/index/index_file_handler_test.cpp b/src/paimon/core/index/index_file_handler_test.cpp index ff33ade3..657cb24f 100644 --- a/src/paimon/core/index/index_file_handler_test.cpp +++ b/src/paimon/core/index/index_file_handler_test.cpp @@ -75,8 +75,7 @@ class IndexFileHandlerTest : public testing::Test { auto path_factories = std::make_shared(path_factory); return std::make_unique( core_options.GetFileSystem(), std::move(index_manifest_file), path_factories, - core_options.DeletionVectorTargetFileSize(), core_options.DeletionVectorsBitmap64(), - memory_pool_); + core_options.DeletionVectorsBitmap64(), memory_pool_); } std::shared_ptr memory_pool_; }; diff --git a/src/paimon/core/operation/file_store_write.cpp b/src/paimon/core/operation/file_store_write.cpp index 70816529..f5702bd8 100644 --- a/src/paimon/core/operation/file_store_write.cpp +++ b/src/paimon/core/operation/file_store_write.cpp @@ -137,8 +137,7 @@ Result> FileStoreWrite::Create(std::unique_ptr( options.GetFileSystem(), std::move(index_manifest_file), std::make_shared(file_store_path_factory), - options.DeletionVectorTargetFileSize(), options.DeletionVectorsBitmap64(), - ctx->GetMemoryPool()); + options.DeletionVectorsBitmap64(), ctx->GetMemoryPool()); dv_maintainer_factory = std::make_shared(index_file_handler); } diff --git a/src/paimon/core/table/source/table_scan.cpp b/src/paimon/core/table/source/table_scan.cpp index e521bf63..dae778bb 100644 --- a/src/paimon/core/table/source/table_scan.cpp +++ b/src/paimon/core/table/source/table_scan.cpp @@ -147,8 +147,7 @@ class TableScanImpl { return std::make_unique( core_options.GetFileSystem(), std::move(index_manifest_file), std::make_shared(path_factory), - core_options.DeletionVectorTargetFileSize(), core_options.DeletionVectorsBitmap64(), - memory_pool); + core_options.DeletionVectorsBitmap64(), memory_pool); } }; From b58088ee4e90636a804293d0c618fd580a60d6a5 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 13 Mar 2026 11:52:16 +0800 Subject: [PATCH 05/27] fix --- src/paimon/CMakeLists.txt | 2 + .../bitmap_deletion_vector.cpp | 13 +- .../deletionvectors/bitmap_deletion_vector.h | 3 + .../deletion_file_writer_test.cpp | 122 ++++++++++++++++++ .../core/deletionvectors/deletion_vector.cpp | 7 +- ...deletion_vector_index_file_writer_test.cpp | 121 +++++++++++++++++ .../deletionvectors/deletion_vector_test.cpp | 80 ++++++++++++ .../deletion_vectors_index_file_test.cpp | 103 +++++++++++++++ .../core/index/index_file_handler_test.cpp | 65 ++++++++++ 9 files changed, 509 insertions(+), 7 deletions(-) create mode 100644 src/paimon/core/deletionvectors/deletion_file_writer_test.cpp create mode 100644 src/paimon/core/deletionvectors/deletion_vector_index_file_writer_test.cpp diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index e4ff1138..90ff16a3 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -508,7 +508,9 @@ if(PAIMON_BUILD_TESTS) 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_file_writer_test.cpp core/deletionvectors/deletion_vector_test.cpp + core/deletionvectors/deletion_vector_index_file_writer_test.cpp core/deletionvectors/deletion_vectors_index_file_test.cpp core/index/index_in_data_file_dir_path_factory_test.cpp core/index/deletion_vector_meta_test.cpp diff --git a/src/paimon/core/deletionvectors/bitmap_deletion_vector.cpp b/src/paimon/core/deletionvectors/bitmap_deletion_vector.cpp index 66be892d..250b1f45 100644 --- a/src/paimon/core/deletionvectors/bitmap_deletion_vector.cpp +++ b/src/paimon/core/deletionvectors/bitmap_deletion_vector.cpp @@ -63,6 +63,13 @@ Status BitmapDeletionVector::CheckPosition(int64_t position) const { return Status::OK(); } +Result> BitmapDeletionVector::DeserializeWithoutMagicNumber( + const char* buffer, int32_t length, MemoryPool* pool) { + RoaringBitmap32 roaring_bitmap; + PAIMON_RETURN_NOT_OK(roaring_bitmap.Deserialize(buffer, length)); + return pool->AllocateUnique(roaring_bitmap); +} + Result> BitmapDeletionVector::Deserialize(const char* buffer, int32_t length, MemoryPool* pool) { @@ -73,10 +80,8 @@ Result> BitmapDeletionVector::Deserialize(cons return Status::Invalid(fmt::format( "Unable to deserialize deletion vector, invalid magic number: {}", magic_num)); } - RoaringBitmap32 roaring_bitmap; - PAIMON_RETURN_NOT_OK(roaring_bitmap.Deserialize(buffer + MAGIC_NUMBER_SIZE_BYTES, - length - MAGIC_NUMBER_SIZE_BYTES)); - return pool->AllocateUnique(roaring_bitmap); + return DeserializeWithoutMagicNumber(buffer + MAGIC_NUMBER_SIZE_BYTES, + length - MAGIC_NUMBER_SIZE_BYTES, pool); } } // namespace paimon diff --git a/src/paimon/core/deletionvectors/bitmap_deletion_vector.h b/src/paimon/core/deletionvectors/bitmap_deletion_vector.h index 21361dba..fd93a6da 100644 --- a/src/paimon/core/deletionvectors/bitmap_deletion_vector.h +++ b/src/paimon/core/deletionvectors/bitmap_deletion_vector.h @@ -71,6 +71,9 @@ class BitmapDeletionVector : public DeletionVector { static Result> Deserialize(const char* buffer, int32_t length, MemoryPool* pool); + static Result> DeserializeWithoutMagicNumber( + const char* buffer, int32_t length, MemoryPool* pool); + private: Status CheckPosition(int64_t position) const; diff --git a/src/paimon/core/deletionvectors/deletion_file_writer_test.cpp b/src/paimon/core/deletionvectors/deletion_file_writer_test.cpp new file mode 100644 index 00000000..9ec2d25c --- /dev/null +++ b/src/paimon/core/deletionvectors/deletion_file_writer_test.cpp @@ -0,0 +1,122 @@ +/* + * 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/deletion_file_writer.h" + +#include +#include +#include + +#include "gtest/gtest.h" +#include "paimon/core/deletionvectors/bitmap_deletion_vector.h" +#include "paimon/core/deletionvectors/deletion_vectors_index_file.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 { + +TEST(DeletionFileWriterTest, WriteCloseAndReadBack) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + ASSERT_OK_AND_ASSIGN(auto writer, DeletionFileWriter::Create(path_factory, fs, pool)); + + RoaringBitmap32 roaring_1; + roaring_1.Add(1); + roaring_1.Add(3); + roaring_1.Add(5); + auto dv_1 = std::make_shared(roaring_1); + + RoaringBitmap32 roaring_2; + roaring_2.Add(100); + auto dv_2 = std::make_shared(roaring_2); + + ASSERT_OK(writer->Write("data-file-1", dv_1)); + ASSERT_OK(writer->Write("data-file-2", dv_2)); + ASSERT_OK(writer->Close()); + + ASSERT_OK_AND_ASSIGN(auto meta_unique, writer->GetResult()); + ASSERT_EQ(meta_unique->IndexType(), DeletionVectorsIndexFile::DELETION_VECTORS_INDEX); + ASSERT_EQ(meta_unique->FileName(), "index-0"); + ASSERT_EQ(meta_unique->RowCount(), 2); + ASSERT_EQ(meta_unique->ExternalPath(), std::nullopt); + ASSERT_GT(meta_unique->FileSize(), 0); + + const auto& dv_ranges = meta_unique->DvRanges(); + ASSERT_TRUE(dv_ranges.has_value()); + ASSERT_EQ(dv_ranges->size(), 2); + + auto first = dv_ranges->find("data-file-1"); + auto second = dv_ranges->find("data-file-2"); + ASSERT_NE(first, dv_ranges->end()); + ASSERT_NE(second, dv_ranges->end()); + ASSERT_EQ(first->second.GetOffset(), 1); + ASSERT_GT(second->second.GetOffset(), first->second.GetOffset()); + + std::shared_ptr meta = std::move(meta_unique); + DeletionVectorsIndexFile index_file(fs, path_factory, /*bitmap64=*/false, pool); + ASSERT_OK_AND_ASSIGN(auto read_back, index_file.ReadAllDeletionVectors(meta)); + + ASSERT_EQ(read_back.size(), 2); + ASSERT_EQ(read_back.at("data-file-1")->GetCardinality(), 3); + ASSERT_EQ(read_back.at("data-file-2")->GetCardinality(), 1); + + ASSERT_OK_AND_ASSIGN(bool is_deleted, read_back.at("data-file-1")->IsDeleted(3)); + ASSERT_TRUE(is_deleted); + ASSERT_OK_AND_ASSIGN(is_deleted, read_back.at("data-file-1")->IsDeleted(4)); + ASSERT_FALSE(is_deleted); +} + +TEST(DeletionFileWriterTest, GetResultWithoutCloseShouldFail) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + ASSERT_OK_AND_ASSIGN(auto writer, DeletionFileWriter::Create(path_factory, fs, pool)); + auto result = writer->GetResult(); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("result length -1 out of int32 range") != + std::string::npos); +} + +TEST(DeletionFileWriterTest, ExternalPathInResult) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + path_factory->SetExternal(true); + auto pool = GetDefaultPool(); + + ASSERT_OK_AND_ASSIGN(auto writer, DeletionFileWriter::Create(path_factory, fs, pool)); + + RoaringBitmap32 roaring; + roaring.Add(0); + auto dv = std::make_shared(roaring); + ASSERT_OK(writer->Write("data-file-ext", dv)); + ASSERT_OK(writer->Close()); + + ASSERT_OK_AND_ASSIGN(auto meta, writer->GetResult()); + ASSERT_TRUE(meta->ExternalPath().has_value()); + ASSERT_EQ(meta->ExternalPath().value(), PathUtil::JoinPath(dir->Str(), "index-0")); +} + +} // namespace paimon::test diff --git a/src/paimon/core/deletionvectors/deletion_vector.cpp b/src/paimon/core/deletionvectors/deletion_vector.cpp index 43999a10..549fbf8f 100644 --- a/src/paimon/core/deletionvectors/deletion_vector.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector.cpp @@ -44,7 +44,7 @@ Result> DeletionVector::Read(const FileSystem* PAIMON_ASSIGN_OR_RAISE(int32_t actual_length, file_input_stream.ReadValue()); if (actual_length != deletion_file.length) { return Status::Invalid( - fmt::format("Size not match, actual size: {}, expect size: {}, , file path: {}", + fmt::format("Size not match, actual size: {}, expect size: {}, file path: {}", actual_length, deletion_file.length, deletion_file.path)); } auto bytes = Bytes::AllocateBytes(deletion_file.length, pool); @@ -75,10 +75,11 @@ Result> DeletionVector::Read(DataInputStream* PAIMON_ASSIGN_OR_RAISE([[maybe_unused]] int32_t unused_crc, input_stream->ReadValue()); - return BitmapDeletionVector::Deserialize(bytes->data(), bytes->size(), pool); + return BitmapDeletionVector::DeserializeWithoutMagicNumber(bytes->data(), bytes->size(), + pool); } else if (EndianSwapValue(magic_number) == Bitmap64DeletionVector::MAGIC_NUMBER) { return Status::NotImplemented( - "bitmap64 deletion vectors are not supported in this version; " + "bitmap64 deletion vectors are not supported in this version, " "please use bitmap deletion vectors instead or upgrade to a version " "that supports bitmap64."); } diff --git a/src/paimon/core/deletionvectors/deletion_vector_index_file_writer_test.cpp b/src/paimon/core/deletionvectors/deletion_vector_index_file_writer_test.cpp new file mode 100644 index 00000000..e804e0fd --- /dev/null +++ b/src/paimon/core/deletionvectors/deletion_vector_index_file_writer_test.cpp @@ -0,0 +1,121 @@ +/* + * 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/deletion_vector_index_file_writer.h" + +#include +#include +#include + +#include "gtest/gtest.h" +#include "paimon/core/deletionvectors/bitmap_deletion_vector.h" +#include "paimon/core/deletionvectors/deletion_vectors_index_file.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 { + +class FailingDeletionVector : public DeletionVector { + public: + Status Delete(int64_t) override { + return Status::Invalid("injected delete failure"); + } + + Result CheckedDelete(int64_t) override { + return Status::Invalid("injected checked-delete failure"); + } + + Result IsDeleted(int64_t) const override { + return Status::Invalid("injected is-deleted failure"); + } + + bool IsEmpty() const override { + return true; + } + + int64_t GetCardinality() const override { + return 0; + } + + Result SerializeTo(const std::shared_ptr&, DataOutputStream*) override { + return Status::Invalid("injected serialize failure"); + } + + Result> SerializeToBytes(const std::shared_ptr&) override { + return Status::Invalid("injected serialize failure"); + } +}; + +} // namespace + +TEST(DeletionVectorIndexFileWriterTest, WriteSingleFileRoundTrip) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + DeletionVectorIndexFileWriter writer(fs, path_factory, pool); + + std::map> input; + RoaringBitmap32 roaring_1; + roaring_1.Add(1); + roaring_1.Add(2); + input["data-a"] = std::make_shared(roaring_1); + + RoaringBitmap32 roaring_2; + roaring_2.Add(10); + input["data-b"] = std::make_shared(roaring_2); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr meta, writer.WriteSingleFile(input)); + ASSERT_EQ(meta->IndexType(), DeletionVectorsIndexFile::DELETION_VECTORS_INDEX); + ASSERT_EQ(meta->FileName(), "index-0"); + ASSERT_EQ(meta->RowCount(), 2); + + DeletionVectorsIndexFile index_file(fs, path_factory, /*bitmap64=*/false, pool); + ASSERT_OK_AND_ASSIGN(auto read_back, index_file.ReadAllDeletionVectors(meta)); + ASSERT_EQ(read_back.size(), 2); + + ASSERT_OK_AND_ASSIGN(bool is_deleted, read_back.at("data-a")->IsDeleted(1)); + ASSERT_TRUE(is_deleted); + ASSERT_OK_AND_ASSIGN(is_deleted, read_back.at("data-a")->IsDeleted(3)); + ASSERT_FALSE(is_deleted); + + ASSERT_OK_AND_ASSIGN(is_deleted, read_back.at("data-b")->IsDeleted(10)); + ASSERT_TRUE(is_deleted); +} + +TEST(DeletionVectorIndexFileWriterTest, WriteSingleFileShouldReturnSerializeError) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + DeletionVectorIndexFileWriter writer(fs, path_factory, pool); + + std::map> input; + input["bad"] = std::make_shared(); + + auto result = writer.WriteSingleFile(input); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("injected serialize failure") != std::string::npos); +} + +} // namespace paimon::test diff --git a/src/paimon/core/deletionvectors/deletion_vector_test.cpp b/src/paimon/core/deletionvectors/deletion_vector_test.cpp index 4e79dd7d..f83790db 100644 --- a/src/paimon/core/deletionvectors/deletion_vector_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector_test.cpp @@ -16,15 +16,33 @@ #include "paimon/core/deletionvectors/deletion_vector.h" +#include #include #include #include +#include #include "gtest/gtest.h" +#include "paimon/core/deletionvectors/bitmap64_deletion_vector.h" #include "paimon/core/deletionvectors/bitmap_deletion_vector.h" +#include "paimon/io/byte_array_input_stream.h" +#include "paimon/io/byte_order.h" +#include "paimon/io/data_input_stream.h" #include "paimon/testing/utils/testharness.h" namespace paimon::test { + +namespace { + +void AppendInt32BigEndian(std::vector* bytes, int32_t value) { + bytes->push_back(static_cast((value >> 24) & 0xFF)); + bytes->push_back(static_cast((value >> 16) & 0xFF)); + bytes->push_back(static_cast((value >> 8) & 0xFF)); + bytes->push_back(static_cast(value & 0xFF)); +} + +} // namespace + TEST(DeletionVectorTest, TestSimple) { std::set to_deleted; for (int32_t i = 0; i < 10000; i++) { @@ -86,4 +104,66 @@ TEST(DeletionVectorTest, TestCompatibleWithJava) { ASSERT_OK_AND_ASSIGN(auto serialized_dv, deletion_vector->SerializeToBytes(pool)); ASSERT_EQ(*serialized_dv, *serialize_bytes); } + +TEST(DeletionVectorTest, ReadFromDataInputStreamLengthMismatch) { + std::vector data; + AppendInt32BigEndian(&data, /*bitmap_length=*/8); + AppendInt32BigEndian(&data, BitmapDeletionVector::MAGIC_NUMBER); + + auto input_stream = std::make_shared( + reinterpret_cast(data.data()), data.size()); + DataInputStream in(input_stream); + auto pool = GetDefaultPool(); + + auto result = DeletionVector::Read(&in, /*length=*/9, pool.get()); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("Size not match") != std::string::npos); +} + +TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidBitmapLength) { + std::vector data; + AppendInt32BigEndian(&data, /*bitmap_length=*/3); + AppendInt32BigEndian(&data, BitmapDeletionVector::MAGIC_NUMBER); + + auto input_stream = std::make_shared( + reinterpret_cast(data.data()), data.size()); + DataInputStream in(input_stream); + auto pool = GetDefaultPool(); + + auto result = DeletionVector::Read(&in, std::nullopt, pool.get()); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("Invalid bitmap length") != std::string::npos); +} + +TEST(DeletionVectorTest, ReadFromDataInputStreamBitmap64NotImplemented) { + std::vector data; + AppendInt32BigEndian(&data, /*bitmap_length=*/8); + // Trigger: EndianSwapValue(magic_number) == Bitmap64DeletionVector::MAGIC_NUMBER. + AppendInt32BigEndian(&data, EndianSwapValue(Bitmap64DeletionVector::MAGIC_NUMBER)); + + auto input_stream = std::make_shared( + reinterpret_cast(data.data()), data.size()); + DataInputStream in(input_stream); + auto pool = GetDefaultPool(); + + auto result = DeletionVector::Read(&in, std::nullopt, pool.get()); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().IsNotImplemented()); +} + +TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidMagicNumber) { + std::vector data; + AppendInt32BigEndian(&data, /*bitmap_length=*/8); + AppendInt32BigEndian(&data, /*invalid_magic=*/123456789); + + auto input_stream = std::make_shared( + reinterpret_cast(data.data()), data.size()); + DataInputStream in(input_stream); + auto pool = GetDefaultPool(); + + auto result = DeletionVector::Read(&in, std::nullopt, pool.get()); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("Invalid magic number") != std::string::npos); +} + } // namespace paimon::test diff --git a/src/paimon/core/deletionvectors/deletion_vectors_index_file_test.cpp b/src/paimon/core/deletionvectors/deletion_vectors_index_file_test.cpp index f9de086e..bd1889c3 100644 --- a/src/paimon/core/deletionvectors/deletion_vectors_index_file_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_vectors_index_file_test.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include "gtest/gtest.h" #include "paimon/core/deletionvectors/bitmap_deletion_vector.h" @@ -55,6 +56,21 @@ TEST(DeletionVectorsIndexFileTest, Basic) { ASSERT_EQ(meta->IndexType(), DeletionVectorsIndexFile::DELETION_VECTORS_INDEX); ASSERT_EQ(meta->FileName(), "index-0"); ASSERT_EQ(meta->ExternalPath(), std::nullopt); + + // Round trip: write then read all deletion vectors from index file. + ASSERT_OK_AND_ASSIGN(auto read_back, index_file.ReadAllDeletionVectors(meta)); + ASSERT_EQ(read_back.size(), input.size()); + ASSERT_EQ(read_back.at("dv1")->GetCardinality(), 10); + ASSERT_EQ(read_back.at("dv2")->GetCardinality(), 10); + + ASSERT_OK_AND_ASSIGN(bool is_deleted, read_back.at("dv1")->IsDeleted(0)); + ASSERT_TRUE(is_deleted); + ASSERT_OK_AND_ASSIGN(is_deleted, read_back.at("dv1")->IsDeleted(10)); + ASSERT_FALSE(is_deleted); + ASSERT_OK_AND_ASSIGN(is_deleted, read_back.at("dv2")->IsDeleted(100)); + ASSERT_TRUE(is_deleted); + ASSERT_OK_AND_ASSIGN(is_deleted, read_back.at("dv2")->IsDeleted(99)); + ASSERT_FALSE(is_deleted); } TEST(DeletionVectorsIndexFileTest, ExternalPathAndIndexFileMeta) { @@ -76,6 +92,93 @@ TEST(DeletionVectorsIndexFileTest, ExternalPathAndIndexFileMeta) { ASSERT_OK_AND_ASSIGN(auto meta, index_file.WriteSingleFile(input)); ASSERT_EQ(meta->ExternalPath().value(), PathUtil::JoinPath(dir->Str(), "index-0")); + + // Round trip for external path index file. + ASSERT_OK_AND_ASSIGN(auto read_back, index_file.ReadAllDeletionVectors(meta)); + ASSERT_EQ(read_back.size(), 1); + ASSERT_EQ(read_back.at("dv_ext")->GetCardinality(), 5); + ASSERT_OK_AND_ASSIGN(bool is_deleted, read_back.at("dv_ext")->IsDeleted(0)); + ASSERT_TRUE(is_deleted); + ASSERT_OK_AND_ASSIGN(is_deleted, read_back.at("dv_ext")->IsDeleted(5)); + ASSERT_FALSE(is_deleted); +} + +TEST(DeletionVectorsIndexFileTest, RoundTripEmptyInput) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + DeletionVectorsIndexFile index_file(fs, path_factory, /*bitmap64=*/false, pool); + + std::map> input; + ASSERT_OK_AND_ASSIGN(auto meta, index_file.WriteSingleFile(input)); + ASSERT_EQ(meta->RowCount(), 0); + ASSERT_OK_AND_ASSIGN(auto read_back, index_file.ReadAllDeletionVectors(meta)); + ASSERT_TRUE(read_back.empty()); +} + +TEST(DeletionVectorsIndexFileTest, RoundTripMultipleIndexFilesMerge) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + DeletionVectorsIndexFile index_file(fs, path_factory, /*bitmap64=*/false, pool); + + std::map> input1; + RoaringBitmap32 roaring_1; + roaring_1.Add(1); + roaring_1.Add(3); + input1["dv_a"] = std::make_shared(roaring_1); + ASSERT_OK_AND_ASSIGN(auto meta1, index_file.WriteSingleFile(input1)); + + std::map> input2; + RoaringBitmap32 roaring_2; + roaring_2.Add(8); + input2["dv_b"] = std::make_shared(roaring_2); + ASSERT_OK_AND_ASSIGN(auto meta2, index_file.WriteSingleFile(input2)); + + ASSERT_OK_AND_ASSIGN(auto read_back, + index_file.ReadAllDeletionVectors( + std::vector>{meta1, meta2})); + ASSERT_EQ(read_back.size(), 2); + + ASSERT_OK_AND_ASSIGN(bool is_deleted, read_back.at("dv_a")->IsDeleted(1)); + ASSERT_TRUE(is_deleted); + ASSERT_OK_AND_ASSIGN(is_deleted, read_back.at("dv_b")->IsDeleted(8)); + ASSERT_TRUE(is_deleted); +} + +TEST(DeletionVectorsIndexFileTest, RoundTripMultipleIndexFilesLastWriteWinsOnSameKey) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + DeletionVectorsIndexFile index_file(fs, path_factory, /*bitmap64=*/false, pool); + + std::map> input1; + RoaringBitmap32 roaring_old; + roaring_old.Add(2); + input1["same_dv"] = std::make_shared(roaring_old); + ASSERT_OK_AND_ASSIGN(auto meta1, index_file.WriteSingleFile(input1)); + + std::map> input2; + RoaringBitmap32 roaring_new; + roaring_new.Add(9); + input2["same_dv"] = std::make_shared(roaring_new); + ASSERT_OK_AND_ASSIGN(auto meta2, index_file.WriteSingleFile(input2)); + + ASSERT_OK_AND_ASSIGN(auto read_back, + index_file.ReadAllDeletionVectors( + std::vector>{meta1, meta2})); + ASSERT_EQ(read_back.size(), 1); + + ASSERT_OK_AND_ASSIGN(bool is_deleted_old, read_back.at("same_dv")->IsDeleted(2)); + ASSERT_FALSE(is_deleted_old); + ASSERT_OK_AND_ASSIGN(bool is_deleted_new, read_back.at("same_dv")->IsDeleted(9)); + ASSERT_TRUE(is_deleted_new); } } // namespace paimon::test diff --git a/src/paimon/core/index/index_file_handler_test.cpp b/src/paimon/core/index/index_file_handler_test.cpp index 657cb24f..d5ee1802 100644 --- a/src/paimon/core/index/index_file_handler_test.cpp +++ b/src/paimon/core/index/index_file_handler_test.cpp @@ -245,4 +245,69 @@ TEST_F(IndexFileHandlerTest, Test09VersionScan) { ObjectUtils::Equal(index_file_metas[std::make_pair(partition, 1)], index_meta_p10_b1)); } +TEST_F(IndexFileHandlerTest, TestScanWithNoIndexManifest) { + std::string table_path = paimon::test::GetDataDir() + "/orc/pk_09.db/pk_09/"; + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, + CoreOptions::FromMap({{Options::MANIFEST_FORMAT, "orc"}})); + ASSERT_OK_AND_ASSIGN(std::unique_ptr index_file_handler, + CreateIndexFileHandler(table_path, core_options)); + + SnapshotManager snapshot_manager(core_options.GetFileSystem(), table_path); + ASSERT_OK_AND_ASSIGN(Snapshot snapshot, snapshot_manager.LoadSnapshot(/*snapshot_id=*/6)); + + Snapshot no_index_manifest_snapshot( + std::optional(snapshot.Version()), snapshot.Id(), snapshot.SchemaId(), + snapshot.BaseManifestList(), snapshot.BaseManifestListSize(), snapshot.DeltaManifestList(), + snapshot.DeltaManifestListSize(), snapshot.ChangelogManifestList(), + snapshot.ChangelogManifestListSize(), /*index_manifest=*/std::nullopt, + snapshot.CommitUser(), snapshot.CommitIdentifier(), snapshot.GetCommitKind(), + snapshot.TimeMillis(), snapshot.LogOffsets(), snapshot.TotalRecordCount(), + snapshot.DeltaRecordCount(), snapshot.ChangelogRecordCount(), snapshot.Watermark(), + snapshot.Statistics(), snapshot.Properties(), snapshot.NextRowId()); + + auto partition = BinaryRowGenerator::GenerateRow({10}, memory_pool_.get()); + std::unordered_set partitions = {partition}; + ASSERT_OK_AND_ASSIGN( + auto index_file_metas, + index_file_handler->Scan(no_index_manifest_snapshot, + std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), + partitions)); + ASSERT_TRUE(index_file_metas.empty()); + + ASSERT_OK_AND_ASSIGN( + auto index_entries, + index_file_handler->Scan(no_index_manifest_snapshot, + [](const IndexManifestEntry&) -> Result { return true; })); + ASSERT_TRUE(index_entries.empty()); +} + +TEST_F(IndexFileHandlerTest, TestScanByPartitionBucketAndReadAllDeletionVectors) { + std::string table_path = paimon::test::GetDataDir() + + "/orc/pk_table_with_dv_cardinality.db/pk_table_with_dv_cardinality/"; + + ASSERT_OK_AND_ASSIGN(CoreOptions core_options, + CoreOptions::FromMap({{Options::MANIFEST_FORMAT, "orc"}})); + ASSERT_OK_AND_ASSIGN(std::unique_ptr index_file_handler, + CreateIndexFileHandler(table_path, core_options)); + + SnapshotManager snapshot_manager(core_options.GetFileSystem(), table_path); + ASSERT_OK_AND_ASSIGN(Snapshot snapshot, snapshot_manager.LoadSnapshot(/*snapshot_id=*/4)); + + auto partition = BinaryRowGenerator::GenerateRow({10}, memory_pool_.get()); + ASSERT_OK_AND_ASSIGN( + auto index_file_metas, + index_file_handler->Scan( + snapshot, std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), partition, + /*bucket=*/0)); + ASSERT_EQ(index_file_metas.size(), 1); + + ASSERT_OK_AND_ASSIGN(auto deletion_vectors, index_file_handler->ReadAllDeletionVectors( + partition, /*bucket=*/0, index_file_metas)); + ASSERT_EQ(deletion_vectors.size(), 1); + ASSERT_TRUE(deletion_vectors.find("data-0d0f29cc-63c6-4fab-a594-71bd7d06fcde-0.orc") != + deletion_vectors.end()); + ASSERT_EQ(deletion_vectors["data-0d0f29cc-63c6-4fab-a594-71bd7d06fcde-0.orc"]->GetCardinality(), + 1); +} + } // namespace paimon::test From 1a58462bf9bb59a93d301a188d98851a42b2ea99 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 13 Mar 2026 13:26:43 +0800 Subject: [PATCH 06/27] fix --- src/paimon/core/deletionvectors/deletion_vector_test.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/paimon/core/deletionvectors/deletion_vector_test.cpp b/src/paimon/core/deletionvectors/deletion_vector_test.cpp index f83790db..6b93dd52 100644 --- a/src/paimon/core/deletionvectors/deletion_vector_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector_test.cpp @@ -107,7 +107,7 @@ TEST(DeletionVectorTest, TestCompatibleWithJava) { TEST(DeletionVectorTest, ReadFromDataInputStreamLengthMismatch) { std::vector data; - AppendInt32BigEndian(&data, /*bitmap_length=*/8); + AppendInt32BigEndian(&data, /*value=*/8); AppendInt32BigEndian(&data, BitmapDeletionVector::MAGIC_NUMBER); auto input_stream = std::make_shared( @@ -122,7 +122,7 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamLengthMismatch) { TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidBitmapLength) { std::vector data; - AppendInt32BigEndian(&data, /*bitmap_length=*/3); + AppendInt32BigEndian(&data, /*value=*/3); AppendInt32BigEndian(&data, BitmapDeletionVector::MAGIC_NUMBER); auto input_stream = std::make_shared( @@ -137,7 +137,7 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidBitmapLength) { TEST(DeletionVectorTest, ReadFromDataInputStreamBitmap64NotImplemented) { std::vector data; - AppendInt32BigEndian(&data, /*bitmap_length=*/8); + AppendInt32BigEndian(&data, /*value=*/8); // Trigger: EndianSwapValue(magic_number) == Bitmap64DeletionVector::MAGIC_NUMBER. AppendInt32BigEndian(&data, EndianSwapValue(Bitmap64DeletionVector::MAGIC_NUMBER)); @@ -153,7 +153,7 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamBitmap64NotImplemented) { TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidMagicNumber) { std::vector data; - AppendInt32BigEndian(&data, /*bitmap_length=*/8); + AppendInt32BigEndian(&data, /*value=*/8); AppendInt32BigEndian(&data, /*invalid_magic=*/123456789); auto input_stream = std::make_shared( From ddef2d771a9c0320ba591e114eafb819944fd769 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 13 Mar 2026 14:38:03 +0800 Subject: [PATCH 07/27] fix --- src/paimon/CMakeLists.txt | 1 + .../core/compact/compact_deletion_file.h | 5 + .../compact/compact_deletion_file_test.cpp | 235 ++++++++++++++++++ 3 files changed, 241 insertions(+) create mode 100644 src/paimon/core/compact/compact_deletion_file_test.cpp diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index 90ff16a3..b1a593fc 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -504,6 +504,7 @@ if(PAIMON_BUILD_TESTS) core/catalog/file_system_catalog_test.cpp core/catalog/catalog_test.cpp core/catalog/identifier_test.cpp + core/compact/compact_deletion_file_test.cpp core/core_options_test.cpp core/deletionvectors/apply_deletion_vector_batch_reader_test.cpp core/deletionvectors/bitmap_deletion_vector_test.cpp diff --git a/src/paimon/core/compact/compact_deletion_file.h b/src/paimon/core/compact/compact_deletion_file.h index 82db8380..2223c3d4 100644 --- a/src/paimon/core/compact/compact_deletion_file.h +++ b/src/paimon/core/compact/compact_deletion_file.h @@ -26,10 +26,14 @@ namespace paimon { +/// Deletion File from compaction. class CompactDeletionFile { public: virtual ~CompactDeletionFile() = default; + /// Used by async compaction, when compaction task is completed, deletions file will be + /// generated immediately, so when updateCompactResult, we need to merge old deletion files + /// (just delete them). static Result> GenerateFiles( const std::shared_ptr& maintainer); @@ -41,6 +45,7 @@ class CompactDeletionFile { virtual void Clean() = 0; }; +/// A generated files implementation of `CompactDeletionFile`. class GeneratedDeletionFile : public CompactDeletionFile, public std::enable_shared_from_this { public: diff --git a/src/paimon/core/compact/compact_deletion_file_test.cpp b/src/paimon/core/compact/compact_deletion_file_test.cpp new file mode 100644 index 00000000..884d2273 --- /dev/null +++ b/src/paimon/core/compact/compact_deletion_file_test.cpp @@ -0,0 +1,235 @@ +/* + * 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/compact/compact_deletion_file.h" + +#include +#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 CreateMaintainer( + const std::shared_ptr& fs, + const std::shared_ptr& path_factory, + const std::shared_ptr& pool, + const std::map>& deletion_vectors) { + auto dv_index_file = + std::make_shared(fs, path_factory, /*bitmap64=*/false, pool); + return std::make_shared(dv_index_file, deletion_vectors); +} + +class TestGeneratedDeletionFile : public GeneratedDeletionFile { + public: + using GeneratedDeletionFile::GeneratedDeletionFile; + + void Clean() override { + cleaned_ = true; + } + + bool Cleaned() const { + return cleaned_; + } + + private: + bool cleaned_ = false; +}; + +class NonGeneratedCompactDeletionFile : public CompactDeletionFile { + public: + std::optional> GetOrCompute() override { + return std::nullopt; + } + + Result> MergeOldFile( + const std::shared_ptr&) override { + return Status::Invalid("not used in this test"); + } + + void Clean() override {} +}; + +} // namespace + +TEST(CompactDeletionFileTest, GenerateFilesShouldReturnFileWhenModified) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + RoaringBitmap32 roaring; + roaring.Add(1); + std::map> deletion_vectors; + deletion_vectors["data-a"] = std::make_shared(roaring); + + auto maintainer = CreateMaintainer(fs, path_factory, pool, deletion_vectors); + maintainer->RemoveDeletionVectorOf("data-a"); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr generated, + CompactDeletionFile::GenerateFiles(maintainer)); + auto file = generated->GetOrCompute(); + ASSERT_TRUE(file.has_value()); + ASSERT_NE(file.value(), nullptr); + ASSERT_EQ(file.value()->IndexType(), DeletionVectorsIndexFile::DELETION_VECTORS_INDEX); +} + +TEST(CompactDeletionFileTest, GenerateFilesShouldReturnNulloptWhenNotModified) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + std::map> deletion_vectors; + auto maintainer = CreateMaintainer(fs, path_factory, pool, deletion_vectors); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr generated, + CompactDeletionFile::GenerateFiles(maintainer)); + auto file = generated->GetOrCompute(); + ASSERT_FALSE(file.has_value()); +} + +TEST(CompactDeletionFileTest, MergeOldFileShouldRejectNonGeneratedType) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + auto dv_index_file = + std::make_shared(fs, path_factory, /*bitmap64=*/false, pool); + auto meta = std::make_shared( + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "index-0", 1, 0, + LinkedHashMap(), std::nullopt); + + auto current = std::make_shared(meta, dv_index_file); + auto old = std::make_shared(); + + auto result = current->MergeOldFile(old); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("old should be a GeneratedDeletionFile") != + std::string::npos); +} + +TEST(CompactDeletionFileTest, MergeOldFileShouldRejectInvokedOldFile) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + auto dv_index_file = + std::make_shared(fs, path_factory, /*bitmap64=*/false, pool); + auto current_meta = std::make_shared( + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "index-current", 1, 0, + LinkedHashMap(), std::nullopt); + auto old_meta = std::make_shared( + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "index-old", 1, 0, + LinkedHashMap(), std::nullopt); + + auto current = std::make_shared(current_meta, dv_index_file); + auto old = std::make_shared(old_meta, dv_index_file); + ASSERT_TRUE(old->GetOrCompute().has_value()); + + auto result = current->MergeOldFile(old); + ASSERT_FALSE(result.ok()); + ASSERT_TRUE(result.status().ToString().find("old should not be get") != std::string::npos); +} + +TEST(CompactDeletionFileTest, MergeOldFileShouldReturnOldWhenCurrentIsNull) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + auto dv_index_file = + std::make_shared(fs, path_factory, /*bitmap64=*/false, pool); + auto old_meta = std::make_shared( + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "index-old", 1, 0, + LinkedHashMap(), std::nullopt); + + auto current = std::make_shared(nullptr, dv_index_file); + auto old = std::make_shared(old_meta, dv_index_file); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr merged, current->MergeOldFile(old)); + ASSERT_EQ(merged.get(), old.get()); + ASSERT_FALSE(old->Cleaned()); +} + +TEST(CompactDeletionFileTest, MergeOldFileShouldCleanOldAndReturnCurrent) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + auto dv_index_file = + std::make_shared(fs, path_factory, /*bitmap64=*/false, pool); + auto current_meta = std::make_shared( + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "index-current", 1, 0, + LinkedHashMap(), std::nullopt); + auto old_meta = std::make_shared( + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "index-old", 1, 0, + LinkedHashMap(), std::nullopt); + + auto current = std::make_shared(current_meta, dv_index_file); + auto old = std::make_shared(old_meta, dv_index_file); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr merged, current->MergeOldFile(old)); + ASSERT_EQ(merged.get(), current.get()); + ASSERT_TRUE(old->Cleaned()); +} + +TEST(CompactDeletionFileTest, CleanShouldDeleteIndexFile) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_OK_AND_ASSIGN(std::shared_ptr fs, + FileSystemFactory::Get("local", dir->Str(), {})); + auto path_factory = std::make_shared(dir->Str()); + auto pool = GetDefaultPool(); + + auto dv_index_file = + std::make_shared(fs, path_factory, /*bitmap64=*/false, pool); + + auto file_meta = std::make_shared( + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "index-clean-target", 1, 0, + LinkedHashMap(), std::nullopt); + const std::string file_path = PathUtil::JoinPath(dir->Str(), file_meta->FileName()); + ASSERT_OK_AND_ASSIGN(std::shared_ptr out, + fs->Create(file_path, /*overwrite=*/true)); + ASSERT_OK(out->Close()); + + ASSERT_OK_AND_ASSIGN(bool exists_before, dv_index_file->Exists(file_meta)); + ASSERT_TRUE(exists_before); + + GeneratedDeletionFile generated(file_meta, dv_index_file); + generated.Clean(); + + ASSERT_OK_AND_ASSIGN(bool exists_after, dv_index_file->Exists(file_meta)); + ASSERT_FALSE(exists_after); +} + +} // namespace paimon::test From dd579466b25cf570ccec52f92ef0433c9c0c351c Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 13 Mar 2026 14:44:05 +0800 Subject: [PATCH 08/27] fix --- src/paimon/core/deletionvectors/deletion_vector_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/paimon/core/deletionvectors/deletion_vector_test.cpp b/src/paimon/core/deletionvectors/deletion_vector_test.cpp index 6b93dd52..01913b0f 100644 --- a/src/paimon/core/deletionvectors/deletion_vector_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector_test.cpp @@ -154,7 +154,7 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamBitmap64NotImplemented) { TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidMagicNumber) { std::vector data; AppendInt32BigEndian(&data, /*value=*/8); - AppendInt32BigEndian(&data, /*invalid_magic=*/123456789); + AppendInt32BigEndian(&data, /*value=*/123456789); auto input_stream = std::make_shared( reinterpret_cast(data.data()), data.size()); From c2a446f44d4c0ee55d0c23359648870674243abb Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 13 Mar 2026 15:50:07 +0800 Subject: [PATCH 09/27] fix --- src/paimon/CMakeLists.txt | 1 + .../operation/abstract_file_store_write.cpp | 8 +-- .../operation/data_evolution_split_read.cpp | 2 +- .../file_system_write_restore_test.cpp | 72 +++++++++++++++++++ 4 files changed, 77 insertions(+), 6 deletions(-) create mode 100644 src/paimon/core/operation/file_system_write_restore_test.cpp diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index b1a593fc..0c94e8cf 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -589,6 +589,7 @@ if(PAIMON_BUILD_TESTS) core/operation/append_only_file_store_scan_test.cpp core/operation/key_value_file_store_scan_test.cpp core/operation/file_store_scan_test.cpp + core/operation/file_system_write_restore_test.cpp core/operation/file_store_write_test.cpp core/operation/manifest_file_merger_test.cpp core/operation/merge_file_split_read_test.cpp diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index 565219e3..1078be03 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -74,15 +74,13 @@ AbstractFileStoreWrite::AbstractFileStoreWrite( partition_schema_(partition_schema), dv_maintainer_factory_(dv_maintainer_factory), options_(options), + compact_executor_(CreateDefaultExecutor(4)), + compaction_metrics_(std::make_shared()), ignore_previous_files_(ignore_previous_files), is_streaming_mode_(is_streaming_mode), ignore_num_bucket_check_(ignore_num_bucket_check), metrics_(std::make_shared()), - logger_(Logger::GetLogger("AbstractFileStoreWrite")) { - // TODO(yonghao.fyh): support with - compact_executor_ = CreateDefaultExecutor(4); - compaction_metrics_ = std::make_shared(); -} + logger_(Logger::GetLogger("AbstractFileStoreWrite")) {} Status AbstractFileStoreWrite::Write(std::unique_ptr&& batch) { if (PAIMON_UNLIKELY(batch == nullptr)) { diff --git a/src/paimon/core/operation/data_evolution_split_read.cpp b/src/paimon/core/operation/data_evolution_split_read.cpp index 26bb006c..8ea69c8f 100644 --- a/src/paimon/core/operation/data_evolution_split_read.cpp +++ b/src/paimon/core/operation/data_evolution_split_read.cpp @@ -158,7 +158,7 @@ Result> DataEvolutionSplitRead::InnerCreateReader( std::vector> raw_file_readers, CreateRawFileReaders(split_impl->Partition(), need_merge_files, raw_read_schema_, /*predicate=*/nullptr, - /*deletion_file_map=*/{}, row_ranges, data_file_path_factory)); + /*dv_factory=*/nullptr, row_ranges, data_file_path_factory)); assert(raw_file_readers.size() == 1); sub_readers.push_back(std::move(raw_file_readers[0])); } else { diff --git a/src/paimon/core/operation/file_system_write_restore_test.cpp b/src/paimon/core/operation/file_system_write_restore_test.cpp new file mode 100644 index 00000000..f6dab274 --- /dev/null +++ b/src/paimon/core/operation/file_system_write_restore_test.cpp @@ -0,0 +1,72 @@ +/* + * 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/file_system_write_restore.h" + +#include +#include + +#include "gtest/gtest.h" +#include "paimon/core/utils/snapshot_manager.h" +#include "paimon/fs/local/local_file_system.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::test { + +TEST(FileSystemWriteRestoreTest, LatestCommittedIdentifierNoSnapshot) { + auto fs = std::make_shared(); + auto snapshot_manager = std::make_shared( + fs, paimon::test::GetDataDir() + "/orc/append_09.db/not_exist"); + + FileSystemWriteRestore restore(snapshot_manager, /*scan=*/nullptr, + /*index_file_handler=*/nullptr); + + ASSERT_OK_AND_ASSIGN(int64_t latest_identifier, + restore.LatestCommittedIdentifier("unknown_user")); + ASSERT_EQ(latest_identifier, std::numeric_limits::min()); +} + +TEST(FileSystemWriteRestoreTest, LatestCommittedIdentifierWithSnapshot) { + auto fs = std::make_shared(); + auto snapshot_manager = std::make_shared( + fs, paimon::test::GetDataDir() + "/orc/append_09.db/append_09"); + + FileSystemWriteRestore restore(snapshot_manager, /*scan=*/nullptr, + /*index_file_handler=*/nullptr); + + ASSERT_OK_AND_ASSIGN(int64_t latest_identifier, + restore.LatestCommittedIdentifier("b02e4322-9c5f-41e1-a560-c0156fdf7b9c")); + ASSERT_EQ(latest_identifier, std::numeric_limits::max()); +} + +TEST(FileSystemWriteRestoreTest, GetRestoreFilesReturnsEmptyWhenNoLatestSnapshot) { + auto fs = std::make_shared(); + auto snapshot_manager = std::make_shared( + fs, paimon::test::GetDataDir() + "/orc/append_09.db/not_exist"); + + FileSystemWriteRestore restore(snapshot_manager, /*scan=*/nullptr, + /*index_file_handler=*/nullptr); + + ASSERT_OK_AND_ASSIGN(std::shared_ptr files, + restore.GetRestoreFiles(BinaryRow::EmptyRow(), /*bucket=*/0, + /*scan_delete_vectors_index=*/true)); + ASSERT_FALSE(files->GetSnapshot().has_value()); + ASSERT_FALSE(files->TotalBuckets().has_value()); + ASSERT_TRUE(files->DataFiles().empty()); + ASSERT_TRUE(files->DeleteVectorsIndex().empty()); +} + +} // namespace paimon::test From 9fb1ba85cb1ac7dd3a53f30f3d8d852c5f64e0f3 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Fri, 13 Mar 2026 17:49:33 +0800 Subject: [PATCH 10/27] fix --- src/paimon/core/index/index_file_handler_test.cpp | 12 ++++++++---- .../core/index/index_file_meta_serializer_test.cpp | 9 ++++++--- .../core/operation/abstract_file_store_write.cpp | 12 +++++++++++- .../core/operation/append_only_file_store_write.cpp | 9 ++++----- .../core/operation/file_system_write_restore.h | 12 ++++++------ src/paimon/core/utils/commit_increment.h | 8 ++++++++ 6 files changed, 43 insertions(+), 19 deletions(-) diff --git a/src/paimon/core/index/index_file_handler_test.cpp b/src/paimon/core/index/index_file_handler_test.cpp index d5ee1802..e89d19c5 100644 --- a/src/paimon/core/index/index_file_handler_test.cpp +++ b/src/paimon/core/index/index_file_handler_test.cpp @@ -172,7 +172,8 @@ TEST_F(IndexFileHandlerTest, TestScan) { /*length=*/22, /*cardinality=*/1)); std::vector> index_meta_p10_b0 = { std::make_shared( - "DELETION_VECTORS", "index-86356766-3238-46e6-990b-656cd7409eaa-0", + std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), + "index-86356766-3238-46e6-990b-656cd7409eaa-0", /*file_size=*/31, /*row_count=*/1, dv_meta_p10_b0, /*external_path=*/std::nullopt)}; LinkedHashMap dv_meta_p10_b1; @@ -182,7 +183,8 @@ TEST_F(IndexFileHandlerTest, TestScan) { /*length=*/24, /*cardinality=*/2)); std::vector> index_meta_p10_b1 = { std::make_shared( - "DELETION_VECTORS", "index-86356766-3238-46e6-990b-656cd7409eaa-1", + std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), + "index-86356766-3238-46e6-990b-656cd7409eaa-1", /*file_size=*/33, /*row_count=*/1, dv_meta_p10_b1, /*external_path=*/std::nullopt)}; ASSERT_TRUE( ObjectUtils::Equal(index_file_metas[std::make_pair(partition, 0)], index_meta_p10_b0)); @@ -227,7 +229,8 @@ TEST_F(IndexFileHandlerTest, Test09VersionScan) { /*length=*/22, /*cardinality=*/std::nullopt)); std::vector> index_meta_p10_b0 = { std::make_shared( - "DELETION_VECTORS", "index-7badd250-6c0b-49e9-8e40-2449ae9a2539-0", + std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), + "index-7badd250-6c0b-49e9-8e40-2449ae9a2539-0", /*file_size=*/61, /*row_count=*/2, dv_meta_p10_b0, /*external_path=*/std::nullopt)}; LinkedHashMap dv_meta_p10_b1; @@ -237,7 +240,8 @@ TEST_F(IndexFileHandlerTest, Test09VersionScan) { /*length=*/22, /*cardinality=*/std::nullopt)); std::vector> index_meta_p10_b1 = { std::make_shared( - "DELETION_VECTORS", "index-7badd250-6c0b-49e9-8e40-2449ae9a2539-1", + std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), + "index-7badd250-6c0b-49e9-8e40-2449ae9a2539-1", /*file_size=*/31, /*row_count=*/1, dv_meta_p10_b1, /*external_path=*/std::nullopt)}; ASSERT_TRUE( ObjectUtils::Equal(index_file_metas[std::make_pair(partition, 0)], index_meta_p10_b0)); diff --git a/src/paimon/core/index/index_file_meta_serializer_test.cpp b/src/paimon/core/index/index_file_meta_serializer_test.cpp index af00f69d..2593d246 100644 --- a/src/paimon/core/index/index_file_meta_serializer_test.cpp +++ b/src/paimon/core/index/index_file_meta_serializer_test.cpp @@ -21,6 +21,7 @@ #include "gtest/gtest.h" #include "paimon/common/io/memory_segment_output_stream.h" #include "paimon/common/memory/memory_segment_utils.h" +#include "paimon/core/deletionvectors/deletion_vectors_index_file.h" #include "paimon/core/index/index_file_meta.h" #include "paimon/io/byte_array_input_stream.h" #include "paimon/io/data_input_stream.h" @@ -47,8 +48,9 @@ class IndexFileMetaSerializerTest : public testing::Test { "my_file_name2", DeletionVectorMeta("my_file_name2", std::rand(), std::rand(), std::nullopt)); return std::make_shared( - "DELETION_VECTORS", "deletion_vectors_index_file_name" + std::to_string(std::rand()), - std::rand(), rand(), deletion_vectors_ranges, /*external_path=*/std::nullopt); + std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), + "deletion_vectors_index_file_name" + std::to_string(std::rand()), std::rand(), rand(), + deletion_vectors_ranges, /*external_path=*/std::nullopt); } const int32_t TRIES = 100; @@ -82,7 +84,8 @@ TEST_F(IndexFileMetaSerializerTest, TestToFromRow) { TEST_F(IndexFileMetaSerializerTest, TestToFromRowWithNullDeletionVectorMetas) { IndexFileMetaSerializer serializer(memory_pool_); auto expected = std::make_shared( - "DELETION_VECTORS", "deletion_vectors_index_file_0", /*file_size=*/10, + std::string(DeletionVectorsIndexFile::DELETION_VECTORS_INDEX), + "deletion_vectors_index_file_0", /*file_size=*/10, /*row_count=*/5, /*dv_ranges=*/std::nullopt, /*external_path=*/std::nullopt); ASSERT_OK_AND_ASSIGN(BinaryRow row, serializer.ToRow(expected)); diff --git a/src/paimon/core/operation/abstract_file_store_write.cpp b/src/paimon/core/operation/abstract_file_store_write.cpp index 1078be03..5448a06c 100644 --- a/src/paimon/core/operation/abstract_file_store_write.cpp +++ b/src/paimon/core/operation/abstract_file_store_write.cpp @@ -184,9 +184,19 @@ Result>> AbstractFileStoreWrite::Prep WriterContainer& writer_container = bucket_iter->second; PAIMON_ASSIGN_OR_RAISE(CommitIncrement increment, writer_container.writer->PrepareCommit(wait_compaction)); + auto compact_deletion_file = increment.GetCompactDeletionFile(); + auto& compact_increment = increment.GetCompactIncrement(); + if (compact_deletion_file) { + std::optional> dv_index_file_meta = + compact_deletion_file->GetOrCompute(); + if (dv_index_file_meta) { + compact_increment.AddNewIndexFiles({dv_index_file_meta.value()}); + } + } + auto committable = std::make_shared( partition, bucket, writer_container.total_buckets, increment.GetNewFilesIncrement(), - increment.GetCompactIncrement()); + compact_increment); result.push_back(committable); if (!committable->IsEmpty()) { writer_container.last_modified_commit_identifier = commit_identifier; 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 35d862d4..6ffbd500 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -154,8 +154,6 @@ Result>> AppendOnlyFileStoreWrite::Com } rewriter_guard.Release(); PAIMON_RETURN_NOT_OK(rewriter->Close()); - reader_guard.Release(); - reader->Close(); return rewriter->GetResult(); } @@ -170,15 +168,16 @@ Result> AppendOnlyFileStoreWrite::CreateWriter( std::shared_ptr compact_manager; auto schemas = BlobUtils::SeparateBlobSchema(write_schema_); - if (options_.WriteOnly() || options_.GetBucket() == -1 || with_blob_) { - // TODO(yonghao.fyh): check data evolution + if (options_.WriteOnly() || options_.DataEvolutionEnabled() || options_.GetBucket() == -1 || + with_blob_) { compact_manager = std::make_shared(); } else { auto dv_factory = [dv_maintainer]( const std::string& file_name) -> Result> { if (dv_maintainer) { - return dv_maintainer->DeletionVectorOf(file_name).value_or(nullptr); + return dv_maintainer->DeletionVectorOf(file_name).value_or( + std::shared_ptr()); } return std::shared_ptr(); }; diff --git a/src/paimon/core/operation/file_system_write_restore.h b/src/paimon/core/operation/file_system_write_restore.h index 9acf3f52..7a24c14b 100644 --- a/src/paimon/core/operation/file_system_write_restore.h +++ b/src/paimon/core/operation/file_system_write_restore.h @@ -52,7 +52,8 @@ class FileSystemWriteRestore : public WriteRestore { } Result> GetRestoreFiles( - const BinaryRow& partition, int32_t bucket, bool scan_delete_vectors_index) const override { + const BinaryRow& partition, int32_t bucket, + bool scan_deletion_vectors_index) 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, @@ -61,16 +62,15 @@ class FileSystemWriteRestore : public WriteRestore { 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; + std::vector> restore_data_files; PAIMON_ASSIGN_OR_RAISE(std::optional total_buckets, - WriteRestore::ExtractDataFiles(entries, &restore_files)); + WriteRestore::ExtractDataFiles(entries, &restore_data_files)); std::vector> deletion_vectors_index; - if (scan_delete_vectors_index) { + if (scan_deletion_vectors_index) { PAIMON_ASSIGN_OR_RAISE( deletion_vectors_index, index_file_handler_->Scan( @@ -78,7 +78,7 @@ class FileSystemWriteRestore : public WriteRestore { partition, bucket)); } - return std::make_shared(snapshot, total_buckets, restore_files, + return std::make_shared(snapshot, total_buckets, restore_data_files, /*dynamic_bucket_index=*/nullptr, deletion_vectors_index); } diff --git a/src/paimon/core/utils/commit_increment.h b/src/paimon/core/utils/commit_increment.h index 89a72c4e..94930ffa 100644 --- a/src/paimon/core/utils/commit_increment.h +++ b/src/paimon/core/utils/commit_increment.h @@ -41,6 +41,14 @@ class CommitIncrement { return compact_increment_; } + DataIncrement& GetNewFilesIncrement() { + return data_increment_; + } + + CompactIncrement& GetCompactIncrement() { + return compact_increment_; + } + std::shared_ptr GetCompactDeletionFile() const { return compact_deletion_file_; } From d6e443ccd3f38b93589c3ba22c63ad33fd62950c Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 16 Mar 2026 09:30:24 +0800 Subject: [PATCH 11/27] fix --- src/paimon/core/operation/data_evolution_split_read.cpp | 9 ++++----- .../core/operation/file_system_write_restore_test.cpp | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/paimon/core/operation/data_evolution_split_read.cpp b/src/paimon/core/operation/data_evolution_split_read.cpp index 8ea69c8f..61a6e4c8 100644 --- a/src/paimon/core/operation/data_evolution_split_read.cpp +++ b/src/paimon/core/operation/data_evolution_split_read.cpp @@ -333,11 +333,10 @@ Result> DataEvolutionSplitRead::CreateU if (!read_fields_in_file.empty()) { // create new FieldMappingReader for read partial fields auto file_read_schema = DataField::ConvertDataFieldsToArrowSchema(read_fields_in_file); - PAIMON_ASSIGN_OR_RAISE( - std::vector> file_readers, - CreateRawFileReaders(partition, bunch->Files(), file_read_schema, - /*predicate=*/nullptr, /*deletion_file_map=*/{}, row_ranges, - data_file_path_factory)); + PAIMON_ASSIGN_OR_RAISE(std::vector> file_readers, + CreateRawFileReaders(partition, bunch->Files(), file_read_schema, + /*predicate=*/nullptr, /*dv_factory=*/{}, + row_ranges, data_file_path_factory)); if (file_readers.size() == 1) { file_batch_readers[file_idx] = std::move(file_readers[0]); } else { diff --git a/src/paimon/core/operation/file_system_write_restore_test.cpp b/src/paimon/core/operation/file_system_write_restore_test.cpp index f6dab274..73043e5f 100644 --- a/src/paimon/core/operation/file_system_write_restore_test.cpp +++ b/src/paimon/core/operation/file_system_write_restore_test.cpp @@ -62,7 +62,7 @@ TEST(FileSystemWriteRestoreTest, GetRestoreFilesReturnsEmptyWhenNoLatestSnapshot ASSERT_OK_AND_ASSIGN(std::shared_ptr files, restore.GetRestoreFiles(BinaryRow::EmptyRow(), /*bucket=*/0, - /*scan_delete_vectors_index=*/true)); + /*scan_deletion_vectors_index=*/true)); ASSERT_FALSE(files->GetSnapshot().has_value()); ASSERT_FALSE(files->TotalBuckets().has_value()); ASSERT_TRUE(files->DataFiles().empty()); From fa005e28763ab109f54e8df1c55f35dc2f61f4aa Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 16 Mar 2026 10:22:41 +0800 Subject: [PATCH 12/27] fix --- include/paimon/status.h | 11 +++++++++++ src/paimon/common/utils/status.cpp | 3 +++ .../append/bucketed_append_compact_manager.cpp | 11 ++++++++++- .../append/bucketed_append_compact_manager.h | 7 ++++++- .../bucketed_append_compact_manager_test.cpp | 4 +++- src/paimon/core/compact/compact_future_manager.h | 12 +++++++++--- .../operation/append_only_file_store_write.cpp | 16 +++++++++++----- .../operation/append_only_file_store_write.h | 4 +++- 8 files changed, 56 insertions(+), 12 deletions(-) diff --git a/include/paimon/status.h b/include/paimon/status.h index 2de31a77..3598b2e9 100644 --- a/include/paimon/status.h +++ b/include/paimon/status.h @@ -53,6 +53,7 @@ enum class StatusCode : char { IOError = 5, CapacityError = 6, IndexError = 7, + Cancelled = 8, UnknownError = 9, NotImplemented = 10, SerializationError = 11, @@ -176,6 +177,12 @@ class PAIMON_MUST_USE_TYPE PAIMON_EXPORT Status : public util::EqualityComparabl return Status::FromArgs(StatusCode::IndexError, std::forward(args)...); } + /// Return an error status for cancelled operation + template + static Status Cancelled(Args&&... args) { + return Status::FromArgs(StatusCode::Cancelled, std::forward(args)...); + } + /// Return an error status when a container's capacity would exceed its limits template static Status CapacityError(Args&&... args) { @@ -223,6 +230,10 @@ class PAIMON_MUST_USE_TYPE PAIMON_EXPORT Status : public util::EqualityComparabl bool IsInvalid() const { return code() == StatusCode::Invalid; } + /// Return true iff the status indicates a cancelled operation. + bool IsCancelled() const { + return code() == StatusCode::Cancelled; + } /// Return true iff the status indicates an IO-related failure. bool IsIOError() const { return code() == StatusCode::IOError; diff --git a/src/paimon/common/utils/status.cpp b/src/paimon/common/utils/status.cpp index 0ee1a533..183f930c 100644 --- a/src/paimon/common/utils/status.cpp +++ b/src/paimon/common/utils/status.cpp @@ -93,6 +93,9 @@ std::string Status::CodeAsString(StatusCode code) { case StatusCode::IndexError: type = "Index error"; break; + case StatusCode::Cancelled: + type = "Cancelled"; + break; case StatusCode::UnknownError: type = "Unknown error"; break; diff --git a/src/paimon/core/append/bucketed_append_compact_manager.cpp b/src/paimon/core/append/bucketed_append_compact_manager.cpp index d40dce16..45798207 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager.cpp @@ -25,7 +25,8 @@ 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, const std::shared_ptr& reporter) + CompactRewriter rewriter, const std::shared_ptr& reporter, + const std::shared_ptr& cancel_flag) : executor_(executor), dv_maintainer_(dv_maintainer), min_file_num_(min_file_num), @@ -38,12 +39,18 @@ BucketedAppendCompactManager::BucketedAppendCompactManager( [](const std::shared_ptr& lhs, const std::shared_ptr& rhs) { return lhs->min_sequence_number > rhs->min_sequence_number; }), + cancel_flag_(cancel_flag ? cancel_flag : std::make_shared(false)), logger_(Logger::GetLogger("BucketedAppendCompactManager")) { for (const auto& file : restored) { to_compact_.push(file); } } +void BucketedAppendCompactManager::CancelCompaction() { + cancel_flag_->store(true, std::memory_order_relaxed); + CompactFutureManager::CancelCompaction(); +} + Status BucketedAppendCompactManager::TriggerCompaction(bool full_compaction) { if (full_compaction) { PAIMON_RETURN_NOT_OK(TriggerFullCompaction()); @@ -71,6 +78,7 @@ Status BucketedAppendCompactManager::TriggerFullCompaction() { compacting.push_back(to_compact_.top()); to_compact_.pop(); } + cancel_flag_->store(false, std::memory_order_relaxed); auto compact_task = std::make_shared(reporter_, dv_maintainer_, compacting, compaction_file_size_, force_rewrite_all_files_, rewriter_); @@ -87,6 +95,7 @@ void BucketedAppendCompactManager::TriggerCompactionWithBestEffort() { } std::optional>> picked = PickCompactBefore(); if (picked) { + cancel_flag_->store(false, std::memory_order_relaxed); compacting_ = picked.value(); auto compact_task = std::make_shared(reporter_, dv_maintainer_, compacting_.value(), rewriter_); diff --git a/src/paimon/core/append/bucketed_append_compact_manager.h b/src/paimon/core/append/bucketed_append_compact_manager.h index 6076a08a..81613933 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager.h +++ b/src/paimon/core/append/bucketed_append_compact_manager.h @@ -16,6 +16,7 @@ #pragma once +#include #include #include #include @@ -72,9 +73,12 @@ class BucketedAppendCompactManager : public CompactFutureManager { int32_t min_file_num, int64_t target_file_size, int64_t compaction_file_size, bool force_rewrite_all_files, CompactRewriter rewriter, - const std::shared_ptr& reporter); + const std::shared_ptr& reporter, + const std::shared_ptr& cancel_flag); ~BucketedAppendCompactManager() override = default; + void CancelCompaction() override; + Status TriggerCompaction(bool full_compaction) override; bool ShouldWaitForLatestCompaction() const override { @@ -195,6 +199,7 @@ class BucketedAppendCompactManager : public CompactFutureManager { std::shared_ptr reporter_; std::optional>> compacting_; DataFileMetaPriorityQueue to_compact_; + std::shared_ptr cancel_flag_; 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 caa02f5a..bec69648 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager_test.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager_test.cpp @@ -16,6 +16,7 @@ #include "paimon/core/append/bucketed_append_compact_manager.h" +#include #include #include #include @@ -70,7 +71,8 @@ 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, /*reporter=*/nullptr); + /*force_rewrite_all_files=*/false, /*rewriter=*/nullptr, /*reporter=*/nullptr, + /*cancel_flag=*/std::make_shared(false)); auto actual = manager.PickCompactBefore(); if (expected_present) { ASSERT_TRUE(actual.has_value()); diff --git a/src/paimon/core/compact/compact_future_manager.h b/src/paimon/core/compact/compact_future_manager.h index 092a84ac..7b966786 100644 --- a/src/paimon/core/compact/compact_future_manager.h +++ b/src/paimon/core/compact/compact_future_manager.h @@ -62,9 +62,15 @@ class CompactFutureManager : public CompactManager { 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)); + Result> result = + ObtainCompactResult(std::move(task_future_)); + if (result.status.OK()) { + return std::make_optional(std::move(result).value()); + } else if (result.status().IsCancelled()) { + return std::optional>(); + } else { + return result; + } } return std::optional>(); } 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 6ffbd500..184bf586 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -16,6 +16,7 @@ #include "paimon/core/operation/append_only_file_store_write.h" +#include #include #include "paimon/common/data/binary_row.h" @@ -108,7 +109,8 @@ Result> AppendOnlyFileStoreWrite::CreateFileStore Result>> AppendOnlyFileStoreWrite::CompactRewrite( const BinaryRow& partition, int32_t bucket, DeletionVector::Factory dv_factory, - const std::vector>& to_compact) { + const std::vector>& to_compact, + const std::shared_ptr& cancel_flag) { if (to_compact.empty()) { return std::vector>{}; } @@ -133,6 +135,9 @@ Result>> AppendOnlyFileStoreWrite::Com }); while (true) { + if (cancel_flag->load(std::memory_order_relaxed)) { + return Status::Cancelled("Compaction cancelled while rewriting files."); + } PAIMON_ASSIGN_OR_RAISE(BatchReader::ReadBatch batch, reader->NextBatch()); if (BatchReader::IsEofBatch(batch)) { break; @@ -181,11 +186,12 @@ Result> AppendOnlyFileStoreWrite::CreateWriter( } return std::shared_ptr(); }; + auto cancel_flag = std::make_shared(false); - auto rewriter = [this, partition, bucket, - dv_factory](const std::vector>& to_compact) + auto rewriter = [this, partition, bucket, dv_factory, + cancel_flag](const std::vector>& to_compact) -> Result>> { - return CompactRewrite(partition, bucket, dv_factory, to_compact); + return CompactRewrite(partition, bucket, dv_factory, to_compact, cancel_flag); }; compact_manager = std::make_shared( @@ -194,7 +200,7 @@ Result> AppendOnlyFileStoreWrite::CreateWriter( options_.GetTargetFileSize(/*has_primary_key=*/false), options_.GetCompactionFileSize(/*has_primary_key=*/false), options_.CompactionForceRewriteAllFiles(), rewriter, - compaction_metrics_->CreateReporter(partition, bucket)); + compaction_metrics_->CreateReporter(partition, bucket), cancel_flag); } auto writer = std::make_shared( diff --git a/src/paimon/core/operation/append_only_file_store_write.h b/src/paimon/core/operation/append_only_file_store_write.h index aabcd2e1..e112db62 100644 --- a/src/paimon/core/operation/append_only_file_store_write.h +++ b/src/paimon/core/operation/append_only_file_store_write.h @@ -16,6 +16,7 @@ #pragma once +#include #include #include #include @@ -92,7 +93,8 @@ class AppendOnlyFileStoreWrite : public AbstractFileStoreWrite { Result>> CompactRewrite( const BinaryRow& partition, int32_t bucket, DeletionVector::Factory dv_factory, - const std::vector>& to_compact); + const std::vector>& to_compact, + const std::shared_ptr& cancel_flag); SingleFileWriterCreator GetDataFileWriterCreator( const BinaryRow& partition, int32_t bucket, const std::shared_ptr& schema, From 393821c704bf1fbd0236a9dae10cccac7171f4f4 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 16 Mar 2026 10:25:06 +0800 Subject: [PATCH 13/27] fix --- src/paimon/core/compact/compact_future_manager.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/paimon/core/compact/compact_future_manager.h b/src/paimon/core/compact/compact_future_manager.h index 7b966786..1d58f82b 100644 --- a/src/paimon/core/compact/compact_future_manager.h +++ b/src/paimon/core/compact/compact_future_manager.h @@ -64,12 +64,12 @@ class CompactFutureManager : public CompactManager { if (ready) { Result> result = ObtainCompactResult(std::move(task_future_)); - if (result.status.OK()) { + if (result.status().ok()) { return std::make_optional(std::move(result).value()); } else if (result.status().IsCancelled()) { return std::optional>(); } else { - return result; + return result.status(); } } return std::optional>(); From 70362d82000b6766dd6c7dede78af4e20efd944d Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 16 Mar 2026 10:48:43 +0800 Subject: [PATCH 14/27] fix --- .../bucketed_append_compact_manager_test.cpp | 52 +++++++++++++++++++ .../core/operation/file_store_write.cpp | 2 +- 2 files changed, 53 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 bec69648..9784c122 100644 --- a/src/paimon/core/append/bucketed_append_compact_manager_test.cpp +++ b/src/paimon/core/append/bucketed_append_compact_manager_test.cpp @@ -17,8 +17,11 @@ #include "paimon/core/append/bucketed_append_compact_manager.h" #include +#include +#include #include #include +#include #include #include @@ -28,6 +31,7 @@ #include "paimon/core/stats/simple_stats.h" #include "paimon/executor.h" #include "paimon/result.h" +#include "paimon/testing/utils/testharness.h" namespace paimon::test { @@ -262,4 +266,52 @@ TEST_F(BucketedAppendCompactManagerTest, TestPick) { NewFile(2601, 2610), NewFile(2611, 2620), NewFile(2621, 2630)}); } +TEST_F(BucketedAppendCompactManagerTest, TestCancelCompactionPropagatesToRewriteLoop) { + auto cancel_flag = std::make_shared(false); + auto exit_signal = std::make_shared>(); + auto exit_future = exit_signal->get_future(); + + auto rewriter = [cancel_flag, + exit_signal](const std::vector>& to_compact) + -> Result>> { + while (!cancel_flag->load(std::memory_order_relaxed)) { + std::this_thread::sleep_for(std::chrono::milliseconds(1)); + } + exit_signal->set_value(); + return Status::Invalid("compaction cancelled in rewrite loop"); + }; + + BucketedAppendCompactManager manager( + executor_, {NewFile(1, 100), NewFile(101, 200), NewFile(201, 300), NewFile(301, 400)}, + /*dv_maintainer=*/nullptr, + /*min_file_num=*/4, + /*target_file_size=*/1024, + /*compaction_file_size=*/700, + /*force_rewrite_all_files=*/false, rewriter, + /*reporter=*/nullptr, cancel_flag); + + ASSERT_OK(manager.TriggerCompaction(/*full_compaction=*/true)); + manager.CancelCompaction(); + + EXPECT_EQ(exit_future.wait_for(std::chrono::seconds(1)), std::future_status::ready); +} + +TEST_F(BucketedAppendCompactManagerTest, TestTriggerCompactionResetsCancelFlag) { + auto cancel_flag = std::make_shared(true); + auto rewriter = [](const std::vector>& to_compact) + -> Result>> { return to_compact; }; + + BucketedAppendCompactManager manager( + executor_, {NewFile(1, 100), NewFile(101, 200), NewFile(201, 300), NewFile(301, 400)}, + /*dv_maintainer=*/nullptr, + /*min_file_num=*/4, + /*target_file_size=*/1024, + /*compaction_file_size=*/700, + /*force_rewrite_all_files=*/false, rewriter, + /*reporter=*/nullptr, cancel_flag); + + ASSERT_OK(manager.TriggerCompaction(/*full_compaction=*/true)); + EXPECT_FALSE(cancel_flag->load(std::memory_order_relaxed)); +} + } // namespace paimon::test diff --git a/src/paimon/core/operation/file_store_write.cpp b/src/paimon/core/operation/file_store_write.cpp index f5702bd8..5571e625 100644 --- a/src/paimon/core/operation/file_store_write.cpp +++ b/src/paimon/core/operation/file_store_write.cpp @@ -102,7 +102,7 @@ Result> FileStoreWrite::Create(std::unique_ptrIgnorePreviousFiles(); if (schema->PrimaryKeys().empty()) { // append table - bool need_dv_maintainer_factory = true; + bool need_dv_maintainer_factory = options.DeletionVectorsEnabled(); if (options.GetBucket() == -1) { need_dv_maintainer_factory = false; ignore_previous_files = true; From 90b574ca78dda89e07531d7bb2d4ec1a6cf35158 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 16 Mar 2026 14:48:28 +0800 Subject: [PATCH 15/27] fix --- .../append_only_file_store_write.cpp | 4 +- test/inte/compaction_inte_test.cpp | 67 +++++++++++++++++++ 2 files changed, 70 insertions(+), 1 deletion(-) 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 184bf586..15c1395e 100644 --- a/src/paimon/core/operation/append_only_file_store_write.cpp +++ b/src/paimon/core/operation/append_only_file_store_write.cpp @@ -154,8 +154,10 @@ 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); }); + ArrowSchemaRelease(c_schema.get()); + ScopeGuard guard([array = c_array.get()]() { ArrowArrayRelease(array); }); PAIMON_RETURN_NOT_OK(rewriter->Write(c_array.get())); + guard.Release(); } rewriter_guard.Release(); PAIMON_RETURN_NOT_OK(rewriter->Close()); diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index 22ca2200..f892a607 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -23,6 +23,8 @@ #include "gtest/gtest.h" #include "paimon/commit_context.h" #include "paimon/common/data/binary_row.h" +#include "paimon/common/factories/io_hook.h" +#include "paimon/common/utils/scope_guard.h" #include "paimon/core/append/bucketed_append_compact_manager.h" #include "paimon/core/io/data_file_meta.h" #include "paimon/core/operation/append_only_file_store_write.h" @@ -34,6 +36,7 @@ #include "paimon/result.h" #include "paimon/testing/utils/binary_row_generator.h" #include "paimon/testing/utils/data_generator.h" +#include "paimon/testing/utils/io_exception_helper.h" #include "paimon/testing/utils/test_helper.h" #include "paimon/testing/utils/testharness.h" #include "paimon/write_context.h" @@ -389,6 +392,70 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) } } +TEST_F(CompactionInteTest, TestAppendTableCompactionWithIOException) { + 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, "orc"}, + {Options::BUCKET, "2"}, + {Options::BUCKET_KEY, "f2"}, + {Options::FILE_SYSTEM, "local"}, + }; + + bool compaction_run_complete = false; + auto io_hook = IOHook::GetInstance(); + for (size_t i = 0; i < 600; ++i) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + + ASSERT_OK_AND_ASSIGN(auto helper, + TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, + options, /*is_streaming_mode=*/true)); + ASSERT_OK_AND_ASSIGN(std::optional> table_schema, + helper->LatestSchema()); + ASSERT_TRUE(table_schema); + + auto gen = std::make_shared(table_schema.value(), pool_); + int64_t commit_identifier = 0; + PrepareSimpleAppendData(gen, helper.get(), &commit_identifier); + + std::vector data; + data.push_back( + BinaryRowGenerator::GenerateRow({std::string("Lily"), 10, 0, 17.1}, pool_.get())); + ASSERT_OK_AND_ASSIGN(auto batches, gen->SplitArrayByPartitionAndBucket(data)); + ASSERT_EQ(1, batches.size()); + + ScopeGuard guard([&io_hook]() { io_hook->Clear(); }); + io_hook->Reset(i, IOHook::Mode::RETURN_ERROR); + + CHECK_HOOK_STATUS(helper->write_->Write(std::move(batches[0])), i); + CHECK_HOOK_STATUS(helper->write_->Compact(/*partition=*/{{"f1", "10"}}, /*bucket=*/1, + /*full_compaction=*/true), + i); + + Result>> commit_messages = + helper->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier); + CHECK_HOOK_STATUS(commit_messages.status(), i); + CHECK_HOOK_STATUS(helper->commit_->Commit(commit_messages.value(), commit_identifier), i); + + compaction_run_complete = true; + io_hook->Clear(); + + ASSERT_OK_AND_ASSIGN(std::optional latest_snapshot, helper->LatestSnapshot()); + ASSERT_TRUE(latest_snapshot); + ASSERT_EQ(Snapshot::CommitKind::Compact(), latest_snapshot->GetCommitKind()); + break; + } + + ASSERT_TRUE(compaction_run_complete); +} + TEST_F(CompactionInteTest, DISABLED_TestAppendTableWriteAlterTableWithCompaction) { std::string test_data_path = paimon::test::GetDataDir() + From e074ee911aeaf31a95916708d6b0772d6e71ef93 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 16 Mar 2026 15:12:17 +0800 Subject: [PATCH 16/27] fix --- .../deletionvectors/bucketed_dv_maintainer.h | 22 +++++++ .../bucketed_dv_maintainer_test.cpp | 61 ++++++++++++++++++- 2 files changed, 80 insertions(+), 3 deletions(-) diff --git a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h index 3339d472..58a27197 100644 --- a/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h +++ b/src/paimon/core/deletionvectors/bucketed_dv_maintainer.h @@ -21,6 +21,7 @@ #include #include +#include "paimon/core/deletionvectors/bitmap_deletion_vector.h" #include "paimon/core/deletionvectors/deletion_vector.h" #include "paimon/core/deletionvectors/deletion_vectors_index_file.h" #include "paimon/core/index/index_file_handler.h" @@ -38,6 +39,27 @@ class BucketedDvMaintainer { deletion_vectors_(deletion_vectors), bitmap64_(dv_index_file->Bitmap64()) {} + Status NotifyNewDeletion(const std::string& file_name, int64_t position) { + std::shared_ptr dv; + if (auto it = deletion_vectors_.find(file_name); it == deletion_vectors_.end()) { + if (bitmap64_) { + return Status::NotImplemented("not support bitmap 64 deletion vectors"); + } + RoaringBitmap32 roaring_bitmap; + auto inserted = std::make_shared(roaring_bitmap); + deletion_vectors_[file_name] = inserted; + dv = std::move(inserted); + } else { + dv = it->second; + } + + PAIMON_ASSIGN_OR_RAISE(bool updated, dv->CheckedDelete(position)); + if (updated) { + modified_ = true; + } + return Status::OK(); + } + std::optional> DeletionVectorOf( const std::string& file_name) const { if (auto it = deletion_vectors_.find(file_name); it != deletion_vectors_.end()) { diff --git a/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp b/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp index fc9bb204..1daaac72 100644 --- a/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp +++ b/src/paimon/core/deletionvectors/bucketed_dv_maintainer_test.cpp @@ -30,13 +30,13 @@ namespace paimon::test { namespace { -std::shared_ptr CreateDvIndexFile(const std::string& root_path) { +std::shared_ptr CreateDvIndexFile(const std::string& root_path, + bool bitmap64 = false) { 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, - /*bitmap64=*/false, memory_pool); + return std::make_shared(fs, path_factory, bitmap64, memory_pool); } std::shared_ptr CreateDeletionVector(int32_t begin, int32_t end) { @@ -96,4 +96,59 @@ TEST(BucketedDvMaintainerTest, TestWriteDeletionVectorsIndexOnlyWhenModified) { ASSERT_FALSE(write_after_reset.has_value()); } +TEST(BucketedDvMaintainerTest, TestNotifyNewDeletionCreatesVectorAndMarksModified) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + auto index_file = CreateDvIndexFile(dir->Str()); + BucketedDvMaintainer maintainer(index_file, /*deletion_vectors=*/{}); + + ASSERT_OK(maintainer.NotifyNewDeletion("file-new", /*position=*/7)); + + auto created = maintainer.DeletionVectorOf("file-new"); + ASSERT_TRUE(created.has_value()); + ASSERT_OK_AND_ASSIGN(bool deleted, created.value()->IsDeleted(/*position=*/7)); + ASSERT_TRUE(deleted); + + ASSERT_OK_AND_ASSIGN(auto modified_write, maintainer.WriteDeletionVectorsIndex()); + ASSERT_TRUE(modified_write.has_value()); + + ASSERT_OK_AND_ASSIGN(auto write_after_reset, maintainer.WriteDeletionVectorsIndex()); + ASSERT_FALSE(write_after_reset.has_value()); +} + +TEST(BucketedDvMaintainerTest, TestNotifyNewDeletionOnExistingVectorOnlyMarksWhenUpdated) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + auto index_file = CreateDvIndexFile(dir->Str()); + std::map> deletion_vectors = { + {"file-a", CreateDeletionVector(0, 3)}}; // already contains 0,1,2 + BucketedDvMaintainer maintainer(index_file, deletion_vectors); + + // Deleting an existing position should not mark modified. + ASSERT_OK(maintainer.NotifyNewDeletion("file-a", /*position=*/2)); + ASSERT_OK_AND_ASSIGN(auto not_modified_write, maintainer.WriteDeletionVectorsIndex()); + ASSERT_FALSE(not_modified_write.has_value()); + + // Deleting a new position should mark modified. + ASSERT_OK(maintainer.NotifyNewDeletion("file-a", /*position=*/9)); + ASSERT_OK_AND_ASSIGN(auto modified_write, maintainer.WriteDeletionVectorsIndex()); + ASSERT_TRUE(modified_write.has_value()); +} + +TEST(BucketedDvMaintainerTest, TestNotifyNewDeletionReturnsNotImplementedForBitmap64) { + auto dir = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + auto index_file = CreateDvIndexFile(dir->Str(), /*bitmap64=*/true); + BucketedDvMaintainer maintainer(index_file, /*deletion_vectors=*/{}); + + Status status = maintainer.NotifyNewDeletion("file-new", /*position=*/1); + ASSERT_TRUE(status.IsNotImplemented()); + + auto lookup = maintainer.DeletionVectorOf("file-new"); + ASSERT_FALSE(lookup.has_value()); + + ASSERT_OK_AND_ASSIGN(auto write_result, maintainer.WriteDeletionVectorsIndex()); + ASSERT_FALSE(write_result.has_value()); +} + } // namespace paimon::test From a599dcf28842b05e7ba2f5271e096908aca4abe3 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Mon, 16 Mar 2026 18:23:32 +0800 Subject: [PATCH 17/27] fix --- src/paimon/common/data/binary_row.h | 14 + .../core/manifest/index_manifest_file.cpp | 2 +- .../core/manifest/index_manifest_file.h | 2 +- .../manifest/index_manifest_file_handler.cpp | 54 +++- .../manifest/index_manifest_file_handler.h | 10 +- src/paimon/testing/utils/test_helper.h | 9 +- test/inte/compaction_inte_test.cpp | 248 ++++++++++-------- 7 files changed, 219 insertions(+), 120 deletions(-) diff --git a/src/paimon/common/data/binary_row.h b/src/paimon/common/data/binary_row.h index eeb88e0d..8bb8e465 100644 --- a/src/paimon/common/data/binary_row.h +++ b/src/paimon/common/data/binary_row.h @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -163,6 +164,19 @@ struct hash> { } }; +/// for std::unordered_map, ...> +template <> +struct hash> { + size_t operator()( + const std::tuple& partition_bucket_type) const { + const auto& [partition, bucket, index_type] = partition_bucket_type; + size_t hash = paimon::MurmurHashUtils::HashUnsafeBytes( + reinterpret_cast(&bucket), 0, sizeof(bucket), partition.HashCode()); + return paimon::MurmurHashUtils::HashUnsafeBytes(index_type.data(), 0, index_type.size(), + hash); + } +}; + template <> struct hash { size_t operator()(const paimon::BinaryRow& row) const { diff --git a/src/paimon/core/manifest/index_manifest_file.cpp b/src/paimon/core/manifest/index_manifest_file.cpp index 18af1563..84b40688 100644 --- a/src/paimon/core/manifest/index_manifest_file.cpp +++ b/src/paimon/core/manifest/index_manifest_file.cpp @@ -61,7 +61,7 @@ Result> IndexManifestFile::Create( path_factory->CreateIndexManifestFileFactory(); return std::unique_ptr( new IndexManifestFile(file_system, reader_builder, writer_builder, compression, - index_manifest_file_factory, pool)); + index_manifest_file_factory, bucket_mode, pool)); } IndexManifestFile::IndexManifestFile(const std::shared_ptr& file_system, diff --git a/src/paimon/core/manifest/index_manifest_file.h b/src/paimon/core/manifest/index_manifest_file.h index 23143d77..5e7ee651 100644 --- a/src/paimon/core/manifest/index_manifest_file.h +++ b/src/paimon/core/manifest/index_manifest_file.h @@ -47,7 +47,7 @@ class IndexManifestFile : public ObjectsFile { const std::shared_ptr& file_system, const std::shared_ptr& file_format, const std::string& compression, const std::shared_ptr& path_factory, - const std::shared_ptr& pool, const CoreOptions& options); + const std::shared_ptr& pool, int32_t bucket_mode, const CoreOptions& options); /// Write new index files to index manifest. Result> WriteIndexFiles( diff --git a/src/paimon/core/manifest/index_manifest_file_handler.cpp b/src/paimon/core/manifest/index_manifest_file_handler.cpp index 52267f37..6fbfd8cf 100644 --- a/src/paimon/core/manifest/index_manifest_file_handler.cpp +++ b/src/paimon/core/manifest/index_manifest_file_handler.cpp @@ -17,14 +17,59 @@ #include "paimon/core/manifest/index_manifest_file_handler.h" #include +#include #include #include "paimon/core/deletionvectors/deletion_vectors_index_file.h" namespace paimon { + +using BucketIdentifer = std::tuple; + +std::vector IndexManifestFileHandler::BucketedCombiner::Combine( + const std::vector& prev_index_files, + const std::vector& new_index_files) const { + std::unordered_map index_entries; + for (const auto& entry : prev_index_files) { + index_entries.emplace( + BucketIdentifer(entry.partition, entry.bucket, entry.index_file->IndexType()), entry); + } + + std::unordered_map removed; + removed.reserve(new_index_files.size()); + std::unordered_map added; + added.reserve(new_index_files.size()); + + for (const auto& entry : new_index_files) { + if (entry.kind == FileKind::Delete()) { + removed.emplace( + BucketIdentifer(entry.partition, entry.bucket, entry.index_file->IndexType()), + entry); + } else if (entry.kind == FileKind::Add()) { + added.emplace( + BucketIdentifer(entry.partition, entry.bucket, entry.index_file->IndexType()), + entry); + } + } + + // The deleted entry is processed first to avoid overwriting a new entry. + for (const auto& entry : removed) { + index_entries.erase(entry.first); + } + for (const auto& entry : added) { + index_entries.emplace(entry.first, entry.second); + } + + std::vector result_entries; + result_entries.reserve(index_entries.size()); + for (const auto& [_, entry] : index_entries) { + result_entries.push_back(entry); + } + return result_entries; +} + std::vector IndexManifestFileHandler::GlobalFileNameCombiner::Combine( const std::vector& prev_index_files, const std::vector& new_index_files) const { - // TODO(liancheng.lsz): support dv std::map index_entries; for (const auto& entry : prev_index_files) { index_entries.emplace(entry.index_file->FileName(), entry); @@ -121,8 +166,11 @@ IndexManifestFileHandler::GetIndexManifestFileCombine(const std::string& index_t if (index_type != DeletionVectorsIndexFile::DELETION_VECTORS_INDEX && index_type != "HASH") { return std::make_unique(); } - return Status::NotImplemented( - "Do not support handle dv index or hash index in commit process."); + if (index_type == DeletionVectorsIndexFile::DELETION_VECTORS_INDEX && bucket_mode == -1) { + return Status::NotImplemented("not yet support dv with BUCKET_UNAWARE mode"); + } else { + return std::make_unique(); + } } } // namespace paimon diff --git a/src/paimon/core/manifest/index_manifest_file_handler.h b/src/paimon/core/manifest/index_manifest_file_handler.h index 0c764b75..acec920d 100644 --- a/src/paimon/core/manifest/index_manifest_file_handler.h +++ b/src/paimon/core/manifest/index_manifest_file_handler.h @@ -35,7 +35,7 @@ class IndexManifestFileHandler { static Result Write(const std::optional& previous_index_manifest, const std::vector& new_index_entries, - IndexManifestFile* index_manifest_file); + int32_t bucket_mode, IndexManifestFile* index_manifest_file); private: class IndexManifestFileCombiner { @@ -46,6 +46,14 @@ class IndexManifestFileHandler { const std::vector& new_index_files) const = 0; }; + /// Combine previous and new global index files by file `BucketIdentifier`. + class BucketedCombiner : public IndexManifestFileCombiner { + public: + std::vector Combine( + const std::vector& prev_index_files, + const std::vector& new_index_files) const override; + }; + /// Combine previous and new global index files by file name. class GlobalFileNameCombiner : public IndexManifestFileCombiner { public: diff --git a/src/paimon/testing/utils/test_helper.h b/src/paimon/testing/utils/test_helper.h index 70582b5d..186be7ff 100644 --- a/src/paimon/testing/utils/test_helper.h +++ b/src/paimon/testing/utils/test_helper.h @@ -56,18 +56,19 @@ class TestHelper { const std::string& root_path, const std::shared_ptr& schema, const std::vector& partition_keys, const std::vector& primary_keys, - const std::map& options, bool is_streaming_mode) { + const std::map& options, bool is_streaming_mode, + bool ignore_if_exists = false) { // only for test && only check the key auto new_options = options; new_options["enable-object-store-catalog-in-inte-test"] = ""; PAIMON_ASSIGN_OR_RAISE(auto catalog, Catalog::Create(root_path, new_options)); - PAIMON_RETURN_NOT_OK( - catalog->CreateDatabase("foo", new_options, /*ignore_if_exists=*/false)); + PAIMON_RETURN_NOT_OK(catalog->CreateDatabase("foo", new_options, ignore_if_exists)); ::ArrowSchema c_schema; + ScopeGuard guard([schema = &c_schema]() { ArrowSchemaRelease(schema); }); PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportSchema(*schema, &c_schema)); PAIMON_RETURN_NOT_OK(catalog->CreateTable(Identifier("foo", "bar"), &c_schema, partition_keys, primary_keys, new_options, - /*ignore_if_exists=*/false)); + ignore_if_exists)); std::string table_path = PathUtil::JoinPath(root_path, "foo.db/bar"); return Create(table_path, new_options, is_streaming_mode); } diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index f892a607..3f575a01 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -28,6 +28,7 @@ #include "paimon/core/append/bucketed_append_compact_manager.h" #include "paimon/core/io/data_file_meta.h" #include "paimon/core/operation/append_only_file_store_write.h" +#include "paimon/core/operation/restore_files.h" #include "paimon/core/table/sink/commit_message_impl.h" #include "paimon/core/table/source/data_split_impl.h" #include "paimon/executor.h" @@ -49,8 +50,8 @@ class CompactionInteTest : public testing::Test, public ::testing::WithParamInte pool_ = GetDefaultPool(); } - void PrepareSimpleAppendData(const std::shared_ptr& gen, TestHelper* helper, - int64_t* identifier) { + void PrepareSimpleAppendData(const std::shared_ptr& gen, bool with_dv, + TestHelper* helper, int64_t* identifier) { auto& commit_identifier = *identifier; std::vector datas_1; datas_1.push_back( @@ -107,6 +108,29 @@ class CompactionInteTest : public testing::Test, public ::testing::WithParamInte ASSERT_EQ(3, snapshot3.value().Id()); ASSERT_EQ(10, snapshot3.value().TotalRecordCount().value()); ASSERT_EQ(1, snapshot3.value().DeltaRecordCount().value()); + + if (with_dv) { + auto partition = BinaryRowGenerator::GenerateRow({10}, pool_.get()); + int32_t bucket = 1; + auto abstract_write = dynamic_cast(helper->write_.get()); + ASSERT_OK_AND_ASSIGN(auto restore_files, + abstract_write->ScanExistingFileMetas(partition, bucket)); + ASSERT_OK_AND_ASSIGN( + auto dv_maintainer, + abstract_write->dv_maintainer_factory_->Create( + partition, bucket, std::vector>{})); + for (const auto& data_file : restore_files->DataFiles()) { + ASSERT_OK(dv_maintainer->NotifyNewDeletion(data_file->file_name, 0)); + } + ASSERT_OK_AND_ASSIGN(auto index_file_meta, dv_maintainer->WriteDeletionVectorsIndex()); + + auto commit_message = std::make_shared( + partition, bucket, 2, DataIncrement({}, {}, {}, {index_file_meta.value()}, {}), + CompactIncrement({}, {}, {})); + std::vector> commit_messages; + commit_messages.push_back(commit_message); + ASSERT_OK(helper->commit_->Commit(commit_messages, commit_identifier++)); + } } private: @@ -156,7 +180,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompaction) { ASSERT_TRUE(table_schema); auto gen = std::make_shared(table_schema.value(), pool_); int64_t commit_identifier = 0; - PrepareSimpleAppendData(gen, helper.get(), &commit_identifier); + PrepareSimpleAppendData(gen, /*with_dv=*/false, helper.get(), &commit_identifier); std::vector datas_4; datas_4.push_back( BinaryRowGenerator::GenerateRow({std::string("Lily"), 10, 0, 17.1}, pool_.get())); @@ -216,6 +240,91 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompaction) { } } +TEST_P(CompactionInteTest, TestAppendTableStreamWriteFullCompactionWithDv) { + 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::DELETION_VECTORS_ENABLED, "true"}}; + 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, /*with_dv=*/true, 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_AND_ASSIGN( + auto helper2, TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, + /*is_streaming_mode=*/true, /*ignore_if_exists=*/true)); + + ASSERT_OK(helper2->write_->Write(std::move(batches_4[0]))); + ASSERT_OK(helper2->write_->Compact(/*partition=*/{{"f1", "10"}}, /*bucket=*/1, + /*full_compaction=*/true)); + ASSERT_OK_AND_ASSIGN( + std::vector> commit_messages, + helper2->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier)); + ASSERT_OK(helper2->commit_->Commit(commit_messages, commit_identifier)); + ASSERT_OK_AND_ASSIGN(std::optional snapshot5, helper2->LatestSnapshot()); + ASSERT_EQ(6, snapshot5.value().Id()); + ASSERT_EQ(8, snapshot5.value().TotalRecordCount().value()); + ASSERT_EQ(-3, snapshot5.value().DeltaRecordCount().value()); + ASSERT_EQ(Snapshot::CommitKind::Compact(), snapshot5.value().GetCommitKind()); + ASSERT_OK_AND_ASSIGN(std::vector> data_splits, + helper2->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, "Emily", 10, 0, 13.1], +[0, "Tony", 10, 0, 14.1], +[0, "Bob", 10, 0, 12.1], +[0, "Alex", 10, 0, 16.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, + helper2->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, + helper2->ReadAndCheckResult(data_type, {split}, iter->second)); + ASSERT_TRUE(success); + } +} + TEST_P(CompactionInteTest, TestAppendTableStreamWriteBestEffortCompaction) { auto dir = UniqueTestDirectory::Create(); ASSERT_TRUE(dir); @@ -227,13 +336,12 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteBestEffortCompaction) { 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"}, - }; + 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"}, + {Options::DELETION_VECTORS_ENABLED, "true"}}; ASSERT_OK_AND_ASSIGN( auto helper, TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, /*is_streaming_mode=*/true)); @@ -242,7 +350,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteBestEffortCompaction) { ASSERT_TRUE(table_schema); auto gen = std::make_shared(table_schema.value(), pool_); int64_t commit_identifier = 0; - PrepareSimpleAppendData(gen, helper.get(), &commit_identifier); + PrepareSimpleAppendData(gen, /*with_dv=*/false, helper.get(), &commit_identifier); std::vector datas_4; datas_4.push_back( BinaryRowGenerator::GenerateRow({std::string("Lily"), 10, 0, 17.1}, pool_.get())); @@ -322,6 +430,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) {Options::BUCKET, "2"}, {Options::BUCKET_KEY, "f2"}, {Options::FILE_SYSTEM, "local"}, + {Options::DELETION_VECTORS_ENABLED, "true"}, {Options::DATA_FILE_EXTERNAL_PATHS, external_test_dir}, {Options::DATA_FILE_EXTERNAL_PATHS_STRATEGY, "round-robin"}}; ASSERT_OK_AND_ASSIGN( @@ -332,7 +441,7 @@ TEST_P(CompactionInteTest, TestAppendTableStreamWriteCompactionWithExternalPath) ASSERT_TRUE(table_schema); auto gen = std::make_shared(table_schema.value(), pool_); int64_t commit_identifier = 0; - PrepareSimpleAppendData(gen, helper.get(), &commit_identifier); + PrepareSimpleAppendData(gen, /*with_dv=*/false, helper.get(), &commit_identifier); std::vector datas_4; datas_4.push_back( BinaryRowGenerator::GenerateRow({std::string("Lily"), 10, 0, 17.1}, pool_.get())); @@ -400,13 +509,11 @@ TEST_F(CompactionInteTest, TestAppendTableCompactionWithIOException) { std::vector primary_keys = {}; std::vector partition_keys = {"f1"}; - // auto file_format = GetParam(); - std::map options = { - {Options::FILE_FORMAT, "orc"}, - {Options::BUCKET, "2"}, - {Options::BUCKET_KEY, "f2"}, - {Options::FILE_SYSTEM, "local"}, - }; + std::map options = {{Options::FILE_FORMAT, "parquet"}, + {Options::BUCKET, "2"}, + {Options::BUCKET_KEY, "f2"}, + {Options::FILE_SYSTEM, "local"}, + {Options::DELETION_VECTORS_ENABLED, "true"}}; bool compaction_run_complete = false; auto io_hook = IOHook::GetInstance(); @@ -423,7 +530,7 @@ TEST_F(CompactionInteTest, TestAppendTableCompactionWithIOException) { auto gen = std::make_shared(table_schema.value(), pool_); int64_t commit_identifier = 0; - PrepareSimpleAppendData(gen, helper.get(), &commit_identifier); + PrepareSimpleAppendData(gen, /*with_dv=*/true, helper.get(), &commit_identifier); std::vector data; data.push_back( @@ -431,23 +538,28 @@ TEST_F(CompactionInteTest, TestAppendTableCompactionWithIOException) { ASSERT_OK_AND_ASSIGN(auto batches, gen->SplitArrayByPartitionAndBucket(data)); ASSERT_EQ(1, batches.size()); + ASSERT_OK_AND_ASSIGN( + auto helper2, + TestHelper::Create(dir->Str(), schema, partition_keys, primary_keys, options, + /*is_streaming_mode=*/true, /*ignore_if_exists=*/true)); + ScopeGuard guard([&io_hook]() { io_hook->Clear(); }); io_hook->Reset(i, IOHook::Mode::RETURN_ERROR); - CHECK_HOOK_STATUS(helper->write_->Write(std::move(batches[0])), i); - CHECK_HOOK_STATUS(helper->write_->Compact(/*partition=*/{{"f1", "10"}}, /*bucket=*/1, - /*full_compaction=*/true), + CHECK_HOOK_STATUS(helper2->write_->Write(std::move(batches[0])), i); + CHECK_HOOK_STATUS(helper2->write_->Compact(/*partition=*/{{"f1", "10"}}, /*bucket=*/1, + /*full_compaction=*/true), i); Result>> commit_messages = - helper->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier); + helper2->write_->PrepareCommit(/*wait_compaction=*/true, commit_identifier); CHECK_HOOK_STATUS(commit_messages.status(), i); - CHECK_HOOK_STATUS(helper->commit_->Commit(commit_messages.value(), commit_identifier), i); + CHECK_HOOK_STATUS(helper2->commit_->Commit(commit_messages.value(), commit_identifier), i); compaction_run_complete = true; io_hook->Clear(); - ASSERT_OK_AND_ASSIGN(std::optional latest_snapshot, helper->LatestSnapshot()); + ASSERT_OK_AND_ASSIGN(std::optional latest_snapshot, helper2->LatestSnapshot()); ASSERT_TRUE(latest_snapshot); ASSERT_EQ(Snapshot::CommitKind::Compact(), latest_snapshot->GetCommitKind()); break; @@ -456,88 +568,4 @@ TEST_F(CompactionInteTest, TestAppendTableCompactionWithIOException) { ASSERT_TRUE(compaction_run_complete); } -TEST_F(CompactionInteTest, DISABLED_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(); - 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 782b1c9c11b3378b273c43171e393bbac038ed39 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 17 Mar 2026 09:23:21 +0800 Subject: [PATCH 18/27] fix --- .../core/global_index/global_index_scan_impl.cpp | 9 +++++---- src/paimon/core/index/index_file_handler_test.cpp | 8 ++++---- src/paimon/core/manifest/index_manifest_file.cpp | 12 +++++++----- src/paimon/core/manifest/index_manifest_file.h | 8 +++++--- .../core/manifest/index_manifest_file_handler.cpp | 10 +++++----- .../core/manifest/index_manifest_file_handler.h | 2 +- src/paimon/core/mergetree/lookup_levels.cpp | 13 ++++++++++++- src/paimon/core/operation/file_store_commit.cpp | 2 +- src/paimon/core/operation/file_store_write.cpp | 2 +- src/paimon/core/table/source/table_scan.cpp | 8 ++++---- 10 files changed, 45 insertions(+), 29 deletions(-) diff --git a/src/paimon/core/global_index/global_index_scan_impl.cpp b/src/paimon/core/global_index/global_index_scan_impl.cpp index 6b59a209..0bf5733e 100644 --- a/src/paimon/core/global_index/global_index_scan_impl.cpp +++ b/src/paimon/core/global_index/global_index_scan_impl.cpp @@ -114,10 +114,11 @@ Status GlobalIndexScanImpl::Scan() { options_.DataFilePrefix(), options_.LegacyPartitionNameEnabled(), external_paths, global_index_external_path, options_.IndexFileInDataFileDir(), pool_)); - PAIMON_ASSIGN_OR_RAISE(std::unique_ptr index_manifest_file, - IndexManifestFile::Create( - options_.GetFileSystem(), options_.GetManifestFormat(), - options_.GetManifestCompression(), path_factory_, pool_, options_)); + PAIMON_ASSIGN_OR_RAISE( + std::unique_ptr index_manifest_file, + IndexManifestFile::Create(options_.GetFileSystem(), options_.GetManifestFormat(), + options_.GetManifestCompression(), path_factory_, + options_.GetBucket(), pool_, options_)); auto index_file_handler = std::make_unique(options_.GetFileSystem(), std::move(index_manifest_file), std::make_shared(path_factory_), diff --git a/src/paimon/core/index/index_file_handler_test.cpp b/src/paimon/core/index/index_file_handler_test.cpp index e89d19c5..35783f1b 100644 --- a/src/paimon/core/index/index_file_handler_test.cpp +++ b/src/paimon/core/index/index_file_handler_test.cpp @@ -68,10 +68,10 @@ class IndexFileHandlerTest : public testing::Test { /*index_file_in_data_file_dir=*/core_options.IndexFileInDataFileDir(), memory_pool_)); PAIMON_ASSIGN_OR_RAISE(std::unique_ptr index_manifest_file, - IndexManifestFile::Create(core_options.GetFileSystem(), - core_options.GetManifestFormat(), - core_options.GetManifestCompression(), - path_factory, memory_pool_, core_options)); + IndexManifestFile::Create( + core_options.GetFileSystem(), core_options.GetManifestFormat(), + core_options.GetManifestCompression(), path_factory, + core_options.GetBucket(), memory_pool_, core_options)); auto path_factories = std::make_shared(path_factory); return std::make_unique( core_options.GetFileSystem(), std::move(index_manifest_file), path_factories, diff --git a/src/paimon/core/manifest/index_manifest_file.cpp b/src/paimon/core/manifest/index_manifest_file.cpp index 84b40688..4fadbf81 100644 --- a/src/paimon/core/manifest/index_manifest_file.cpp +++ b/src/paimon/core/manifest/index_manifest_file.cpp @@ -44,7 +44,7 @@ class MemoryPool; Result> IndexManifestFile::Create( const std::shared_ptr& file_system, const std::shared_ptr& file_format, const std::string& compression, const std::shared_ptr& path_factory, - const std::shared_ptr& pool, const CoreOptions& options) { + int32_t bucket_mode, const std::shared_ptr& pool, const CoreOptions& options) { std::shared_ptr data_type = VersionedObjectSerializer::VersionType(IndexManifestEntry::DataType()); @@ -69,10 +69,11 @@ IndexManifestFile::IndexManifestFile(const std::shared_ptr& file_sys const std::shared_ptr& writer_builder, const std::string& compression, const std::shared_ptr& path_factory, - const std::shared_ptr& pool) + int32_t bucket_mode, const std::shared_ptr& pool) : ObjectsFile(file_system, reader_builder, writer_builder, std::make_unique(pool), - compression, path_factory, pool) {} + compression, path_factory, pool), + bucket_mode_(bucket_mode) {} Result> IndexManifestFile::WriteIndexFiles( const std::optional& previous_index_manifest, @@ -80,8 +81,9 @@ Result> IndexManifestFile::WriteIndexFiles( if (new_index_files.empty()) { return previous_index_manifest; } - PAIMON_ASSIGN_OR_RAISE(std::string file, IndexManifestFileHandler::Write( - previous_index_manifest, new_index_files, this)); + PAIMON_ASSIGN_OR_RAISE(std::string file, + IndexManifestFileHandler::Write(previous_index_manifest, new_index_files, + bucket_mode_, this)); return std::optional(file); } diff --git a/src/paimon/core/manifest/index_manifest_file.h b/src/paimon/core/manifest/index_manifest_file.h index 5e7ee651..31473eff 100644 --- a/src/paimon/core/manifest/index_manifest_file.h +++ b/src/paimon/core/manifest/index_manifest_file.h @@ -46,8 +46,8 @@ class IndexManifestFile : public ObjectsFile { static Result> Create( const std::shared_ptr& file_system, const std::shared_ptr& file_format, const std::string& compression, - const std::shared_ptr& path_factory, - const std::shared_ptr& pool, int32_t bucket_mode, const CoreOptions& options); + const std::shared_ptr& path_factory, int32_t bucket_mode, + const std::shared_ptr& pool, const CoreOptions& options); /// Write new index files to index manifest. Result> WriteIndexFiles( @@ -59,7 +59,9 @@ class IndexManifestFile : public ObjectsFile { const std::shared_ptr& reader_builder, const std::shared_ptr& writer_builder, const std::string& compression, - const std::shared_ptr& path_factory, + const std::shared_ptr& path_factory, int32_t bucket_mode, const std::shared_ptr& pool); + + const int32_t bucket_mode_; }; } // namespace paimon diff --git a/src/paimon/core/manifest/index_manifest_file_handler.cpp b/src/paimon/core/manifest/index_manifest_file_handler.cpp index 6fbfd8cf..d6317689 100644 --- a/src/paimon/core/manifest/index_manifest_file_handler.cpp +++ b/src/paimon/core/manifest/index_manifest_file_handler.cpp @@ -106,7 +106,7 @@ std::vector IndexManifestFileHandler::GlobalFileNameCombiner Result IndexManifestFileHandler::Write( const std::optional& previous_index_manifest, - const std::vector& new_index_entries, + const std::vector& new_index_entries, int32_t bucket_mode, IndexManifestFile* index_manifest_file) { std::vector entries; if (previous_index_manifest != std::nullopt) { @@ -135,7 +135,7 @@ Result IndexManifestFileHandler::Write( for (const auto& index_type : index_types) { PAIMON_ASSIGN_OR_RAISE( std::unique_ptr combiner, - GetIndexManifestFileCombine(index_type)); + GetIndexManifestFileCombine(index_type, bucket_mode)); std::vector typed_previous_entries = previous[index_type]; std::vector typed_current_entries = current[index_type]; std::vector combined_entries = @@ -162,15 +162,15 @@ IndexManifestFileHandler::SeparateIndexEntries( } Result> -IndexManifestFileHandler::GetIndexManifestFileCombine(const std::string& index_type) { +IndexManifestFileHandler::GetIndexManifestFileCombine(const std::string& index_type, + int32_t bucket_mode) { if (index_type != DeletionVectorsIndexFile::DELETION_VECTORS_INDEX && index_type != "HASH") { return std::make_unique(); } if (index_type == DeletionVectorsIndexFile::DELETION_VECTORS_INDEX && bucket_mode == -1) { return Status::NotImplemented("not yet support dv with BUCKET_UNAWARE mode"); - } else { - return std::make_unique(); } + return std::make_unique(); } } // namespace paimon diff --git a/src/paimon/core/manifest/index_manifest_file_handler.h b/src/paimon/core/manifest/index_manifest_file_handler.h index acec920d..46828c8d 100644 --- a/src/paimon/core/manifest/index_manifest_file_handler.h +++ b/src/paimon/core/manifest/index_manifest_file_handler.h @@ -66,6 +66,6 @@ class IndexManifestFileHandler { const std::vector& index_entries); static Result> GetIndexManifestFileCombine( - const std::string& index_type); + const std::string& index_type, int32_t bucket_mode); }; } // namespace paimon diff --git a/src/paimon/core/mergetree/lookup_levels.cpp b/src/paimon/core/mergetree/lookup_levels.cpp index 5c12cce7..4ab3001c 100644 --- a/src/paimon/core/mergetree/lookup_levels.cpp +++ b/src/paimon/core/mergetree/lookup_levels.cpp @@ -187,10 +187,21 @@ template Status LookupLevels::CreateSstFileFromDataFile(const std::shared_ptr& file, const std::string& kv_file_path) { // Prepare reader to iterate KeyValue + auto dv_factory = + [this](const std::string& file_name) -> Result> { + auto iter = deletion_file_map_.find(file_name); + if (iter != deletion_file_map_.end()) { + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr dv, + DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); + return dv; + } + return std::shared_ptr(); + }; PAIMON_ASSIGN_OR_RAISE( std::vector> raw_readers, split_read_->CreateRawFileReaders(partition_, {file}, read_schema_, - /*predicate=*/nullptr, deletion_file_map_, + /*predicate=*/nullptr, dv_factory, /*row_ranges=*/std::nullopt, data_file_path_factory_)); if (raw_readers.size() != 1) { return Status::Invalid("Unexpected, CreateSstFileFromDataFile only create single reader"); diff --git a/src/paimon/core/operation/file_store_commit.cpp b/src/paimon/core/operation/file_store_commit.cpp index dbf6a7ce..5630776c 100644 --- a/src/paimon/core/operation/file_store_commit.cpp +++ b/src/paimon/core/operation/file_store_commit.cpp @@ -121,7 +121,7 @@ Result> FileStoreCommit::Create( std::shared_ptr index_manifest_file, IndexManifestFile::Create(options.GetFileSystem(), options.GetManifestFormat(), options.GetManifestCompression(), path_factory, - ctx->GetMemoryPool(), options)); + options.GetBucket(), ctx->GetMemoryPool(), options)); auto expire_snapshots = std::make_shared( snapshot_manager, path_factory, manifest_list, manifest_file, options.GetFileSystem(), diff --git a/src/paimon/core/operation/file_store_write.cpp b/src/paimon/core/operation/file_store_write.cpp index 5571e625..86c7a497 100644 --- a/src/paimon/core/operation/file_store_write.cpp +++ b/src/paimon/core/operation/file_store_write.cpp @@ -133,7 +133,7 @@ Result> FileStoreWrite::Create(std::unique_ptr index_manifest_file, IndexManifestFile::Create(options.GetFileSystem(), options.GetManifestFormat(), options.GetManifestCompression(), file_store_path_factory, - ctx->GetMemoryPool(), options)); + options.GetBucket(), ctx->GetMemoryPool(), options)); auto index_file_handler = std::make_shared( options.GetFileSystem(), std::move(index_manifest_file), std::make_shared(file_store_path_factory), diff --git a/src/paimon/core/table/source/table_scan.cpp b/src/paimon/core/table/source/table_scan.cpp index dae778bb..de870395 100644 --- a/src/paimon/core/table/source/table_scan.cpp +++ b/src/paimon/core/table/source/table_scan.cpp @@ -140,10 +140,10 @@ class TableScanImpl { const CoreOptions& core_options, const std::shared_ptr& path_factory, const std::shared_ptr& memory_pool) { PAIMON_ASSIGN_OR_RAISE(std::unique_ptr index_manifest_file, - IndexManifestFile::Create(core_options.GetFileSystem(), - core_options.GetManifestFormat(), - core_options.GetManifestCompression(), - path_factory, memory_pool, core_options)); + IndexManifestFile::Create( + core_options.GetFileSystem(), core_options.GetManifestFormat(), + core_options.GetManifestCompression(), path_factory, + core_options.GetBucket(), memory_pool, core_options)); return std::make_unique( core_options.GetFileSystem(), std::move(index_manifest_file), std::make_shared(path_factory), From 167d93907cc9c04a70554735d623ca711bc9ec84 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 17 Mar 2026 10:22:54 +0800 Subject: [PATCH 19/27] fix --- src/paimon/CMakeLists.txt | 1 + src/paimon/common/utils/status.cpp | 2 +- .../index_manifest_file_handler_test.cpp | 171 ++++++++++++++++++ 3 files changed, 173 insertions(+), 1 deletion(-) create mode 100644 src/paimon/core/manifest/index_manifest_file_handler_test.cpp diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index 0c94e8cf..55eb91e7 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -543,6 +543,7 @@ if(PAIMON_BUILD_TESTS) core/manifest/partition_entry_test.cpp core/manifest/file_entry_test.cpp core/manifest/index_manifest_entry_serializer_test.cpp + core/manifest/index_manifest_file_handler_test.cpp core/mergetree/levels_test.cpp core/mergetree/lookup_file_test.cpp core/mergetree/lookup_levels_test.cpp diff --git a/src/paimon/common/utils/status.cpp b/src/paimon/common/utils/status.cpp index 183f930c..40ff0645 100644 --- a/src/paimon/common/utils/status.cpp +++ b/src/paimon/common/utils/status.cpp @@ -132,7 +132,7 @@ void Status::Abort() const { } void Status::Abort(const std::string& message) const { - std::cerr << "-- Arrow Fatal Error --\n"; + std::cerr << "-- Paimon Fatal Error --\n"; if (!message.empty()) { std::cerr << message << "\n"; } diff --git a/src/paimon/core/manifest/index_manifest_file_handler_test.cpp b/src/paimon/core/manifest/index_manifest_file_handler_test.cpp new file mode 100644 index 00000000..467fd952 --- /dev/null +++ b/src/paimon/core/manifest/index_manifest_file_handler_test.cpp @@ -0,0 +1,171 @@ +/* + * Copyright 2026-present Alibaba Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "paimon/core/manifest/index_manifest_file_handler.h" + +#include +#include +#include +#include + +#include "arrow/api.h" +#include "gtest/gtest.h" +#include "paimon/core/core_options.h" +#include "paimon/core/deletionvectors/deletion_vectors_index_file.h" +#include "paimon/core/index/index_file_meta.h" +#include "paimon/core/manifest/file_kind.h" +#include "paimon/core/manifest/index_manifest_file.h" +#include "paimon/core/utils/file_store_path_factory.h" +#include "paimon/defs.h" +#include "paimon/format/file_format.h" +#include "paimon/format/file_format_factory.h" +#include "paimon/memory/memory_pool.h" +#include "paimon/testing/utils/binary_row_generator.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::test { + +class IndexManifestFileHandlerTest : public testing::Test { + protected: + void SetUp() override { + pool_ = GetDefaultPool(); + dir_ = UniqueTestDirectory::Create(); + ASSERT_TRUE(dir_ != nullptr); + } + + Result> CreateManifestFile(int32_t bucket_mode) const { + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr file_format, + FileFormatFactory::Get("orc", {})); + auto schema = arrow::schema({arrow::field("f0", arrow::int32())}); + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr path_factory, + FileStorePathFactory::Create( + dir_->Str(), schema, /*partition_keys=*/{}, /*default_part_value=*/"", + file_format->Identifier(), /*data_file_prefix=*/"data-", + /*legacy_partition_name_enabled=*/true, /*external_paths=*/{}, + /*global_index_external_path=*/std::nullopt, + /*index_file_in_data_file_dir=*/false, pool_)); + PAIMON_ASSIGN_OR_RAISE(CoreOptions options, + CoreOptions::FromMap({{Options::MANIFEST_FORMAT, "orc"}})); + return IndexManifestFile::Create(dir_->GetFileSystem(), file_format, "zstd", path_factory, + bucket_mode, pool_, options); + } + + static IndexManifestEntry MakeEntry(const FileKind& kind, const BinaryRow& partition, + int32_t bucket, const std::string& index_type, + const std::string& file_name, int64_t row_count) { + return IndexManifestEntry( + kind, partition, bucket, + std::make_shared(index_type, file_name, /*file_size=*/row_count * 10, + row_count, /*dv_ranges=*/std::nullopt, + /*external_path=*/std::nullopt)); + } + + std::shared_ptr pool_; + std::unique_ptr dir_; +}; + +TEST_F(IndexManifestFileHandlerTest, GlobalCombinerDeletesThenAddsByFileName) { + ASSERT_OK_AND_ASSIGN(auto index_manifest_file, CreateManifestFile(/*bucket_mode=*/4)); + + auto partition = BinaryRow::EmptyRow(); + std::vector previous_entries = { + MakeEntry(FileKind::Add(), partition, /*bucket=*/0, /*index_type=*/"BTREE", "global-0", 1)}; + + ASSERT_OK_AND_ASSIGN(std::string previous_manifest, + IndexManifestFileHandler::Write( + /*previous_index_manifest=*/std::nullopt, previous_entries, + /*bucket_mode=*/4, index_manifest_file.get())); + + std::vector new_entries = { + MakeEntry(FileKind::Delete(), partition, /*bucket=*/999, /*index_type=*/"BTREE", "global-0", + 1), + MakeEntry(FileKind::Add(), partition, /*bucket=*/1, /*index_type=*/"BTREE", "global-0", 2)}; + + ASSERT_OK_AND_ASSIGN( + std::string current_manifest, + IndexManifestFileHandler::Write(previous_manifest, new_entries, + /*bucket_mode=*/4, index_manifest_file.get())); + + std::vector written_entries; + ASSERT_OK(index_manifest_file->Read(current_manifest, /*filter=*/nullptr, &written_entries)); + ASSERT_EQ(written_entries.size(), 1); + ASSERT_EQ(written_entries[0].index_file->FileName(), "global-0"); + ASSERT_EQ(written_entries[0].index_file->RowCount(), 2); +} + +TEST_F(IndexManifestFileHandlerTest, BucketedCombinerUsesPartitionBucketAndIndexType) { + ASSERT_OK_AND_ASSIGN(auto index_manifest_file, CreateManifestFile(/*bucket_mode=*/2)); + + auto partition_10 = BinaryRowGenerator::GenerateRow({10}, pool_.get()); + std::vector previous_entries = { + MakeEntry(FileKind::Add(), partition_10, /*bucket=*/0, + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "dv-0-old", 10), + MakeEntry(FileKind::Add(), partition_10, /*bucket=*/1, + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "dv-1-keep", 11)}; + + ASSERT_OK_AND_ASSIGN(std::string previous_manifest, + IndexManifestFileHandler::Write( + /*previous_index_manifest=*/std::nullopt, previous_entries, + /*bucket_mode=*/2, index_manifest_file.get())); + + std::vector new_entries = { + MakeEntry(FileKind::Delete(), partition_10, /*bucket=*/0, + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "ignored", 10), + MakeEntry(FileKind::Add(), partition_10, /*bucket=*/0, + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "dv-0-new", 12)}; + + ASSERT_OK_AND_ASSIGN( + std::string current_manifest, + IndexManifestFileHandler::Write(previous_manifest, new_entries, + /*bucket_mode=*/2, index_manifest_file.get())); + + std::vector written_entries; + ASSERT_OK(index_manifest_file->Read(current_manifest, /*filter=*/nullptr, &written_entries)); + ASSERT_EQ(written_entries.size(), 2); + + bool found_bucket0 = false; + bool found_bucket1 = false; + for (const auto& entry : written_entries) { + if (entry.bucket == 0) { + found_bucket0 = true; + ASSERT_EQ(entry.index_file->FileName(), "dv-0-new"); + ASSERT_EQ(entry.index_file->RowCount(), 12); + } else if (entry.bucket == 1) { + found_bucket1 = true; + ASSERT_EQ(entry.index_file->FileName(), "dv-1-keep"); + ASSERT_EQ(entry.index_file->RowCount(), 11); + } + } + ASSERT_TRUE(found_bucket0); + ASSERT_TRUE(found_bucket1); +} + +TEST_F(IndexManifestFileHandlerTest, DvWithBucketUnawareModeReturnsNotImplemented) { + ASSERT_OK_AND_ASSIGN(auto index_manifest_file, CreateManifestFile(/*bucket_mode=*/-1)); + + auto partition = BinaryRow::EmptyRow(); + std::vector new_entries = { + MakeEntry(FileKind::Add(), partition, /*bucket=*/0, + DeletionVectorsIndexFile::DELETION_VECTORS_INDEX, "dv-0", 1)}; + + ASSERT_NOK_WITH_MSG(IndexManifestFileHandler::Write( + /*previous_index_manifest=*/std::nullopt, new_entries, + /*bucket_mode=*/-1, index_manifest_file.get()), + "not yet support dv with BUCKET_UNAWARE mode"); +} + +} // namespace paimon::test From 1a76f81a21255a6252289009bd8ad386d708568e Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 17 Mar 2026 10:33:58 +0800 Subject: [PATCH 20/27] fix --- .../core/manifest/index_manifest_file_handler.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/paimon/core/manifest/index_manifest_file_handler.cpp b/src/paimon/core/manifest/index_manifest_file_handler.cpp index d6317689..fd5808dd 100644 --- a/src/paimon/core/manifest/index_manifest_file_handler.cpp +++ b/src/paimon/core/manifest/index_manifest_file_handler.cpp @@ -23,30 +23,30 @@ #include "paimon/core/deletionvectors/deletion_vectors_index_file.h" namespace paimon { -using BucketIdentifer = std::tuple; +using BucketIdentifier = std::tuple; std::vector IndexManifestFileHandler::BucketedCombiner::Combine( const std::vector& prev_index_files, const std::vector& new_index_files) const { - std::unordered_map index_entries; + std::unordered_map index_entries; for (const auto& entry : prev_index_files) { index_entries.emplace( - BucketIdentifer(entry.partition, entry.bucket, entry.index_file->IndexType()), entry); + BucketIdentifier(entry.partition, entry.bucket, entry.index_file->IndexType()), entry); } - std::unordered_map removed; + std::unordered_map removed; removed.reserve(new_index_files.size()); - std::unordered_map added; + std::unordered_map added; added.reserve(new_index_files.size()); for (const auto& entry : new_index_files) { if (entry.kind == FileKind::Delete()) { removed.emplace( - BucketIdentifer(entry.partition, entry.bucket, entry.index_file->IndexType()), + BucketIdentifier(entry.partition, entry.bucket, entry.index_file->IndexType()), entry); } else if (entry.kind == FileKind::Add()) { added.emplace( - BucketIdentifer(entry.partition, entry.bucket, entry.index_file->IndexType()), + BucketIdentifier(entry.partition, entry.bucket, entry.index_file->IndexType()), entry); } } From 4adc0c66223250d3b41812d4b2354315b4c998b4 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 17 Mar 2026 11:38:18 +0800 Subject: [PATCH 21/27] fix --- src/paimon/core/operation/abstract_file_store_write.h | 4 ++++ src/paimon/core/operation/file_store_commit_impl.h | 8 ++++++++ src/paimon/testing/utils/test_helper.h | 6 +++--- 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/src/paimon/core/operation/abstract_file_store_write.h b/src/paimon/core/operation/abstract_file_store_write.h index e1726617..8fc0347b 100644 --- a/src/paimon/core/operation/abstract_file_store_write.h +++ b/src/paimon/core/operation/abstract_file_store_write.h @@ -102,6 +102,10 @@ class AbstractFileStoreWrite : public FileStoreWrite { int32_t total_buckets = -1; }; + std::shared_ptr TEST_GetFileStorePathFactory() const { + return file_store_path_factory_; + } + protected: virtual Result> CreateWriter( const BinaryRow& partition, int32_t bucket, diff --git a/src/paimon/core/operation/file_store_commit_impl.h b/src/paimon/core/operation/file_store_commit_impl.h index f54c85ac..768cd2be 100644 --- a/src/paimon/core/operation/file_store_commit_impl.h +++ b/src/paimon/core/operation/file_store_commit_impl.h @@ -202,6 +202,14 @@ class FileStoreCommitImpl : public FileStoreCommit { static int64_t NumChangedBuckets(const std::vector>& changes); + std::shared_ptr TEST_GetSchemaManager() const { + return schema_manager_; + } + + std::shared_ptr TEST_GetSnapshotManager() const { + return snapshot_manager_; + } + private: std::shared_ptr memory_pool_; std::shared_ptr executor_; diff --git a/src/paimon/testing/utils/test_helper.h b/src/paimon/testing/utils/test_helper.h index 186be7ff..a8ae3447 100644 --- a/src/paimon/testing/utils/test_helper.h +++ b/src/paimon/testing/utils/test_helper.h @@ -340,17 +340,17 @@ class TestHelper { Result> LatestSnapshot() const { auto commit_impl = dynamic_cast(commit_.get()); - return commit_impl->snapshot_manager_->LatestSnapshotOfUser(commit_user_); + return commit_impl->TEST_GetSnapshotManager()->LatestSnapshotOfUser(commit_user_); } Result>> LatestSchema() const { auto commit_impl = dynamic_cast(commit_.get()); - return commit_impl->schema_manager_->Latest(); + return commit_impl->TEST_GetSchemaManager()->Latest(); } Result PartitionStr(const BinaryRow& partition) const { auto abstract_write = dynamic_cast(write_.get()); - return abstract_write->file_store_path_factory_->GetPartitionString(partition); + return abstract_write->TEST_GetFileStorePathFactory()->GetPartitionString(partition); } static void CheckCommitMessages(std::vector> expected, From 5c3255d249a2625dbca05eadddd03f07e1c4ac45 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Tue, 17 Mar 2026 13:15:11 +0800 Subject: [PATCH 22/27] fix --- .../core/operation/file_store_commit_impl.h | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/paimon/core/operation/file_store_commit_impl.h b/src/paimon/core/operation/file_store_commit_impl.h index 768cd2be..a7b61eb8 100644 --- a/src/paimon/core/operation/file_store_commit_impl.h +++ b/src/paimon/core/operation/file_store_commit_impl.h @@ -120,6 +120,14 @@ class FileStoreCommitImpl : public FileStoreCommit { Status Init(std::unique_ptr ctx); + std::shared_ptr TEST_GetSchemaManager() const { + return schema_manager_; + } + + std::shared_ptr TEST_GetSnapshotManager() const { + return snapshot_manager_; + } + private: Status Commit(const std::shared_ptr& manifest_committable, bool check_append_files); @@ -202,14 +210,6 @@ class FileStoreCommitImpl : public FileStoreCommit { static int64_t NumChangedBuckets(const std::vector>& changes); - std::shared_ptr TEST_GetSchemaManager() const { - return schema_manager_; - } - - std::shared_ptr TEST_GetSnapshotManager() const { - return snapshot_manager_; - } - private: std::shared_ptr memory_pool_; std::shared_ptr executor_; From 2878528fcace827bdea334bfd5add64126bfe1fd Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 18 Mar 2026 10:22:27 +0800 Subject: [PATCH 23/27] fix --- include/paimon/defs.h | 1 + .../compact/compact_deletion_file_test.cpp | 9 ++---- .../deletion_file_writer_test.cpp | 5 +--- ...deletion_vector_index_file_writer_test.cpp | 4 +-- .../deletionvectors/deletion_vector_test.cpp | 19 ++++--------- .../operation/abstract_file_store_write.h | 4 --- .../core/operation/abstract_split_read.cpp | 15 ++++++++++ .../core/operation/abstract_split_read.h | 3 ++ .../core/operation/file_store_commit_impl.h | 8 ------ .../core/operation/merge_file_split_read.cpp | 28 ++----------------- .../core/operation/raw_file_split_read.cpp | 15 ++-------- src/paimon/testing/utils/test_helper.h | 6 ++-- test/inte/compaction_inte_test.cpp | 3 ++ 13 files changed, 39 insertions(+), 81 deletions(-) diff --git a/include/paimon/defs.h b/include/paimon/defs.h index b09a0496..f21e5303 100644 --- a/include/paimon/defs.h +++ b/include/paimon/defs.h @@ -238,6 +238,7 @@ struct PAIMON_EXPORT Options { /// "deletion-vectors.bitmap64" - Enable 64 bit bitmap implementation. Note that only 64 bit /// bitmap implementation is compatible with Iceberg. Default value is "false". + /// @note: bitmap64 dv is not supported. static const char DELETION_VECTOR_BITMAP64[]; /// @note `CHANGELOG_PRODUCER` currently only support `none` diff --git a/src/paimon/core/compact/compact_deletion_file_test.cpp b/src/paimon/core/compact/compact_deletion_file_test.cpp index 884d2273..bb36426d 100644 --- a/src/paimon/core/compact/compact_deletion_file_test.cpp +++ b/src/paimon/core/compact/compact_deletion_file_test.cpp @@ -128,10 +128,7 @@ TEST(CompactDeletionFileTest, MergeOldFileShouldRejectNonGeneratedType) { auto current = std::make_shared(meta, dv_index_file); auto old = std::make_shared(); - auto result = current->MergeOldFile(old); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().ToString().find("old should be a GeneratedDeletionFile") != - std::string::npos); + ASSERT_NOK_WITH_MSG(current->MergeOldFile(old), "old should be a GeneratedDeletionFile"); } TEST(CompactDeletionFileTest, MergeOldFileShouldRejectInvokedOldFile) { @@ -154,9 +151,7 @@ TEST(CompactDeletionFileTest, MergeOldFileShouldRejectInvokedOldFile) { auto old = std::make_shared(old_meta, dv_index_file); ASSERT_TRUE(old->GetOrCompute().has_value()); - auto result = current->MergeOldFile(old); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().ToString().find("old should not be get") != std::string::npos); + ASSERT_NOK_WITH_MSG(current->MergeOldFile(old), "old should not be get"); } TEST(CompactDeletionFileTest, MergeOldFileShouldReturnOldWhenCurrentIsNull) { diff --git a/src/paimon/core/deletionvectors/deletion_file_writer_test.cpp b/src/paimon/core/deletionvectors/deletion_file_writer_test.cpp index 9ec2d25c..af5c7dd2 100644 --- a/src/paimon/core/deletionvectors/deletion_file_writer_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_file_writer_test.cpp @@ -92,10 +92,7 @@ TEST(DeletionFileWriterTest, GetResultWithoutCloseShouldFail) { auto pool = GetDefaultPool(); ASSERT_OK_AND_ASSIGN(auto writer, DeletionFileWriter::Create(path_factory, fs, pool)); - auto result = writer->GetResult(); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().ToString().find("result length -1 out of int32 range") != - std::string::npos); + ASSERT_NOK_WITH_MSG(writer->GetResult(), "result length -1 out of int32 range"); } TEST(DeletionFileWriterTest, ExternalPathInResult) { diff --git a/src/paimon/core/deletionvectors/deletion_vector_index_file_writer_test.cpp b/src/paimon/core/deletionvectors/deletion_vector_index_file_writer_test.cpp index e804e0fd..a8cb59eb 100644 --- a/src/paimon/core/deletionvectors/deletion_vector_index_file_writer_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector_index_file_writer_test.cpp @@ -113,9 +113,7 @@ TEST(DeletionVectorIndexFileWriterTest, WriteSingleFileShouldReturnSerializeErro std::map> input; input["bad"] = std::make_shared(); - auto result = writer.WriteSingleFile(input); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().ToString().find("injected serialize failure") != std::string::npos); + ASSERT_NOK_WITH_MSG(writer.WriteSingleFile(input), "injected serialize failure"); } } // namespace paimon::test diff --git a/src/paimon/core/deletionvectors/deletion_vector_test.cpp b/src/paimon/core/deletionvectors/deletion_vector_test.cpp index 01913b0f..68e027c2 100644 --- a/src/paimon/core/deletionvectors/deletion_vector_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector_test.cpp @@ -114,10 +114,7 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamLengthMismatch) { reinterpret_cast(data.data()), data.size()); DataInputStream in(input_stream); auto pool = GetDefaultPool(); - - auto result = DeletionVector::Read(&in, /*length=*/9, pool.get()); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().ToString().find("Size not match") != std::string::npos); + ASSERT_NOK_WITH_MSG(DeletionVector::Read(&in, /*length=*/9, pool.get()), "Size not match"); } TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidBitmapLength) { @@ -130,9 +127,8 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidBitmapLength) { DataInputStream in(input_stream); auto pool = GetDefaultPool(); - auto result = DeletionVector::Read(&in, std::nullopt, pool.get()); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().ToString().find("Invalid bitmap length") != std::string::npos); + ASSERT_NOK_WITH_MSG(DeletionVector::Read(&in, std::nullopt, pool.get()), + "Invalid bitmap length"); } TEST(DeletionVectorTest, ReadFromDataInputStreamBitmap64NotImplemented) { @@ -146,9 +142,7 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamBitmap64NotImplemented) { DataInputStream in(input_stream); auto pool = GetDefaultPool(); - auto result = DeletionVector::Read(&in, std::nullopt, pool.get()); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().IsNotImplemented()); + ASSERT_NOK_WITH_MSG(DeletionVector::Read(&in, std::nullopt, pool.get()), "Not implemented"); } TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidMagicNumber) { @@ -161,9 +155,8 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidMagicNumber) { DataInputStream in(input_stream); auto pool = GetDefaultPool(); - auto result = DeletionVector::Read(&in, std::nullopt, pool.get()); - ASSERT_FALSE(result.ok()); - ASSERT_TRUE(result.status().ToString().find("Invalid magic number") != std::string::npos); + ASSERT_NOK_WITH_MSG(DeletionVector::Read(&in, std::nullopt, pool.get()), + "Invalid magic number"); } } // namespace paimon::test diff --git a/src/paimon/core/operation/abstract_file_store_write.h b/src/paimon/core/operation/abstract_file_store_write.h index 8fc0347b..e1726617 100644 --- a/src/paimon/core/operation/abstract_file_store_write.h +++ b/src/paimon/core/operation/abstract_file_store_write.h @@ -102,10 +102,6 @@ class AbstractFileStoreWrite : public FileStoreWrite { int32_t total_buckets = -1; }; - std::shared_ptr TEST_GetFileStorePathFactory() const { - return file_store_path_factory_; - } - protected: virtual Result> CreateWriter( 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 f5f37631..539730f3 100644 --- a/src/paimon/core/operation/abstract_split_read.cpp +++ b/src/paimon/core/operation/abstract_split_read.cpp @@ -124,6 +124,21 @@ std::unordered_map AbstractSplitRead::CreateDeletionF return deletion_file_map; } +DeletionVector::Factory AbstractSplitRead::CreateDeletionVectorFactory( + const std::unordered_map& deletion_file_map) const { + return [this, deletion_file_map]( + const std::string& file_name) -> Result> { + auto iter = deletion_file_map.find(file_name); + if (iter != deletion_file_map.end()) { + PAIMON_ASSIGN_OR_RAISE( + std::shared_ptr dv, + DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); + return dv; + } + return std::shared_ptr(); + }; +} + Result> AbstractSplitRead::ApplyPredicateFilterIfNeeded( std::unique_ptr&& reader, const std::shared_ptr& predicate) const { if (!context_->EnablePredicateFilter() || predicate == nullptr) { diff --git a/src/paimon/core/operation/abstract_split_read.h b/src/paimon/core/operation/abstract_split_read.h index 98bc0efd..94a1c4ec 100644 --- a/src/paimon/core/operation/abstract_split_read.h +++ b/src/paimon/core/operation/abstract_split_read.h @@ -81,6 +81,9 @@ class AbstractSplitRead : public SplitRead { const std::vector>& data_files, const std::vector>& deletion_files); + DeletionVector::Factory CreateDeletionVectorFactory( + const std::unordered_map& deletion_file_map) const; + Result> ApplyPredicateFilterIfNeeded( std::unique_ptr&& reader, const std::shared_ptr& predicate) const; diff --git a/src/paimon/core/operation/file_store_commit_impl.h b/src/paimon/core/operation/file_store_commit_impl.h index a7b61eb8..f54c85ac 100644 --- a/src/paimon/core/operation/file_store_commit_impl.h +++ b/src/paimon/core/operation/file_store_commit_impl.h @@ -120,14 +120,6 @@ class FileStoreCommitImpl : public FileStoreCommit { Status Init(std::unique_ptr ctx); - std::shared_ptr TEST_GetSchemaManager() const { - return schema_manager_; - } - - std::shared_ptr TEST_GetSnapshotManager() const { - return snapshot_manager_; - } - private: Status Commit(const std::shared_ptr& manifest_committable, bool check_append_files); diff --git a/src/paimon/core/operation/merge_file_split_read.cpp b/src/paimon/core/operation/merge_file_split_read.cpp index b124ea32..7154a2b0 100644 --- a/src/paimon/core/operation/merge_file_split_read.cpp +++ b/src/paimon/core/operation/merge_file_split_read.cpp @@ -72,7 +72,6 @@ namespace paimon { class BinaryRow; class DataFilePathFactory; class Executor; -struct DeletionFile; struct KeyValue; template class MergeFunctionWrapper; @@ -217,19 +216,7 @@ Result> MergeFileSplitRead::ApplyIndexAndDvRead Result> MergeFileSplitRead::CreateMergeReader( const std::shared_ptr& data_split, const std::shared_ptr& data_file_path_factory) { - auto deletion_file_map = AbstractSplitRead::CreateDeletionFileMap(*data_split); - - auto dv_factory = [this, deletion_file_map]( - const std::string& file_name) -> Result> { - auto iter = deletion_file_map.find(file_name); - if (iter != deletion_file_map.end()) { - PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr dv, - DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); - return dv; - } - return std::shared_ptr(); - }; + auto dv_factory = CreateDeletionVectorFactory(CreateDeletionFileMap(*data_split)); std::vector> sections = IntervalPartition(data_split->DataFiles(), interval_partition_comparator_).Partition(); @@ -250,18 +237,7 @@ Result> MergeFileSplitRead::CreateMergeReader( Result> MergeFileSplitRead::CreateNoMergeReader( const std::shared_ptr& data_split, bool only_filter_key, const std::shared_ptr& data_file_path_factory) const { - auto deletion_file_map = AbstractSplitRead::CreateDeletionFileMap(*data_split); - auto dv_factory = [this, deletion_file_map]( - const std::string& file_name) -> Result> { - auto iter = deletion_file_map.find(file_name); - if (iter != deletion_file_map.end()) { - PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr dv, - DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); - return dv; - } - return std::shared_ptr(); - }; + auto dv_factory = CreateDeletionVectorFactory(CreateDeletionFileMap(*data_split)); // create read schema without extra fields (e.g., completed key, sequence fields) auto row_kind_field = DataField::ConvertDataFieldToArrowField(SpecialFields::ValueKind()); diff --git a/src/paimon/core/operation/raw_file_split_read.cpp b/src/paimon/core/operation/raw_file_split_read.cpp index 74ce21a2..41928d3b 100644 --- a/src/paimon/core/operation/raw_file_split_read.cpp +++ b/src/paimon/core/operation/raw_file_split_read.cpp @@ -49,7 +49,6 @@ namespace paimon { class DataFilePathFactory; class Executor; class Predicate; -struct DeletionFile; RawFileSplitRead::RawFileSplitRead(const std::shared_ptr& path_factory, const std::shared_ptr& context, @@ -96,18 +95,8 @@ Result> RawFileSplitRead::CreateReader( const BinaryRow& partition, int32_t bucket, const std::vector>& data_files, const std::vector>& deletion_files) { - auto deletion_file_map = CreateDeletionFileMap(data_files, deletion_files); - auto dv_factory = [this, deletion_file_map]( - const std::string& file_name) -> Result> { - auto iter = deletion_file_map.find(file_name); - if (iter != deletion_file_map.end()) { - PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr dv, - DeletionVector::Read(options_.GetFileSystem().get(), iter->second, pool_.get())); - return dv; - } - return std::shared_ptr(); - }; + auto dv_factory = + CreateDeletionVectorFactory(CreateDeletionFileMap(data_files, deletion_files)); return CreateReader(partition, bucket, data_files, dv_factory); } diff --git a/src/paimon/testing/utils/test_helper.h b/src/paimon/testing/utils/test_helper.h index a8ae3447..186be7ff 100644 --- a/src/paimon/testing/utils/test_helper.h +++ b/src/paimon/testing/utils/test_helper.h @@ -340,17 +340,17 @@ class TestHelper { Result> LatestSnapshot() const { auto commit_impl = dynamic_cast(commit_.get()); - return commit_impl->TEST_GetSnapshotManager()->LatestSnapshotOfUser(commit_user_); + return commit_impl->snapshot_manager_->LatestSnapshotOfUser(commit_user_); } Result>> LatestSchema() const { auto commit_impl = dynamic_cast(commit_.get()); - return commit_impl->TEST_GetSchemaManager()->Latest(); + return commit_impl->schema_manager_->Latest(); } Result PartitionStr(const BinaryRow& partition) const { auto abstract_write = dynamic_cast(write_.get()); - return abstract_write->TEST_GetFileStorePathFactory()->GetPartitionString(partition); + return abstract_write->file_store_path_factory_->GetPartitionString(partition); } static void CheckCommitMessages(std::vector> expected, diff --git a/test/inte/compaction_inte_test.cpp b/test/inte/compaction_inte_test.cpp index 3f575a01..299b8c0e 100644 --- a/test/inte/compaction_inte_test.cpp +++ b/test/inte/compaction_inte_test.cpp @@ -109,6 +109,9 @@ class CompactionInteTest : public testing::Test, public ::testing::WithParamInte ASSERT_EQ(10, snapshot3.value().TotalRecordCount().value()); ASSERT_EQ(1, snapshot3.value().DeltaRecordCount().value()); + // @note: for append-only tables in Spark, native row-level deletes aren't supported during + // writing. Instead, deletions are expressed by committing a Deletion Vector (DV) file + // externally. if (with_dv) { auto partition = BinaryRowGenerator::GenerateRow({10}, pool_.get()); int32_t bucket = 1; From 590f16594d419b7f79d5f137e41cc27a0cd29a59 Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 18 Mar 2026 10:31:18 +0800 Subject: [PATCH 24/27] fix --- .../core/deletionvectors/deletion_vector_index_file_writer.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/paimon/core/deletionvectors/deletion_vector_index_file_writer.h b/src/paimon/core/deletionvectors/deletion_vector_index_file_writer.h index ac65110e..f7048995 100644 --- a/src/paimon/core/deletionvectors/deletion_vector_index_file_writer.h +++ b/src/paimon/core/deletionvectors/deletion_vector_index_file_writer.h @@ -34,6 +34,11 @@ class DeletionVectorIndexFileWriter { const std::shared_ptr& pool) : index_path_factory_(path_factory), fs_(fs), pool_(pool) {} + /// The deletion file of the bucketed table is updated according to the bucket. If a compaction + /// occurs and there is no longer a deletion file, an empty deletion file needs to be generated + /// to overwrite the old file. + /// TODO(yonghao.fyh): We can consider sending a message to delete the deletion file in the + /// future. Result> WriteSingleFile( const std::map>& input); From 833de3882dfa1c4537d1d3468611b0682e90819c Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 18 Mar 2026 11:15:22 +0800 Subject: [PATCH 25/27] fix --- src/paimon/core/deletionvectors/deletion_vector_test.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/paimon/core/deletionvectors/deletion_vector_test.cpp b/src/paimon/core/deletionvectors/deletion_vector_test.cpp index 68e027c2..435c75f9 100644 --- a/src/paimon/core/deletionvectors/deletion_vector_test.cpp +++ b/src/paimon/core/deletionvectors/deletion_vector_test.cpp @@ -142,7 +142,9 @@ TEST(DeletionVectorTest, ReadFromDataInputStreamBitmap64NotImplemented) { DataInputStream in(input_stream); auto pool = GetDefaultPool(); - ASSERT_NOK_WITH_MSG(DeletionVector::Read(&in, std::nullopt, pool.get()), "Not implemented"); + ASSERT_NOK_WITH_MSG( + DeletionVector::Read(&in, std::nullopt, pool.get()), + "NotImplemented: bitmap64 deletion vectors are not supported in this version"); } TEST(DeletionVectorTest, ReadFromDataInputStreamInvalidMagicNumber) { From 58c30cb9a4a643edbf9961e804161c3608c9178c Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 18 Mar 2026 13:20:24 +0800 Subject: [PATCH 26/27] fix --- build_support/lint_exclusions.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/build_support/lint_exclusions.txt b/build_support/lint_exclusions.txt index 1ea00ba6..acc713a9 100644 --- a/build_support/lint_exclusions.txt +++ b/build_support/lint_exclusions.txt @@ -10,3 +10,4 @@ *vendored/* *RcppExports.cpp* *arrowExports.cpp* +test_helper.h From c32dd7b76794c0ac4383fc1c7eb84dad0b7934ea Mon Sep 17 00:00:00 2001 From: "yonghao.fyh" Date: Wed, 18 Mar 2026 14:22:33 +0800 Subject: [PATCH 27/27] fix --- build_support/lint_exclusions.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build_support/lint_exclusions.txt b/build_support/lint_exclusions.txt index acc713a9..ac3933d4 100644 --- a/build_support/lint_exclusions.txt +++ b/build_support/lint_exclusions.txt @@ -10,4 +10,4 @@ *vendored/* *RcppExports.cpp* *arrowExports.cpp* -test_helper.h +*src/paimon/testing/utils/test_helper.h