diff --git a/LICENSE b/LICENSE index 18605853..a4f84868 100644 --- a/LICENSE +++ b/LICENSE @@ -266,6 +266,8 @@ This product includes code from Apache Iceberg C++. * Avro direct decoder/encoder: * src/paimon/format/avro/avro_direct_decoder.cpp * src/paimon/format/avro/avro_direct_decoder.h + * src/paimon/format/avro/avro_direct_encoder.cpp + * src/paimon/format/avro/avro_direct_encoder.h Copyright: 2024-2025 The Apache Software Foundation. Home page: https://iceberg.apache.org/ diff --git a/cmake_modules/DefineOptions.cmake b/cmake_modules/DefineOptions.cmake index 0211f409..69668b9f 100644 --- a/cmake_modules/DefineOptions.cmake +++ b/cmake_modules/DefineOptions.cmake @@ -92,7 +92,7 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}") define_option_string(PAIMON_CXXFLAGS "Compiler flags to append when compiling Paimon" "") - define_option(PAIMON_BUILD_STATIC "Build static libraries" OFF) + define_option(PAIMON_BUILD_STATIC "Build static libraries" ON) define_option(PAIMON_BUILD_SHARED "Build shared libraries" ON) #---------------------------------------------------------------------- diff --git a/include/paimon/defs.h b/include/paimon/defs.h index 65367bb5..110b2884 100644 --- a/include/paimon/defs.h +++ b/include/paimon/defs.h @@ -130,7 +130,7 @@ struct PAIMON_EXPORT Options { static const char MANIFEST_TARGET_FILE_SIZE[]; /// "manifest.format" - Specify the message format of manifest files. - /// Default value is orc. + /// Default value is avro. static const char MANIFEST_FORMAT[]; /// "manifest.compression" - File compression for manifest, default value is zstd. diff --git a/include/paimon/read_context.h b/include/paimon/read_context.h index 53544391..f6b624a1 100644 --- a/include/paimon/read_context.h +++ b/include/paimon/read_context.h @@ -27,7 +27,6 @@ #include "paimon/result.h" #include "paimon/type_fwd.h" #include "paimon/utils/read_ahead_cache.h" -#include "paimon/utils/special_field_ids.h" #include "paimon/visibility.h" namespace paimon { @@ -179,9 +178,9 @@ class PAIMON_EXPORT ReadContextBuilder { /// @param read_field_ids Vector of field ids to read from the table. /// @return Reference to this builder for method chaining. /// @note Currently supports top-level field selection. Future versions may support - /// nested field selection using ArrowSchema for more granular projection, - /// If SetReadFieldIds() call and SetReadSchema() are natually are mutually - /// exclusive. Calling both will ignore the read schema set by SetReadSchema(). + /// nested field selection using ArrowSchema for more granular projection. + /// @note SetReadFieldIds() and SetReadSchema() are mutually exclusive. + /// Calling both will ignore the read schema set by SetReadSchema(). ReadContextBuilder& SetReadFieldIds(const std::vector& read_field_ids); /// Set a configuration options map to set some option entries which are not defined in the diff --git a/include/paimon/reader/file_batch_reader.h b/include/paimon/reader/file_batch_reader.h index 21c780a1..708c8b23 100644 --- a/include/paimon/reader/file_batch_reader.h +++ b/include/paimon/reader/file_batch_reader.h @@ -50,7 +50,7 @@ class PAIMON_EXPORT FileBatchReader : public BatchReader { virtual uint64_t GetPreviousBatchFirstRowNumber() const = 0; /// Get the number of rows in the file. - virtual uint64_t GetNumberOfRows() const = 0; + virtual Result GetNumberOfRows() const = 0; /// Get whether or not support read precisely while bitmap pushed down. virtual bool SupportPreciseBitmapSelection() const = 0; diff --git a/src/paimon/common/reader/delegating_prefetch_reader.h b/src/paimon/common/reader/delegating_prefetch_reader.h index 99dd5c62..432ddfd5 100644 --- a/src/paimon/common/reader/delegating_prefetch_reader.h +++ b/src/paimon/common/reader/delegating_prefetch_reader.h @@ -58,7 +58,7 @@ class DelegatingPrefetchReader : public FileBatchReader { return GetReader()->GetPreviousBatchFirstRowNumber(); } - uint64_t GetNumberOfRows() const override { + Result GetNumberOfRows() const override { return GetReader()->GetNumberOfRows(); } diff --git a/src/paimon/common/reader/prefetch_file_batch_reader_impl.cpp b/src/paimon/common/reader/prefetch_file_batch_reader_impl.cpp index 9533e55c..bbb3c396 100644 --- a/src/paimon/common/reader/prefetch_file_batch_reader_impl.cpp +++ b/src/paimon/common/reader/prefetch_file_batch_reader_impl.cpp @@ -208,9 +208,11 @@ Status PrefetchFileBatchReaderImpl::SetReadRanges( read_ranges_.push_back(read_range); } // Note: add a special read range out of file row count, for trigger an EOF access. - read_ranges_.push_back(EofRange()); + std::pair eof_range; + PAIMON_ASSIGN_OR_RAISE(eof_range, EofRange()); + read_ranges_.push_back(eof_range); for (auto& read_ranges : read_ranges_in_group_) { - read_ranges.push_back(EofRange()); + read_ranges.push_back(eof_range); } return Status::OK(); } @@ -420,7 +422,9 @@ Status PrefetchFileBatchReaderImpl::HandleReadResult( } prefetch_queue->push({read_range, std::move(read_batch_with_bitmap), first_row_number}); } else { - prefetch_queue->push({EofRange(), std::move(read_batch_with_bitmap), first_row_number}); + std::pair eof_range; + PAIMON_ASSIGN_OR_RAISE(eof_range, EofRange()); + prefetch_queue->push({eof_range, std::move(read_batch_with_bitmap), first_row_number}); readers_pos_[reader_idx]->store(std::numeric_limits::max()); } return Status::OK(); @@ -490,7 +494,8 @@ Result PrefetchFileBatchReaderImpl::NextBatchW } } value_count++; - if (IsEofRange(peek_batch->read_range)) { + PAIMON_ASSIGN_OR_RAISE(bool is_eof_range, IsEofRange(peek_batch->read_range)); + if (is_eof_range) { eof_count++; continue; } @@ -550,7 +555,7 @@ uint64_t PrefetchFileBatchReaderImpl::GetPreviousBatchFirstRowNumber() const { return previous_batch_first_row_num_; } -uint64_t PrefetchFileBatchReaderImpl::GetNumberOfRows() const { +Result PrefetchFileBatchReaderImpl::GetNumberOfRows() const { assert(!readers_.empty()); return readers_[0]->GetNumberOfRows(); } @@ -569,13 +574,15 @@ Status PrefetchFileBatchReaderImpl::GetReadStatus() const { std::shared_lock lock(rw_mutex_); return read_status_; } -bool PrefetchFileBatchReaderImpl::IsEofRange( +Result PrefetchFileBatchReaderImpl::IsEofRange( const std::pair& read_range) const { - return read_range.first >= GetNumberOfRows(); + PAIMON_ASSIGN_OR_RAISE(uint64_t num_rows, GetNumberOfRows()); + return read_range.first >= num_rows; } -std::pair PrefetchFileBatchReaderImpl::EofRange() const { - return {GetNumberOfRows(), GetNumberOfRows() + 1}; +Result> PrefetchFileBatchReaderImpl::EofRange() const { + PAIMON_ASSIGN_OR_RAISE(uint64_t num_rows, GetNumberOfRows()); + return std::make_pair(num_rows, num_rows + 1); } void PrefetchFileBatchReaderImpl::Close() { diff --git a/src/paimon/common/reader/prefetch_file_batch_reader_impl.h b/src/paimon/common/reader/prefetch_file_batch_reader_impl.h index 23c6da01..d6eb3881 100644 --- a/src/paimon/common/reader/prefetch_file_batch_reader_impl.h +++ b/src/paimon/common/reader/prefetch_file_batch_reader_impl.h @@ -77,7 +77,7 @@ class PrefetchFileBatchReaderImpl : public PrefetchFileBatchReader { Status SeekToRow(uint64_t row_number) override; uint64_t GetPreviousBatchFirstRowNumber() const override; - uint64_t GetNumberOfRows() const override; + Result GetNumberOfRows() const override; uint64_t GetNextRowToRead() const override; void Close() override; Status SetReadRanges(const std::vector>& read_ranges) override; @@ -117,7 +117,7 @@ class PrefetchFileBatchReaderImpl : public PrefetchFileBatchReader { void Workloop(); void SetReadStatus(const Status& status); Status GetReadStatus() const; - bool IsEofRange(const std::pair& read_range) const; + Result IsEofRange(const std::pair& read_range) const; Status DoReadBatch(size_t reader_idx); void ReadBatch(size_t reader_idx); size_t GetEnabledReaderSize() const; @@ -128,7 +128,7 @@ class PrefetchFileBatchReaderImpl : public PrefetchFileBatchReader { static std::vector>> DispatchReadRanges( const std::vector>& read_ranges, size_t reader_count); - std::pair EofRange() const; + Result> EofRange() const; std::optional> GetCurrentReadRange(size_t reader_idx) const; Status EnsureReaderPosition(size_t reader_idx, const std::pair& read_range) const; diff --git a/src/paimon/common/utils/arrow/arrow_utils.h b/src/paimon/common/utils/arrow/arrow_utils.h index 5c23ec83..2a634dc4 100644 --- a/src/paimon/common/utils/arrow/arrow_utils.h +++ b/src/paimon/common/utils/arrow/arrow_utils.h @@ -39,14 +39,17 @@ class ArrowUtils { return std::make_shared(struct_type->fields()); } - static std::vector CreateProjection( + static Result> CreateProjection( const std::shared_ptr<::arrow::Schema>& file_schema, const arrow::FieldVector& read_fields) { std::vector target_to_src_mapping; target_to_src_mapping.reserve(read_fields.size()); for (const auto& field : read_fields) { auto src_field_idx = file_schema->GetFieldIndex(field->name()); - assert(src_field_idx >= 0); + if (src_field_idx < 0) { + return Status::Invalid( + fmt::format("Field '{}' not found or duplicate in file schema", field->name())); + } target_to_src_mapping.push_back(src_field_idx); } return target_to_src_mapping; diff --git a/src/paimon/common/utils/arrow/arrow_utils_test.cpp b/src/paimon/common/utils/arrow/arrow_utils_test.cpp index afc86d10..11c0e717 100644 --- a/src/paimon/common/utils/arrow/arrow_utils_test.cpp +++ b/src/paimon/common/utils/arrow/arrow_utils_test.cpp @@ -18,30 +18,93 @@ #include "arrow/api.h" #include "gtest/gtest.h" -#include "paimon/common/types/data_field.h" +#include "paimon/testing/utils/testharness.h" namespace paimon::test { TEST(ArrowUtilsTest, TestCreateProjection) { - std::vector read_fields = {DataField(1, arrow::field("k1", arrow::int32())), - DataField(3, arrow::field("p1", arrow::int32())), - DataField(5, arrow::field("s1", arrow::utf8())), - DataField(6, arrow::field("v0", arrow::float64())), - DataField(7, arrow::field("v1", arrow::boolean()))}; - auto read_schema = DataField::ConvertDataFieldsToArrowSchema(read_fields); + arrow::FieldVector file_fields = { + arrow::field("k0", arrow::int32()), arrow::field("k1", arrow::int32()), + arrow::field("p1", arrow::int32()), arrow::field("s1", arrow::utf8()), + arrow::field("v0", arrow::float64()), arrow::field("v1", arrow::boolean()), + arrow::field("s0", arrow::utf8())}; + auto file_schema = arrow::schema(file_fields); - std::vector file_fields = {DataField(0, arrow::field("k0", arrow::int32())), - DataField(1, arrow::field("k1", arrow::int32())), - DataField(3, arrow::field("p1", arrow::int32())), - DataField(5, arrow::field("s1", arrow::utf8())), - DataField(6, arrow::field("v0", arrow::float64())), - DataField(7, arrow::field("v1", arrow::boolean())), - DataField(4, arrow::field("s0", arrow::utf8()))}; - auto file_schema = DataField::ConvertDataFieldsToArrowSchema(file_fields); - - auto projection = ArrowUtils::CreateProjection(file_schema, read_schema->fields()); - std::vector expected_projection = {1, 2, 3, 4, 5}; - ASSERT_EQ(projection, expected_projection); + { + // normal case + arrow::FieldVector read_fields = { + arrow::field("k1", arrow::int32()), arrow::field("p1", arrow::int32()), + arrow::field("s1", arrow::utf8()), arrow::field("v0", arrow::float64()), + arrow::field("v1", arrow::boolean())}; + auto read_schema = arrow::schema(read_fields); + ASSERT_OK_AND_ASSIGN(std::vector projection, + ArrowUtils::CreateProjection(file_schema, read_schema->fields())); + std::vector expected_projection = {1, 2, 3, 4, 5}; + ASSERT_EQ(projection, expected_projection); + } + { + // duplicate read field + arrow::FieldVector read_fields = { + arrow::field("k1", arrow::int32()), arrow::field("p1", arrow::int32()), + arrow::field("s1", arrow::utf8()), arrow::field("v0", arrow::float64()), + arrow::field("v0", arrow::float64()), arrow::field("v1", arrow::boolean())}; + auto read_schema = arrow::schema(read_fields); + ASSERT_OK_AND_ASSIGN(std::vector projection, + ArrowUtils::CreateProjection(file_schema, read_schema->fields())); + std::vector expected_projection = {1, 2, 3, 4, 4, 5}; + ASSERT_EQ(projection, expected_projection); + } + { + // duplicate read field, and sizeof(read_fields) > sizeof(file_fields) + arrow::FieldVector read_fields = { + arrow::field("k1", arrow::int32()), arrow::field("p1", arrow::int32()), + arrow::field("s1", arrow::utf8()), arrow::field("v0", arrow::float64()), + arrow::field("v0", arrow::float64()), arrow::field("v0", arrow::float64()), + arrow::field("v0", arrow::float64()), arrow::field("v0", arrow::float64()), + arrow::field("v1", arrow::boolean())}; + auto read_schema = arrow::schema(read_fields); + ASSERT_OK_AND_ASSIGN(std::vector projection, + ArrowUtils::CreateProjection(file_schema, read_schema->fields())); + std::vector expected_projection = {1, 2, 3, 4, 4, 4, 4, 4, 5}; + ASSERT_EQ(projection, expected_projection); + } + { + // read field not found in file schema + arrow::FieldVector read_fields = { + arrow::field("k1", arrow::int32()), arrow::field("p1", arrow::int32()), + arrow::field("s1", arrow::utf8()), arrow::field("v2", arrow::float64()), + arrow::field("v1", arrow::boolean())}; + auto read_schema = arrow::schema(read_fields); + ASSERT_NOK_WITH_MSG(ArrowUtils::CreateProjection(file_schema, read_schema->fields()), + "Field 'v2' not found or duplicate in file schema"); + } + { + // duplicate field in file schema + arrow::FieldVector file_fields_dup = { + arrow::field("k0", arrow::int32()), arrow::field("k1", arrow::int32()), + arrow::field("p1", arrow::int32()), arrow::field("s1", arrow::utf8()), + arrow::field("v0", arrow::float64()), arrow::field("v1", arrow::boolean()), + arrow::field("v1", arrow::boolean()), arrow::field("s0", arrow::utf8())}; + auto file_schema_dup = arrow::schema(file_fields_dup); + arrow::FieldVector read_fields = { + arrow::field("k1", arrow::int32()), arrow::field("p1", arrow::int32()), + arrow::field("s1", arrow::utf8()), arrow::field("v1", arrow::float64()), + arrow::field("v1", arrow::boolean())}; + auto read_schema = arrow::schema(read_fields); + ASSERT_NOK_WITH_MSG(ArrowUtils::CreateProjection(file_schema_dup, read_schema->fields()), + "Field 'v1' not found or duplicate in file schema"); + } + { + arrow::FieldVector read_fields = { + arrow::field("k1", arrow::int32()), arrow::field("p1", arrow::int32()), + arrow::field("s1", arrow::utf8()), arrow::field("v0", arrow::float64()), + arrow::field("v1", arrow::boolean())}; + auto read_schema = arrow::schema(read_fields); + ASSERT_OK_AND_ASSIGN(std::vector projection, + ArrowUtils::CreateProjection(file_schema, read_schema->fields())); + std::vector expected_projection = {1, 2, 3, 4, 5}; + ASSERT_EQ(projection, expected_projection); + } } } // namespace paimon::test diff --git a/src/paimon/common/utils/date_time_utils.h b/src/paimon/common/utils/date_time_utils.h index 96a15373..6a48f438 100644 --- a/src/paimon/common/utils/date_time_utils.h +++ b/src/paimon/common/utils/date_time_utils.h @@ -167,6 +167,22 @@ class DateTimeUtils { return tz ? tz->name() : "UTC"; } + static std::string GetArrowTimeUnitStr(arrow::TimeUnit::type unit) { + switch (unit) { + case arrow::TimeUnit::SECOND: + return "SECOND"; + case arrow::TimeUnit::MILLI: + return "MILLISECOND"; + case arrow::TimeUnit::MICRO: + return "MICROSECOND"; + case arrow::TimeUnit::NANO: + return "NANOSECOND"; + default: + break; + } + return "UNKNOWN"; + } + // there may be a precision loss for nano static Result ToUTCTimestamp(const Timestamp& timestamp) { int64_t micro_second = timestamp.ToMicrosecond(); diff --git a/src/paimon/core/core_options.cpp b/src/paimon/core/core_options.cpp index dadc38a2..ca1c2d99 100644 --- a/src/paimon/core/core_options.cpp +++ b/src/paimon/core/core_options.cpp @@ -353,11 +353,8 @@ Result CoreOptions::FromMap( // Parse file format and file system configurations PAIMON_RETURN_NOT_OK(parser.ParseObject( Options::FILE_FORMAT, /*default_identifier=*/"parquet", &impl->file_format)); - if (impl->file_format->Identifier() == "avro") { - return Status::NotImplemented("not support avro as file format"); - } PAIMON_RETURN_NOT_OK(parser.ParseObject( - Options::MANIFEST_FORMAT, /*default_identifier=*/"orc", &impl->manifest_file_format)); + Options::MANIFEST_FORMAT, /*default_identifier=*/"avro", &impl->manifest_file_format)); PAIMON_RETURN_NOT_OK(parser.ParseFileSystem(fs_scheme_to_identifier_map, specified_file_system, &impl->file_system)); diff --git a/src/paimon/core/core_options_test.cpp b/src/paimon/core/core_options_test.cpp index fa005d4c..ee016723 100644 --- a/src/paimon/core/core_options_test.cpp +++ b/src/paimon/core/core_options_test.cpp @@ -32,7 +32,7 @@ namespace paimon::test { TEST(CoreOptionsTest, TestDefaultValue) { ASSERT_OK_AND_ASSIGN(CoreOptions core_options, CoreOptions::FromMap({})); - ASSERT_EQ(core_options.GetManifestFormat()->Identifier(), "orc"); + ASSERT_EQ(core_options.GetManifestFormat()->Identifier(), "avro"); ASSERT_EQ(core_options.GetWriteFileFormat()->Identifier(), "parquet"); ASSERT_TRUE(core_options.GetFileSystem()); ASSERT_EQ(-1, core_options.GetBucket()); diff --git a/src/paimon/core/io/complete_row_tracking_fields_reader.h b/src/paimon/core/io/complete_row_tracking_fields_reader.h index eed2f4a9..812ae5d5 100644 --- a/src/paimon/core/io/complete_row_tracking_fields_reader.h +++ b/src/paimon/core/io/complete_row_tracking_fields_reader.h @@ -64,7 +64,7 @@ class CompleteRowTrackingFieldsBatchReader : public FileBatchReader { return reader_->GetPreviousBatchFirstRowNumber(); } - uint64_t GetNumberOfRows() const override { + Result GetNumberOfRows() const override { return reader_->GetNumberOfRows(); } diff --git a/src/paimon/core/operation/abstract_split_read.cpp b/src/paimon/core/operation/abstract_split_read.cpp index 5717a1b6..54ec2890 100644 --- a/src/paimon/core/operation/abstract_split_read.cpp +++ b/src/paimon/core/operation/abstract_split_read.cpp @@ -147,7 +147,6 @@ Result> AbstractSplitRead::CreateFileBatchReade // lance do not support stream build with input stream return reader_builder->Build(data_file_path); } - // TODO(zhanyu.fyh): orc format support prefetch if (context_->EnablePrefetch() && file_format_identifier != "blob" && file_format_identifier != "avro") { PAIMON_ASSIGN_OR_RAISE( diff --git a/src/paimon/core/operation/key_value_file_store_scan_test.cpp b/src/paimon/core/operation/key_value_file_store_scan_test.cpp index 1c259d20..5b653746 100644 --- a/src/paimon/core/operation/key_value_file_store_scan_test.cpp +++ b/src/paimon/core/operation/key_value_file_store_scan_test.cpp @@ -66,7 +66,7 @@ class KeyValueFileStoreScanTest : public testing::Test { Result> CreateFileStoreScan( const std::string& table_path, const std::shared_ptr& scan_filter, int32_t table_schema_id, int32_t snapshot_id) const { - std::map options_map = {}; + std::map options_map = {{Options::MANIFEST_FORMAT, "orc"}}; PAIMON_ASSIGN_OR_RAISE(CoreOptions core_options, CoreOptions::FromMap(options_map)); auto fs = core_options.GetFileSystem(); auto schema_manager = std::make_shared(fs, table_path); diff --git a/src/paimon/core/operation/merge_file_split_read.cpp b/src/paimon/core/operation/merge_file_split_read.cpp index c6688e07..f02e1dc8 100644 --- a/src/paimon/core/operation/merge_file_split_read.cpp +++ b/src/paimon/core/operation/merge_file_split_read.cpp @@ -106,8 +106,9 @@ Result> MergeFileSplitRead::Create( int32_t key_arity = trimmed_primary_key.size(); // projection is the mapping from value_schema in KeyValue object to raw_read_schema - std::vector projection = - ArrowUtils::CreateProjection(value_schema, context->GetReadSchema()->fields()); + PAIMON_ASSIGN_OR_RAISE( + std::vector projection, + ArrowUtils::CreateProjection(value_schema, context->GetReadSchema()->fields())); return std::unique_ptr(new MergeFileSplitRead( path_factory, context, diff --git a/src/paimon/core/operation/orphan_files_cleaner.cpp b/src/paimon/core/operation/orphan_files_cleaner.cpp index db61464e..e6d61a79 100644 --- a/src/paimon/core/operation/orphan_files_cleaner.cpp +++ b/src/paimon/core/operation/orphan_files_cleaner.cpp @@ -170,13 +170,14 @@ Result> OrphanFilesCleaner::Create( if (!table_schema.value()->PrimaryKeys().empty()) { return Status::NotImplemented("orphan files cleaner only support append table"); } + // merge options const auto& schema = table_schema.value(); auto opts = schema->Options(); for (const auto& [key, value] : ctx->GetOptions()) { opts[key] = value; } PAIMON_ASSIGN_OR_RAISE(CoreOptions options, - CoreOptions::FromMap(ctx->GetOptions(), ctx->GetSpecificFileSystem())); + CoreOptions::FromMap(opts, ctx->GetSpecificFileSystem())); auto arrow_schema = DataField::ConvertDataFieldsToArrowSchema(schema->Fields()); PAIMON_ASSIGN_OR_RAISE(std::vector external_paths, options.CreateExternalPaths()); PAIMON_ASSIGN_OR_RAISE(std::optional global_index_external_path, diff --git a/src/paimon/core/operation/raw_file_split_read.cpp b/src/paimon/core/operation/raw_file_split_read.cpp index d6da20d1..286c0775 100644 --- a/src/paimon/core/operation/raw_file_split_read.cpp +++ b/src/paimon/core/operation/raw_file_split_read.cpp @@ -151,7 +151,8 @@ Result> RawFileSplitRead::ApplyIndexAndDvReaderIfNe actual_selection = *selection; } else if (deletion) { actual_selection = *deletion; - actual_selection.value().Flip(0, file_reader->GetNumberOfRows()); + PAIMON_ASSIGN_OR_RAISE(uint64_t num_rows, file_reader->GetNumberOfRows()); + actual_selection.value().Flip(0, num_rows); } if (actual_selection && actual_selection.value().IsEmpty()) { diff --git a/src/paimon/core/postpone/postpone_bucket_writer_test.cpp b/src/paimon/core/postpone/postpone_bucket_writer_test.cpp index 798ad4ef..ace20825 100644 --- a/src/paimon/core/postpone/postpone_bucket_writer_test.cpp +++ b/src/paimon/core/postpone/postpone_bucket_writer_test.cpp @@ -89,6 +89,7 @@ class PostponeBucketWriterTest : public ::testing::Test, } void CheckFileContent(const std::string& file_format_str, const std::string& data_file_name, + const std::shared_ptr& file_schema, const std::shared_ptr& expected_array) const { ASSERT_OK_AND_ASSIGN(std::shared_ptr input_stream, file_system_->Open(data_file_name)); @@ -97,9 +98,14 @@ class PostponeBucketWriterTest : public ::testing::Test, ASSERT_OK_AND_ASSIGN(auto reader_builder, file_format->CreateReaderBuilder(/*batch_size=*/10)); ASSERT_OK_AND_ASSIGN(auto batch_reader, reader_builder->Build(input_stream)); + auto c_schema = std::make_unique<::ArrowSchema>(); + ASSERT_TRUE(arrow::ExportType(*file_schema, c_schema.get()).ok()); + ASSERT_OK(batch_reader->SetReadSchema(c_schema.get(), /*predicate=*/nullptr, + /*selection_bitmap=*/std::nullopt)); ASSERT_OK_AND_ASSIGN(std::shared_ptr result_array, ReadResultCollector::CollectResult(batch_reader.get())); - ASSERT_TRUE(expected_array->Equals(result_array)) << result_array->ToString(); + ASSERT_TRUE(expected_array->Equals(result_array)) << result_array->ToString() << "\n != \n" + << expected_array->ToString(); } private: @@ -116,6 +122,9 @@ std::vector GetTestValuesForPostponeBucketWriterTest() { std::vector values = {"parquet"}; #ifdef PAIMON_ENABLE_ORC values.emplace_back("orc"); +#endif +#ifdef PAIMON_ENABLE_AVRO + values.emplace_back("avro"); #endif return values; } @@ -168,7 +177,7 @@ TEST_P(PostponeBucketWriterTest, TestSimple) { ])"}, &expected_array); ASSERT_TRUE(array_status.ok()); - CheckFileContent(file_format, expected_data_file_path, expected_array); + CheckFileContent(file_format, expected_data_file_path, write_type_, expected_array); // check data file meta ASSERT_TRUE(commit_increment.GetCompactIncrement().IsEmpty()); @@ -240,16 +249,16 @@ TEST_P(PostponeBucketWriterTest, TestNestedType) { arrow::field("_VALUE_KIND", arrow::int8())}; write_fields.insert(write_fields.end(), fields.begin(), fields.end()); std::shared_ptr expected_array; - auto array_status = - arrow::ipc::internal::json::ChunkedArrayFromJSON(arrow::struct_(write_fields), {R"([ + auto write_type = arrow::struct_(write_fields); + auto array_status = arrow::ipc::internal::json::ChunkedArrayFromJSON(write_type, {R"([ [-1, 0, "Lucy", [null, [1, true], null], [[[1, true], true]]], [-1, 0, "Bob", [[2, false], null], null], [-1, 0, "David", [[2, false], [3, true], [4, null]], [[[1, true], true], [[5, false], null]]], [-1, 0, "Alice", null, null] ])"}, - &expected_array); + &expected_array); ASSERT_TRUE(array_status.ok()); - CheckFileContent(file_format, expected_data_file_path, expected_array); + CheckFileContent(file_format, expected_data_file_path, write_type, expected_array); // check data file meta ASSERT_TRUE(commit_increment.GetCompactIncrement().IsEmpty()); @@ -345,7 +354,7 @@ TEST_P(PostponeBucketWriterTest, TestWriteMultiBatch) { ])"}, &expected_array); ASSERT_TRUE(array_status.ok()); - CheckFileContent(file_format, expected_data_file_path, expected_array); + CheckFileContent(file_format, expected_data_file_path, write_type_, expected_array); // check data file meta ASSERT_TRUE(commit_increment.GetCompactIncrement().IsEmpty()); @@ -444,7 +453,7 @@ TEST_P(PostponeBucketWriterTest, TestMultiplePrepareCommit) { ])"}, &expected_array1); ASSERT_TRUE(array_status.ok()); - CheckFileContent(file_format, expected_data_file_dir + expected_data_file_name1, + CheckFileContent(file_format, expected_data_file_dir + expected_data_file_name1, write_type_, expected_array1); std::shared_ptr expected_array2; @@ -454,7 +463,7 @@ TEST_P(PostponeBucketWriterTest, TestMultiplePrepareCommit) { ])"}, &expected_array2); ASSERT_TRUE(array_status.ok()); - CheckFileContent(file_format, expected_data_file_dir + expected_data_file_name2, + CheckFileContent(file_format, expected_data_file_dir + expected_data_file_name2, write_type_, expected_array2); // check data file meta diff --git a/src/paimon/format/avro/CMakeLists.txt b/src/paimon/format/avro/CMakeLists.txt index 97c4113a..f0d2c48c 100644 --- a/src/paimon/format/avro/CMakeLists.txt +++ b/src/paimon/format/avro/CMakeLists.txt @@ -15,15 +15,16 @@ if(PAIMON_ENABLE_AVRO) set(PAIMON_AVRO_FILE_FORMAT - avro_adaptor.cpp avro_direct_decoder.cpp + avro_direct_encoder.cpp avro_file_batch_reader.cpp avro_file_format.cpp avro_file_format_factory.cpp avro_format_writer.cpp avro_input_stream_impl.cpp avro_output_stream_impl.cpp - avro_schema_converter.cpp) + avro_schema_converter.cpp + avro_stats_extractor.cpp) add_paimon_lib(paimon_avro_file_format SOURCES @@ -49,11 +50,12 @@ if(PAIMON_ENABLE_AVRO) if(PAIMON_BUILD_TESTS) add_paimon_test(avro_format_test SOURCES - avro_adaptor_test.cpp avro_file_batch_reader_test.cpp avro_file_format_test.cpp + avro_format_writer_test.cpp avro_input_stream_impl_test.cpp avro_schema_converter_test.cpp + avro_stats_extractor_test.cpp avro_writer_builder_test.cpp EXTRA_INCLUDES ${AVRO_INCLUDE_DIR} diff --git a/src/paimon/format/avro/avro_adaptor.cpp b/src/paimon/format/avro/avro_adaptor.cpp deleted file mode 100644 index d71d0126..00000000 --- a/src/paimon/format/avro/avro_adaptor.cpp +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Copyright 2024-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/format/avro/avro_adaptor.h" - -#include -#include - -#include "arrow/api.h" -#include "arrow/array/array_base.h" -#include "arrow/array/array_binary.h" -#include "arrow/array/array_nested.h" -#include "arrow/array/array_primitive.h" -#include "arrow/util/checked_cast.h" -#include "avro/GenericDatum.hh" -#include "avro/Node.hh" -#include "avro/ValidSchema.hh" -#include "paimon/status.h" - -namespace paimon::avro { - -Result> AvroAdaptor::ConvertArrayToGenericDatums( - const std::shared_ptr& array, const ::avro::ValidSchema& avro_schema) const { - std::vector<::avro::GenericDatum> datums; - for (int32_t i = 0; i < array->length(); i++) { - PAIMON_ASSIGN_OR_RAISE(::avro::GenericDatum datum, - ConvertArrayToGenericDatum(avro_schema, array, i)); - datums.push_back(datum); - } - return datums; -} - -Result<::avro::GenericDatum> AvroAdaptor::ConvertArrayToGenericDatum( - const ::avro::ValidSchema& avro_schema, const std::shared_ptr& arrow_array, - int32_t row_idx) const { - std::shared_ptr struct_array = - arrow::internal::checked_pointer_cast(arrow_array); - if (struct_array == nullptr) { - return Status::Invalid("arrow array should be struct array"); - } - ::avro::GenericDatum datum(avro_schema.root()); - auto& record = datum.value<::avro::GenericRecord>(); - const auto& node = avro_schema.root(); - const auto& fields = type_->fields(); - for (size_t i = 0; i < fields.size(); i++) { - std::shared_ptr field_array = struct_array->field(i); - switch (fields[i]->type()->id()) { - case arrow::Type::type::BOOL: { - auto array = - arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - SetValue(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::INT8: { - auto array = arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - // avro only support int32_t and int64_t - SetValue(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::INT16: { - auto array = arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - // avro only support int32_t and int64_t - SetValue(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::INT32: { - auto array = arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - SetValue(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::INT64: { - auto array = arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - SetValue(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::FLOAT: { - auto array = arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - SetValue(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::DOUBLE: { - auto array = arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - SetValue(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::STRING: { - auto array = arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - SetValue(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::BINARY: { - auto array = arrow::internal::checked_pointer_cast(field_array); - auto datum = ::avro::GenericDatum(avro_schema.root()->leafAt(i)); - SetValue, arrow::BinaryArray>(row_idx, array, &datum); - record.setFieldAt(i, datum); - break; - } - case arrow::Type::type::STRUCT: { - ::avro::ValidSchema leaf_schema(node->leafAt(i)); - PAIMON_ASSIGN_OR_RAISE( - ::avro::GenericDatum datum, - ConvertArrayToGenericDatum(leaf_schema, field_array, row_idx)); - record.setFieldAt(i, datum); - break; - } - default: { - return Status::Invalid("unsupported type"); - } - } - } - return datum; -} - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_adaptor.h b/src/paimon/format/avro/avro_adaptor.h deleted file mode 100644 index 6afa9849..00000000 --- a/src/paimon/format/avro/avro_adaptor.h +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Copyright 2024-present Alibaba Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#pragma once - -#include -#include -#include -#include -#include - -#include "arrow/api.h" -#include "arrow/array/array_binary.h" -#include "avro/GenericDatum.hh" -#include "avro/ValidSchema.hh" -#include "paimon/result.h" - -namespace arrow { -class Array; -class DataType; -} // namespace arrow -namespace avro { -class ValidSchema; -} // namespace avro - -namespace paimon::avro { - -class AvroAdaptor { - public: - explicit AvroAdaptor(const std::shared_ptr& type) : type_(type) {} - - Result> ConvertArrayToGenericDatums( - const std::shared_ptr& array, const ::avro::ValidSchema& avro_schema) const; - - private: - Result<::avro::GenericDatum> ConvertArrayToGenericDatum( - const ::avro::ValidSchema& avro_schema, const std::shared_ptr& arrow_array, - int32_t row_idx) const; - - template - static void SetValue(int32_t row_idx, const std::shared_ptr& array, - ::avro::GenericDatum* datum); - - std::shared_ptr type_; -}; - -template -void AvroAdaptor::SetValue(int32_t row_idx, const std::shared_ptr& array, - ::avro::GenericDatum* datum) { - if (datum->isUnion()) { - if (!array->IsNull(row_idx)) { - datum->selectBranch(1); - datum->value() = array->Value(row_idx); - } - } else { - datum->value() = array->Value(row_idx); - } -} - -template <> -inline void AvroAdaptor::SetValue( - int32_t row_idx, const std::shared_ptr& array, - ::avro::GenericDatum* datum) { - if (datum->isUnion()) { - if (!array->IsNull(row_idx)) { - datum->selectBranch(1); - datum->value() = array->GetString(row_idx); - } - } else { - datum->value() = array->GetString(row_idx); - } -} - -template <> -inline void AvroAdaptor::SetValue, arrow::BinaryArray>( - int32_t row_idx, const std::shared_ptr& array, - ::avro::GenericDatum* datum) { - if (datum->isUnion()) { - if (!array->IsNull(row_idx)) { - datum->selectBranch(1); - std::string_view view = array->Value(row_idx); - datum->value>() = std::vector(view.begin(), view.end()); - } - } else { - std::string_view view = array->Value(row_idx); - datum->value>() = std::vector(view.begin(), view.end()); - } -} - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_adaptor_test.cpp b/src/paimon/format/avro/avro_adaptor_test.cpp deleted file mode 100644 index 34cf6624..00000000 --- a/src/paimon/format/avro/avro_adaptor_test.cpp +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Copyright 2024-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/format/avro/avro_adaptor.h" - -#include - -#include "arrow/api.h" -#include "arrow/array/array_base.h" -#include "arrow/c/abi.h" -#include "arrow/c/bridge.h" -#include "arrow/ipc/json_simple.h" -#include "avro/GenericDatum.hh" -#include "gtest/gtest.h" -#include "paimon/common/utils/arrow/mem_utils.h" -#include "paimon/core/utils/manifest_meta_reader.h" -#include "paimon/format/avro/avro_schema_converter.h" -#include "paimon/memory/memory_pool.h" -#include "paimon/status.h" -#include "paimon/testing/utils/testharness.h" - -namespace paimon::avro::test { - -TEST(AvroAdaptorTest, Simple) { - arrow::FieldVector fields = {arrow::field("f0", arrow::boolean(), /*nullable=*/true), - arrow::field("f1", arrow::int8(), /*nullable=*/true), - arrow::field("f2", arrow::int16(), /*nullable=*/true), - arrow::field("f3", arrow::int32(), /*nullable=*/true), - arrow::field("field_null", arrow::int32(), /*nullable=*/true), - arrow::field("f4", arrow::int64(), /*nullable=*/true), - arrow::field("f5", arrow::float32(), /*nullable=*/true), - arrow::field("f6", arrow::float64(), /*nullable=*/true), - arrow::field("f7", arrow::utf8(), /*nullable=*/true)}; - // TODO(jinli.zjw): add test for struct and binary - std::shared_ptr data_type = arrow::struct_(fields); - std::shared_ptr schema = arrow::schema(fields); - - AvroAdaptor adaptor(data_type); - - std::string data_str = - R"([[true, 0, 32767, 2147483647, null, 4294967295, 0.5, 1.141592659, "20250327" ], - [false, 1, 32767, null, null, 4294967296, 1.0, 2.141592658, "20250327" ], - [null, 1, 32767, 2147483647, null, null, 2.0, 3.141592657, null], - [true, -2, -32768, -2147483648, null, -4294967298, 2.0, 3.141592657, "20250326"]])"; - auto array = arrow::ipc::internal::json::ArrayFromJSON(data_type, {data_str}).ValueOrDie(); - - ASSERT_OK_AND_ASSIGN(auto avro_schema, AvroSchemaConverter::ArrowSchemaToAvroSchema(schema)); - ASSERT_OK_AND_ASSIGN(std::vector<::avro::GenericDatum> datums, - adaptor.ConvertArrayToGenericDatums(array, avro_schema)); - ASSERT_EQ(4, datums.size()); -} - -} // namespace paimon::avro::test diff --git a/src/paimon/format/avro/avro_direct_decoder.cpp b/src/paimon/format/avro/avro_direct_decoder.cpp index dc47fe41..37b016b5 100644 --- a/src/paimon/format/avro/avro_direct_decoder.cpp +++ b/src/paimon/format/avro/avro_direct_decoder.cpp @@ -76,10 +76,6 @@ Status SkipAvroValue(const ::avro::NodePtr& avro_node, ::avro::Decoder* decoder) decoder->skipBytes(); return Status::OK(); - case ::avro::AVRO_FIXED: - decoder->skipFixed(avro_node->fixedSize()); - return Status::OK(); - case ::avro::AVRO_RECORD: { // Skip all fields in order for (size_t i = 0; i < avro_node->leaves(); ++i) { @@ -88,10 +84,6 @@ Status SkipAvroValue(const ::avro::NodePtr& avro_node, ::avro::Decoder* decoder) return Status::OK(); } - case ::avro::AVRO_ENUM: - decoder->decodeEnum(); - return Status::OK(); - case ::avro::AVRO_ARRAY: { const auto& element_node = avro_node->leafAt(0); // skipArray() returns count like arrayStart(), must handle all blocks diff --git a/src/paimon/format/avro/avro_direct_encoder.cpp b/src/paimon/format/avro/avro_direct_encoder.cpp new file mode 100644 index 00000000..c906e41f --- /dev/null +++ b/src/paimon/format/avro/avro_direct_encoder.cpp @@ -0,0 +1,378 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +// Adapted from Apache Iceberg C++ +// https://github.com/apache/iceberg-cpp/blob/main/src/iceberg/avro/avro_direct_encoder.cc + +#include "paimon/format/avro/avro_direct_encoder.h" + +#include +#include + +#include "arrow/api.h" +#include "arrow/type.h" +#include "arrow/util/checked_cast.h" +#include "fmt/format.h" +#include "paimon/common/utils/date_time_utils.h" +#include "paimon/format/avro/avro_utils.h" +#include "paimon/result.h" + +namespace paimon::avro { + +namespace { + +// Utility struct for union branch information +struct UnionBranches { + size_t null_index; + size_t value_index; + ::avro::NodePtr value_node; +}; + +Result ValidateUnion(const ::avro::NodePtr& union_node) { + if (PAIMON_UNLIKELY(union_node->leaves() != 2)) { + return Status::Invalid( + fmt::format("Union must have exactly 2 branches, got {}", union_node->leaves())); + } + + const auto& branch_0 = union_node->leafAt(0); + const auto& branch_1 = union_node->leafAt(1); + + if (branch_0->type() == ::avro::AVRO_NULL && branch_1->type() != ::avro::AVRO_NULL) { + return UnionBranches{.null_index = 0, .value_index = 1, .value_node = branch_1}; + } + if (branch_1->type() == ::avro::AVRO_NULL && branch_0->type() != ::avro::AVRO_NULL) { + return UnionBranches{.null_index = 1, .value_index = 0, .value_node = branch_0}; + } + return Status::Invalid("Union must have exactly one null branch"); +} + +} // namespace + +Status AvroDirectEncoder::EncodeArrowToAvro(const ::avro::NodePtr& avro_node, + const arrow::Array& array, int64_t row_index, + ::avro::Encoder* encoder, EncodeContext* ctx) { + if (PAIMON_UNLIKELY(row_index < 0 || row_index >= array.length())) { + return Status::Invalid( + fmt::format("Row index {} out of bounds {}", row_index, array.length())); + } + + const bool is_null = array.IsNull(row_index); + + if (avro_node->type() == ::avro::AVRO_UNION) { + PAIMON_ASSIGN_OR_RAISE(UnionBranches branches, ValidateUnion(avro_node)); + + if (is_null) { + encoder->encodeUnionIndex(branches.null_index); + encoder->encodeNull(); + return Status::OK(); + } + + encoder->encodeUnionIndex(branches.value_index); + return EncodeArrowToAvro(branches.value_node, array, row_index, encoder, ctx); + } + + if (is_null) { + return Status::Invalid("Null value in non-nullable field"); + } + + switch (avro_node->type()) { + case ::avro::AVRO_NULL: + encoder->encodeNull(); + return Status::OK(); + + case ::avro::AVRO_BOOL: { + const auto& bool_array = + arrow::internal::checked_cast(array); + encoder->encodeBool(bool_array.Value(row_index)); + return Status::OK(); + } + + case ::avro::AVRO_INT: { + // AVRO_INT can represent: int32, date (days since epoch) + switch (array.type()->id()) { + case arrow::Type::INT8: { + const auto& int8_array = + arrow::internal::checked_cast(array); + encoder->encodeInt(int8_array.Value(row_index)); + return Status::OK(); + } + case arrow::Type::INT16: { + const auto& int16_array = + arrow::internal::checked_cast(array); + encoder->encodeInt(int16_array.Value(row_index)); + return Status::OK(); + } + + case arrow::Type::INT32: { + const auto& int32_array = + arrow::internal::checked_cast(array); + encoder->encodeInt(int32_array.Value(row_index)); + return Status::OK(); + } + case arrow::Type::DATE32: { + const auto& date_array = + arrow::internal::checked_cast(array); + encoder->encodeInt(date_array.Value(row_index)); + return Status::OK(); + } + default: + return Status::Invalid( + fmt::format("AVRO_INT expects Int32Array or Date32Array, got {}", + array.type()->ToString())); + } + } + + case ::avro::AVRO_LONG: { + // AVRO_LONG can represent: int64, timestamp + switch (array.type()->id()) { + case arrow::Type::INT64: { + const auto& int64_array = + arrow::internal::checked_cast(array); + encoder->encodeLong(int64_array.Value(row_index)); + return Status::OK(); + } + case arrow::Type::TIMESTAMP: { + const auto& timestamp_array = + arrow::internal::checked_cast(array); + int64_t timestamp = timestamp_array.Value(row_index); + + auto ts_type = + arrow::internal::checked_pointer_cast(array.type()); + arrow::TimeUnit::type unit = ts_type->unit(); + const auto& logical_type = avro_node->logicalType().type(); + + // NOTE: Java Avro only support TIMESTAMP_MILLIS && TIMESTAMP_MICROS + if (((logical_type == ::avro::LogicalType::TIMESTAMP_MILLIS || + logical_type == ::avro::LogicalType::LOCAL_TIMESTAMP_MILLIS) && + unit == arrow::TimeUnit::MILLI) || + ((logical_type == ::avro::LogicalType::TIMESTAMP_MICROS || + logical_type == ::avro::LogicalType::LOCAL_TIMESTAMP_MICROS) && + unit == arrow::TimeUnit::MICRO) || + ((logical_type == ::avro::LogicalType::TIMESTAMP_NANOS || + logical_type == ::avro::LogicalType::LOCAL_TIMESTAMP_NANOS) && + unit == arrow::TimeUnit::NANO)) { + encoder->encodeLong(timestamp); + } else if ((logical_type == ::avro::LogicalType::TIMESTAMP_MILLIS || + logical_type == ::avro::LogicalType::LOCAL_TIMESTAMP_MILLIS) && + unit == arrow::TimeUnit::SECOND) { + // for arrow second, we need to convert it to avro millisecond + encoder->encodeLong( + timestamp * + DateTimeUtils::CONVERSION_FACTORS[DateTimeUtils::MILLISECOND]); + } else { + return Status::Invalid( + fmt::format("Unsupported timestamp type with avro logical type {} and " + "arrow time unit {}.", + AvroUtils::ToString(avro_node->logicalType()), + DateTimeUtils::GetArrowTimeUnitStr(unit))); + } + return Status::OK(); + } + default: + return Status::Invalid( + fmt::format("AVRO_LONG expects Int64Array, or TimestampArray, got {}", + array.type()->ToString())); + } + } + + case ::avro::AVRO_FLOAT: { + const auto& float_array = + arrow::internal::checked_cast(array); + encoder->encodeFloat(float_array.Value(row_index)); + return Status::OK(); + } + + case ::avro::AVRO_DOUBLE: { + const auto& double_array = + arrow::internal::checked_cast(array); + encoder->encodeDouble(double_array.Value(row_index)); + return Status::OK(); + } + + case ::avro::AVRO_STRING: { + const auto& string_array = + arrow::internal::checked_cast(array); + std::string_view value = string_array.GetView(row_index); + encoder->encodeString(std::string(value)); + return Status::OK(); + } + + case ::avro::AVRO_BYTES: { + // Handle DECIMAL + if (avro_node->logicalType().type() == ::avro::LogicalType::DECIMAL) { + const auto& decimal_array = + arrow::internal::checked_cast(array); + std::string_view decimal_value = decimal_array.GetView(row_index); + ctx->assign(decimal_value.begin(), decimal_value.end()); + // Arrow Decimal128 bytes are in little-endian order, Avro requires big-endian + std::reverse(ctx->begin(), ctx->end()); + encoder->encodeBytes(ctx->data(), ctx->size()); + return Status::OK(); + } + + // Handle regular BYTES + const auto& binary_array = + arrow::internal::checked_cast(array); + std::string_view value = binary_array.GetView(row_index); + // TODO(jinli.zjw): need to copy to ctx? + ctx->assign(value.begin(), value.end()); + encoder->encodeBytes(ctx->data(), ctx->size()); + return Status::OK(); + } + + case ::avro::AVRO_RECORD: { + if (PAIMON_UNLIKELY(array.type()->id() != arrow::Type::STRUCT)) { + return Status::Invalid(fmt::format("AVRO_RECORD expects StructArray, got {}", + array.type()->ToString())); + } + + const auto& struct_array = + arrow::internal::checked_cast(array); + const size_t num_fields = avro_node->leaves(); + + if (PAIMON_UNLIKELY(struct_array.num_fields() != static_cast(num_fields))) { + return Status::Invalid(fmt::format( + "Field count mismatch: Arrow struct has {} fields, Avro node has {} fields", + struct_array.num_fields(), num_fields)); + } + + for (size_t i = 0; i < num_fields; ++i) { + const auto& field_node = avro_node->leafAt(i); + const auto& field_array = struct_array.field(static_cast(i)); + + PAIMON_RETURN_NOT_OK( + EncodeArrowToAvro(field_node, *field_array, row_index, encoder, ctx)); + } + return Status::OK(); + } + + case ::avro::AVRO_ARRAY: { + const auto& element_node = avro_node->leafAt(0); + + // Handle ListArray + if (array.type()->id() == arrow::Type::LIST) { + const auto& list_array = + arrow::internal::checked_cast(array); + + const auto start = list_array.value_offset(row_index); + const auto end = list_array.value_offset(row_index + 1); + const auto length = end - start; + + encoder->arrayStart(); + if (length > 0) { + encoder->setItemCount(length); + const auto& values = list_array.values(); + + for (int64_t i = start; i < end; ++i) { + encoder->startItem(); + PAIMON_RETURN_NOT_OK( + EncodeArrowToAvro(element_node, *values, i, encoder, ctx)); + } + } + encoder->arrayEnd(); + return Status::OK(); + } else if (array.type()->id() == arrow::Type::MAP && + AvroUtils::HasMapLogicalType(avro_node)) { + // Handle MapArray (for Avro maps with non-string keys) + if (PAIMON_UNLIKELY(element_node->type() != ::avro::AVRO_RECORD || + element_node->leaves() != 2)) { + return Status::Invalid( + fmt::format("Expected AVRO_RECORD for map key-value pair, got {}", + ::avro::toString(element_node->type()))); + } + + const auto& map_array = + arrow::internal::checked_cast(array); + + const auto start = map_array.value_offset(row_index); + const auto end = map_array.value_offset(row_index + 1); + const auto length = end - start; + + encoder->arrayStart(); + if (length > 0) { + encoder->setItemCount(length); + const auto& keys = map_array.keys(); + const auto& values = map_array.items(); + + // The element_node should be a RECORD with "key" and "value" fields + for (int64_t i = start; i < end; ++i) { + const auto& key_node = element_node->leafAt(0); + const auto& value_node = element_node->leafAt(1); + + encoder->startItem(); + PAIMON_RETURN_NOT_OK(EncodeArrowToAvro(key_node, *keys, i, encoder, ctx)); + PAIMON_RETURN_NOT_OK( + EncodeArrowToAvro(value_node, *values, i, encoder, ctx)); + } + } + encoder->arrayEnd(); + return Status::OK(); + } + + return Status::Invalid(fmt::format( + "AVRO_ARRAY must map to ListArray or MapArray, got {}", array.type()->ToString())); + } + + case ::avro::AVRO_MAP: { + if (PAIMON_UNLIKELY(array.type()->id() != arrow::Type::MAP)) { + return Status::Invalid( + fmt::format("AVRO_MAP expects MapArray, got {}", array.type()->ToString())); + } + const auto& map_array = arrow::internal::checked_cast(array); + + const auto start = map_array.value_offset(row_index); + const auto end = map_array.value_offset(row_index + 1); + const auto length = end - start; + + encoder->mapStart(); + if (length > 0) { + encoder->setItemCount(length); + const auto& keys = map_array.keys(); + const auto& values = map_array.items(); + const auto& value_node = avro_node->leafAt(1); + + if (PAIMON_UNLIKELY(keys->type()->id() != arrow::Type::STRING)) { + return Status::Invalid(fmt::format("AVRO_MAP keys must be StringArray, got {}", + keys->type()->ToString())); + } + + for (int64_t i = start; i < end; ++i) { + encoder->startItem(); + const auto& string_array = + arrow::internal::checked_cast(*keys); + std::string_view key_value = string_array.GetView(i); + encoder->encodeString(std::string(key_value)); + + PAIMON_RETURN_NOT_OK(EncodeArrowToAvro(value_node, *values, i, encoder, ctx)); + } + } + encoder->mapEnd(); + return Status::OK(); + } + + case ::avro::AVRO_UNION: + // Already handled above + return Status::Invalid("Unexpected union handling"); + default: + return Status::Invalid( + fmt::format("Unsupported Avro type: {}", ::avro::toString(avro_node->type()))); + } +} + +} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_direct_encoder.h b/src/paimon/format/avro/avro_direct_encoder.h new file mode 100644 index 00000000..6d0f995a --- /dev/null +++ b/src/paimon/format/avro/avro_direct_encoder.h @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +// Adapted from Apache Iceberg C++ +// https://github.com/apache/iceberg-cpp/blob/main/src/iceberg/avro/avro_direct_encoder_internal.h + +#pragma once + +#include + +#include "arrow/api.h" +#include "avro/Encoder.hh" +#include "avro/Node.hh" +#include "paimon/status.h" + +namespace paimon::avro { + +class AvroDirectEncoder { + public: + /// Context for reusing scratch buffers during Avro encoding + /// + /// Avoids frequent small allocations by reusing temporary buffers across multiple encode + /// operations. This is particularly important for binary/decimal types (reused across rows). + using EncodeContext = std::vector; + + /// Directly encode Arrow data to Avro without GenericDatum + /// + /// Eliminates the GenericDatum intermediate layer by directly calling Avro encoder + /// methods from Arrow arrays. + /// + /// @param avro_node The Avro schema node for the data being encoded + /// @param array The Arrow array containing the data to encode + /// @param row_index The index of the row to encode within the array + /// @param encoder The Avro encoder to write data to + /// @param ctx Encode context for reusing scratch buffers + /// @return Status indicating success, or an error status + static Status EncodeArrowToAvro(const ::avro::NodePtr& avro_node, const arrow::Array& array, + int64_t row_index, ::avro::Encoder* encoder, + EncodeContext* ctx); +}; + +} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_file_batch_reader.cpp b/src/paimon/format/avro/avro_file_batch_reader.cpp index 3d068d99..1cccdf1a 100644 --- a/src/paimon/format/avro/avro_file_batch_reader.cpp +++ b/src/paimon/format/avro/avro_file_batch_reader.cpp @@ -25,6 +25,7 @@ #include "paimon/common/utils/arrow/arrow_utils.h" #include "paimon/common/utils/arrow/mem_utils.h" #include "paimon/common/utils/arrow/status_utils.h" +#include "paimon/common/utils/scope_guard.h" #include "paimon/format/avro/avro_input_stream_impl.h" #include "paimon/format/avro/avro_schema_converter.h" #include "paimon/reader/batch_reader.h" @@ -158,7 +159,8 @@ Status AvroFileBatchReader::SetReadSchema(::ArrowSchema* read_schema, Result> AvroFileBatchReader::CalculateReadFieldsProjection( const std::shared_ptr<::arrow::Schema>& file_schema, const arrow::FieldVector& read_fields) { std::set projection_set; - auto projection = ArrowUtils::CreateProjection(file_schema, read_fields); + PAIMON_ASSIGN_OR_RAISE(std::vector projection, + ArrowUtils::CreateProjection(file_schema, read_fields)); int32_t prev_index = -1; for (auto& index : projection) { if (index <= prev_index) { @@ -178,4 +180,31 @@ Result> AvroFileBatchReader::GetFileSchema() cons return c_schema; } +Result AvroFileBatchReader::GetNumberOfRows() const { + if (!total_rows_) { + PAIMON_ASSIGN_OR_RAISE(int64_t current_pos, input_stream_->GetPos()); + ScopeGuard stream_guard([this, current_pos]() -> void { + // reset input stream position to original position + Status status = input_stream_->Seek(current_pos, SeekOrigin::FS_SEEK_SET); + (void)status; + }); + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr<::avro::DataFileReaderBase> reader, + CreateDataFileReader(input_stream_, pool_)); + ScopeGuard reader_guard([&reader]() -> void { reader->close(); }); + try { + while (reader->hasMore()) { + reader->decr(); + total_rows_ = total_rows_.value_or(0) + 1; + } + } catch (const ::avro::Exception& e) { + return Status::Invalid(fmt::format("avro reader GetNumberOfRows failed. {}", e.what())); + } catch (const std::exception& e) { + return Status::Invalid(fmt::format("avro reader GetNumberOfRows failed. {}", e.what())); + } catch (...) { + return Status::Invalid("avro reader GetNumberOfRows failed. unknown error"); + } + } + return *total_rows_; +} + } // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_file_batch_reader.h b/src/paimon/format/avro/avro_file_batch_reader.h index 495c4a2a..38aa054c 100644 --- a/src/paimon/format/avro/avro_file_batch_reader.h +++ b/src/paimon/format/avro/avro_file_batch_reader.h @@ -49,10 +49,7 @@ class AvroFileBatchReader : public FileBatchReader { return previous_first_row_; } - uint64_t GetNumberOfRows() const override { - assert(false); - return -1; - } + Result GetNumberOfRows() const override; std::shared_ptr GetReaderMetrics() const override { return metrics_; @@ -93,6 +90,7 @@ class AvroFileBatchReader : public FileBatchReader { std::optional> read_fields_projection_; uint64_t previous_first_row_ = std::numeric_limits::max(); uint64_t next_row_to_read_ = std::numeric_limits::max(); + mutable std::optional total_rows_ = std::nullopt; const int32_t batch_size_; bool close_ = false; std::shared_ptr metrics_; diff --git a/src/paimon/format/avro/avro_file_batch_reader_test.cpp b/src/paimon/format/avro/avro_file_batch_reader_test.cpp index 5f03eade..10ddda00 100644 --- a/src/paimon/format/avro/avro_file_batch_reader_test.cpp +++ b/src/paimon/format/avro/avro_file_batch_reader_test.cpp @@ -47,15 +47,8 @@ class AvroFileBatchReaderTest : public ::testing::Test, public ::testing::WithPa } void TearDown() override {} - std::pair, std::shared_ptr> - ReadBatchWithCustomizedData(const std::shared_ptr& src_array, - int32_t read_batch_size) { - std::string file_path = PathUtil::JoinPath(dir_->Str(), "file.avro"); - WriteData(src_array, file_path); - return ReadData(file_path, read_batch_size); - } - - void WriteData(const std::shared_ptr& src_array, const std::string& file_path) { + void WriteData(const std::shared_ptr& src_array, const std::string& file_path, + const std::string& compression) { arrow::Schema src_schema(src_array->type()->fields()); ::ArrowSchema c_schema; ASSERT_TRUE(arrow::ExportSchema(src_schema, &c_schema).ok()); @@ -63,7 +56,7 @@ class AvroFileBatchReaderTest : public ::testing::Test, public ::testing::WithPa file_format_->CreateWriterBuilder(&c_schema, /*batch_size=*/-1)); ASSERT_OK_AND_ASSIGN(std::shared_ptr out, fs_->Create(file_path, /*overwrite=*/false)); - ASSERT_OK_AND_ASSIGN(auto writer, writer_builder->Build(out, "zstd")); + ASSERT_OK_AND_ASSIGN(auto writer, writer_builder->Build(out, compression)); ::ArrowArray arrow_array; ASSERT_TRUE(arrow::ExportArray(*src_array, &arrow_array).ok()); @@ -117,6 +110,8 @@ TEST_F(AvroFileBatchReaderTest, TestReadDataWithNull) { ASSERT_TRUE(array_status.ok()) << array_status.ToString(); ASSERT_TRUE(result_array->Equals(expected_array)); ASSERT_TRUE(expected_array->Equals(result_array)); + auto read_metrics = reader_holder->GetReaderMetrics(); + ASSERT_TRUE(read_metrics); } TEST_F(AvroFileBatchReaderTest, TestReadWithDifferentBatchSize) { @@ -149,9 +144,9 @@ TEST_F(AvroFileBatchReaderTest, TestReadWithDifferentBatchSize) { data_str.append("]"); std::shared_ptr src_array = - arrow::ipc::internal::json::ArrayFromJSON(arrow_data_type, data_str).ValueOr(nullptr); + arrow::ipc::internal::json::ArrayFromJSON(arrow_data_type, data_str).ValueOrDie(); ASSERT_TRUE(src_array); - WriteData(src_array, file_path); + WriteData(src_array, file_path, /*compression=*/"zstd"); for (int32_t batch_size : {1024, 512, 256, 128, 64, 32, 16, 8, 4, 2, 1}) { auto [reader_holder, result_array] = ReadData(file_path, batch_size); @@ -246,7 +241,7 @@ TEST_P(AvroFileBatchReaderTest, TestReadTimestampTypes) { ASSERT_TRUE(expected_array->Equals(result_array)); } -TEST_P(AvroFileBatchReaderTest, TestReadMapTypes) { +TEST_F(AvroFileBatchReaderTest, TestReadMapTypes) { std::string path = paimon::test::GetDataDir() + "/avro/append_with_multiple_map.db/" "append_with_multiple_map/bucket-0/" @@ -299,7 +294,7 @@ TEST_P(AvroFileBatchReaderTest, TestReadMapTypes) { ASSERT_TRUE(expected_array->Equals(result_array)); } -TEST_P(AvroFileBatchReaderTest, TestReadRowNumbers) { +TEST_F(AvroFileBatchReaderTest, TestGetPreviousBatchFirstRowNumber) { std::string path = paimon::test::GetDataDir() + "/avro/append_simple.db/" "append_simple/bucket-0/" @@ -322,6 +317,8 @@ TEST_P(AvroFileBatchReaderTest, TestReadRowNumbers) { EXPECT_OK(reader->SetReadSchema(c_schema.get(), /*predicate=*/nullptr, /*selection_bitmap=*/std::nullopt)); + ASSERT_OK_AND_ASSIGN(auto num_rows, reader->GetNumberOfRows()); + ASSERT_EQ(4, num_rows); ASSERT_EQ(std::numeric_limits::max(), reader->GetPreviousBatchFirstRowNumber()); ASSERT_OK_AND_ASSIGN(auto batch1, reader->NextBatch()); ArrowArrayRelease(batch1.first.get()); @@ -344,6 +341,66 @@ TEST_P(AvroFileBatchReaderTest, TestReadRowNumbers) { ASSERT_TRUE(BatchReader::IsEofBatch(batch5)); } +TEST_F(AvroFileBatchReaderTest, TestGetNumberOfRows) { + std::string file_path = PathUtil::JoinPath(dir_->Str(), "file.avro"); + + arrow::FieldVector fields = { + arrow::field("f0", arrow::boolean()), arrow::field("f1", arrow::int32()), + arrow::field("f2", arrow::int64()), arrow::field("f3", arrow::float32()), + arrow::field("f4", arrow::float64()), arrow::field("f5", arrow::utf8()), + arrow::field("f6", arrow::binary())}; + auto arrow_data_type = arrow::struct_(fields); + + size_t length = 102400; + std::string data_str = "["; + for (size_t i = 0; i < length; i++) { + data_str.append(fmt::format(R"([{}, {}, {}, {}, {}, "str_{}", "bin_{}"])", "true", i, + i * 100000000000L, i * 0.12, i * 123.45678901, i, i)); + if (i != length - 1) { + data_str.append(","); + } + } + data_str.append("]"); + + std::shared_ptr src_array = + arrow::ipc::internal::json::ArrayFromJSON(arrow_data_type, data_str).ValueOrDie(); + ASSERT_TRUE(src_array); + WriteData(src_array, file_path, /*compression=*/"null"); + + ASSERT_OK_AND_ASSIGN(auto reader_builder, file_format_->CreateReaderBuilder(25600)); + + // check GetNumberOfRows can be called at any position, and continue read + int32_t expected_batches = 4; + for (int32_t pos = 0; pos < expected_batches; pos++) { + ASSERT_OK_AND_ASSIGN(std::shared_ptr in, fs_->Open(file_path)); + ASSERT_OK_AND_ASSIGN(auto reader, reader_builder->Build(in)); + + arrow::ArrayVector result_array_vector; + for (int32_t i = 0; i < pos; i++) { + ASSERT_OK_AND_ASSIGN(auto batch, reader->NextBatch()); + auto result_array = + arrow::ImportArray(batch.first.get(), batch.second.get()).ValueOrDie(); + result_array_vector.push_back(result_array); + } + // check number of rows, and continue read + ASSERT_OK_AND_ASSIGN(auto num_rows, reader->GetNumberOfRows()); + ASSERT_EQ(length, num_rows); + for (int32_t i = pos; i < expected_batches; i++) { + ASSERT_OK_AND_ASSIGN(auto batch, reader->NextBatch()); + auto result_array = + arrow::ImportArray(batch.first.get(), batch.second.get()).ValueOrDie(); + result_array_vector.push_back(result_array); + } + ASSERT_OK_AND_ASSIGN(auto eof_batch, reader->NextBatch()); + ASSERT_TRUE(BatchReader::IsEofBatch(eof_batch)); + ASSERT_OK_AND_ASSIGN(num_rows, reader->GetNumberOfRows()); + ASSERT_EQ(length, num_rows); + + auto result_array = arrow::ChunkedArray(result_array_vector); + ASSERT_TRUE(result_array.Equals(arrow::ChunkedArray(src_array))); + } +} + INSTANTIATE_TEST_SUITE_P(TestParam, AvroFileBatchReaderTest, ::testing::Values(false, true)); } // namespace paimon::avro::test diff --git a/src/paimon/format/avro/avro_file_format.cpp b/src/paimon/format/avro/avro_file_format.cpp index 0d9e7908..df08d5ce 100644 --- a/src/paimon/format/avro/avro_file_format.cpp +++ b/src/paimon/format/avro/avro_file_format.cpp @@ -23,6 +23,7 @@ #include "arrow/c/helpers.h" #include "paimon/common/utils/arrow/status_utils.h" #include "paimon/format/avro/avro_reader_builder.h" +#include "paimon/format/avro/avro_stats_extractor.h" #include "paimon/format/avro/avro_writer_builder.h" #include "paimon/status.h" @@ -54,7 +55,7 @@ Result> AvroFileFormat::CreateWriterBuilder( Result> AvroFileFormat::CreateStatsExtractor( ::ArrowSchema* schema) const { ArrowSchemaRelease(schema); - return Status::NotImplemented("do not support extract stats in avro format for now"); + return std::make_unique(options_); } } // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_file_format_test.cpp b/src/paimon/format/avro/avro_file_format_test.cpp index 0f0f0e04..f51364f7 100644 --- a/src/paimon/format/avro/avro_file_format_test.cpp +++ b/src/paimon/format/avro/avro_file_format_test.cpp @@ -21,14 +21,12 @@ #include #include "arrow/api.h" -#include "arrow/array/array_base.h" #include "arrow/c/abi.h" #include "arrow/c/bridge.h" #include "arrow/ipc/json_simple.h" #include "gtest/gtest.h" -#include "paimon/common/utils/arrow/mem_utils.h" +#include "paimon/common/utils/date_time_utils.h" #include "paimon/common/utils/path_util.h" -#include "paimon/core/utils/manifest_meta_reader.h" #include "paimon/format/file_format.h" #include "paimon/format/file_format_factory.h" #include "paimon/format/format_writer.h" @@ -36,7 +34,6 @@ #include "paimon/format/writer_builder.h" #include "paimon/fs/file_system.h" #include "paimon/fs/local/local_file_system.h" -#include "paimon/memory/memory_pool.h" #include "paimon/reader/file_batch_reader.h" #include "paimon/result.h" #include "paimon/status.h" @@ -45,71 +42,172 @@ namespace paimon::avro::test { -class AvroFileFormatTest : public testing::Test, - public ::testing::WithParamInterface {}; +class AvroFileFormatTest : public testing::Test, public ::testing::WithParamInterface { + public: + void SetUp() override { + ASSERT_OK_AND_ASSIGN(file_format_, FileFormatFactory::Get("avro", {})); + fs_ = std::make_shared(); + dir_ = ::paimon::test::UniqueTestDirectory::Create(); + ASSERT_TRUE(dir_); + } + void TearDown() override {} + + void WriteDataAndCheckResult(const arrow::FieldVector& fields, const std::string& data_str) { + auto schema = arrow::schema(fields); + auto data_type = arrow::struct_(fields); + + ::ArrowSchema c_schema; + ASSERT_TRUE(arrow::ExportSchema(*schema, &c_schema).ok()); + ASSERT_OK_AND_ASSIGN(auto writer_builder, + file_format_->CreateWriterBuilder(&c_schema, 1024)); + ASSERT_OK_AND_ASSIGN( + std::shared_ptr out, + fs_->Create(PathUtil::JoinPath(dir_->Str(), "file.avro"), /*overwrite=*/false)); + + auto compression = GetParam(); + ASSERT_OK_AND_ASSIGN(auto writer, writer_builder->Build(out, compression)); + + auto input_array = + arrow::ipc::internal::json::ArrayFromJSON(data_type, data_str).ValueOrDie(); + ASSERT_TRUE(input_array); + ::ArrowArray c_array; + ASSERT_TRUE(arrow::ExportArray(*input_array, &c_array).ok()); + ASSERT_OK(writer->AddBatch(&c_array)); + ASSERT_OK(writer->Flush()); + ASSERT_OK(writer->Finish()); + ASSERT_OK(out->Flush()); + ASSERT_OK(out->Close()); + + // read + ASSERT_OK_AND_ASSIGN(auto reader_builder, file_format_->CreateReaderBuilder(1024)); + ASSERT_OK_AND_ASSIGN(std::shared_ptr in, + fs_->Open(PathUtil::JoinPath(dir_->Str(), "file.avro"))); + ASSERT_OK_AND_ASSIGN(auto batch_reader, reader_builder->Build(in)); + ASSERT_TRUE(arrow::ExportSchema(*schema, &c_schema).ok()); + ASSERT_OK(batch_reader->SetReadSchema(&c_schema, /*predicate=*/nullptr, + /*selection_bitmap=*/std::nullopt)); + ASSERT_OK_AND_ASSIGN(auto output_array, ::paimon::test::ReadResultCollector::CollectResult( + batch_reader.get())); + ASSERT_TRUE(output_array->Equals(arrow::ChunkedArray(input_array))) + << output_array->ToString() << "\n vs \n" + << input_array->ToString(); + } + + private: + std::shared_ptr file_format_; + std::shared_ptr fs_; + std::unique_ptr dir_; +}; INSTANTIATE_TEST_SUITE_P(Compression, AvroFileFormatTest, ::testing::ValuesIn(std::vector( {"zstd", "zstandard", "snappy", "null", "deflate"}))); TEST_P(AvroFileFormatTest, TestSimple) { - ASSERT_OK_AND_ASSIGN(std::unique_ptr file_format, - FileFormatFactory::Get("avro", {})); - arrow::FieldVector fields = { arrow::field("f0", arrow::boolean()), arrow::field("f1", arrow::int8()), arrow::field("f2", arrow::int16()), arrow::field("f3", arrow::int32()), arrow::field("field_null", arrow::int32()), arrow::field("f4", arrow::int64()), arrow::field("f5", arrow::float32()), arrow::field("f6", arrow::float64()), arrow::field("f7", arrow::utf8()), arrow::field("f8", arrow::binary())}; - auto schema = arrow::schema(fields); - auto data_type = arrow::struct_(fields); - - ::ArrowSchema c_schema; - ASSERT_TRUE(arrow::ExportSchema(*schema, &c_schema).ok()); - ASSERT_OK_AND_ASSIGN(auto writer_builder, file_format->CreateWriterBuilder(&c_schema, 1024)); - std::shared_ptr fs = std::make_shared(); - auto dir = ::paimon::test::UniqueTestDirectory::Create(); - ASSERT_TRUE(dir); - ASSERT_OK_AND_ASSIGN( - std::shared_ptr out, - fs->Create(PathUtil::JoinPath(dir->Str(), "file.avro"), /*overwrite=*/false)); - - auto compression = GetParam(); - ASSERT_OK_AND_ASSIGN(auto writer, writer_builder->Build(out, compression)); - std::string data_str = R"([[true, 0, 32767, 2147483647, null, 4294967295, 0.5, 1.141592659, "20250327", "banana"], [false, 1, 32767, null, null, 4294967296, 1.0, 2.141592658, "20250327", "dog"], [null, 1, 32767, 2147483647, null, null, 2.0, 3.141592657, null, "lucy"], [true, -2, -32768, -2147483648, null, -4294967298, 2.0, 3.141592657, "20250326", "mouse"]])"; + WriteDataAndCheckResult(fields, data_str); +} + +TEST_P(AvroFileFormatTest, TestComplexTypes) { + auto timezone = DateTimeUtils::GetLocalTimezoneName(); + arrow::FieldVector fields = { + arrow::field("f0", arrow::timestamp(arrow::TimeUnit::SECOND)), + arrow::field("f1", arrow::timestamp(arrow::TimeUnit::MILLI)), + arrow::field("f2", arrow::timestamp(arrow::TimeUnit::MICRO)), + arrow::field("f3", arrow::timestamp(arrow::TimeUnit::NANO)), + arrow::field("f4", arrow::timestamp(arrow::TimeUnit::SECOND, timezone)), + arrow::field("f5", arrow::timestamp(arrow::TimeUnit::MILLI, timezone)), + arrow::field("f6", arrow::timestamp(arrow::TimeUnit::MICRO, timezone)), + arrow::field("f7", arrow::timestamp(arrow::TimeUnit::NANO, timezone)), + arrow::field("f8", arrow::decimal128(30, 5)), + arrow::field("f9", arrow::decimal128(2, 2)), + arrow::field("f10", arrow::date32()), + arrow::field("f11", arrow::list(arrow::int32())), + arrow::field("f12", arrow::map(arrow::utf8(), arrow::utf8())), + arrow::field("f13", arrow::map(arrow::int32(), arrow::utf8())), + arrow::field("f14", arrow::map(arrow::struct_({arrow::field("f0", arrow::int32())}), + arrow::map(arrow::int32(), arrow::utf8()))), + arrow::field("f15", arrow::struct_({arrow::field("sub1", arrow::int64()), + arrow::field("sub2", arrow::float64()), + arrow::field("sub3", arrow::boolean())})), + }; + std::string src_data_str = R"([ + ["1970-01-02 00:00:00", "1970-01-02 00:00:00.001", "1970-01-02 00:00:00.000001", "1970-01-02 00:00:00.000000001", + "1970-01-02 00:00:00", "1970-01-02 00:00:00.001", "1970-01-02 00:00:00.000001", "1970-01-02 00:00:00.000000001", + "-123456789987654321.45678", "0.78", 12345, [1,2,3], [["test1","a"],["test2","value-3"]], [[1001,"a"],[1002,"value-3"]], + [[[1001],[[1,"a"],[2,"b"]]],[[1002],[[11,"aa"],[22,"bb"]]]], [10,11.2,false]] + ])"; + WriteDataAndCheckResult(fields, src_data_str); +} + +TEST_P(AvroFileFormatTest, TestNestedMap) { + auto timezone = DateTimeUtils::GetLocalTimezoneName(); + arrow::FieldVector fields = { + arrow::field("f0", arrow::map(arrow::utf8(), arrow::int16())), + arrow::field("f1", arrow::map(arrow::utf8(), arrow::utf8())), + arrow::field("f2", arrow::map(arrow::utf8(), arrow::timestamp(arrow::TimeUnit::MICRO))), + arrow::field( + "f3", arrow::map(arrow::utf8(), arrow::list(arrow::timestamp(arrow::TimeUnit::MICRO)))), + arrow::field("f4", arrow::map(arrow::utf8(), arrow::list(arrow::utf8()))), + arrow::field( + "f5", + arrow::map( + arrow::int32(), + arrow::struct_( + {arrow::field( + "f5.a", + arrow::struct_( + {arrow::field("f5.a.0", arrow::utf8()), + arrow::field("f5.sub2", arrow::int32()), + arrow::field("f5.a.1", arrow::timestamp(arrow::TimeUnit::MICRO))})), + arrow::field("f5.b", arrow::list(arrow::utf8())), + arrow::field("f5.c", arrow::map(arrow::utf8(), arrow::int32()))}))), + arrow::field( + "f6", arrow::map(arrow::utf8(), arrow::map(arrow::utf8(), arrow::list(arrow::utf8())))), + arrow::field("f7", arrow::map(arrow::int32(), arrow::boolean())), + arrow::field("f8", arrow::map(arrow::int64(), arrow::decimal128(2, 2))), + arrow::field("f9", arrow::map(arrow::date32(), arrow::float32())), + arrow::field("f10", arrow::map(arrow::binary(), arrow::float64())), + arrow::field("f11", arrow::map(arrow::int32(), arrow::list(arrow::int64()))), + arrow::field( + "f12", + arrow::map(arrow::utf8(), + arrow::list(arrow::struct_( + {arrow::field("name", arrow::utf8()), + arrow::field("scores", arrow::list(arrow::float32())), + arrow::field("info", arrow::map(arrow::float32(), arrow::utf8()))}))))}; - auto input_array = - arrow::ipc::internal::json::ArrayFromJSON(data_type, data_str).ValueOr(nullptr); - ASSERT_TRUE(input_array); - ::ArrowArray c_array; - ASSERT_TRUE(arrow::ExportArray(*input_array, &c_array).ok()); - ASSERT_OK(writer->AddBatch(&c_array)); - ASSERT_OK(writer->Flush()); - ASSERT_OK(writer->Finish()); - ASSERT_OK(out->Flush()); - ASSERT_OK(out->Close()); - - // read - ASSERT_OK_AND_ASSIGN(auto reader_builder, file_format->CreateReaderBuilder(1024)); - ASSERT_OK_AND_ASSIGN(std::shared_ptr in, - fs->Open(PathUtil::JoinPath(dir->Str(), "file.avro"))); - ASSERT_OK_AND_ASSIGN(auto batch_reader, reader_builder->Build(in)); - ASSERT_OK_AND_ASSIGN(auto chunked_array, - ::paimon::test::ReadResultCollector::CollectResult(batch_reader.get())); - - auto arrow_pool = GetArrowPool(GetDefaultPool()); - - auto output_array = arrow::Concatenate(chunked_array->chunks()).ValueOr(nullptr); - ASSERT_TRUE(output_array); - ASSERT_OK_AND_ASSIGN(output_array, ManifestMetaReader::AlignArrayWithSchema( - output_array, data_type, arrow_pool.get())); - ASSERT_TRUE(output_array->Equals(input_array)); + std::string src_data_str = R"([ + [ + [["f0_key_0", 1],["f0_key_1", 2]], + [["f1_key_0","val-1"],["f1_key_1","value-2"]], + [["f2_key_0","1970-01-01 00:00:00.000001"],["f2_key_1","1970-01-02 00:00:00.000001"]], + [["f3_key_0",["1970-01-01 00:00:00.000001"]],["f3_key_1",["1970-01-02 00:00:00.000001"]]], + [["f4_key_0",["val-1", "val-2"]],["f4_key_1",["val-3","val-4"]]], + [[500,[["sub1",1,"1970-01-01 00:00:00.000001"],["test-1","test-2"],[["subkey_0",1],["subkey_1", 2]]]],[600,[["sub2",2,"1970-01-02 00:00:00.000001"],["test-3","test-4"],[["subkey_2", 1],["subkey_3", 2]]]]], + [["f6_key_0",[["f6_sub_key_0",["value-0","value-1"]],["f6_sub_key_1",["value-2","value-3"]]]],["f6_key_1",[["f6_sub_key_2",["value-0","value-1"]],["f6_sub_key_3",["value-2","value-3"]]]]], + [[100, true], [200, false]], + [[1000, "0.78"], [2000, "0.91"]], + [[10, 1.5], [20, 2.75]], + [["aGVsbG8=", 3.14159], ["d29ybGQ=", 2.71828]], + [[1, [1000000, 2000000]], [2, [3000000, 4000000, 5000000]]], + [["group1", [["Alice", [95.5, 96.0], [[100.1, "info"]]], ["Bob", [88.0, 89.5],[[200.1,"info"]]]]],["group2", [["Charlie",[92.3, 93.1],[[300.1,"info"]]]]]] + ], + [ + null,null,null,null,null,null,null,null,null,null,null,null,null + ] + ])"; + WriteDataAndCheckResult(fields, src_data_str); } } // namespace paimon::avro::test diff --git a/src/paimon/format/avro/avro_format_writer.cpp b/src/paimon/format/avro/avro_format_writer.cpp index b19f3881..f9c69b68 100644 --- a/src/paimon/format/avro/avro_format_writer.cpp +++ b/src/paimon/format/avro/avro_format_writer.cpp @@ -20,20 +20,18 @@ #include #include #include -#include #include "arrow/api.h" #include "arrow/c/bridge.h" #include "avro/Compiler.hh" // IWYU pragma: keep #include "avro/DataFile.hh" #include "avro/Exception.hh" -#include "avro/Generic.hh" // IWYU pragma: keep -#include "avro/GenericDatum.hh" +#include "avro/Generic.hh" // IWYU pragma: keep #include "avro/Specific.hh" // IWYU pragma: keep #include "avro/ValidSchema.hh" #include "fmt/format.h" +#include "paimon/common/metrics/metrics_impl.h" #include "paimon/common/utils/arrow/status_utils.h" -#include "paimon/format/avro/avro_adaptor.h" #include "paimon/format/avro/avro_schema_converter.h" namespace arrow { @@ -43,14 +41,14 @@ struct ArrowArray; namespace paimon::avro { -AvroFormatWriter::AvroFormatWriter( - const std::shared_ptr<::avro::DataFileWriter<::avro::GenericDatum>>& file_writer, - const ::avro::ValidSchema& avro_schema, const std::shared_ptr& data_type, - std::unique_ptr adaptor, AvroOutputStreamImpl* avro_output_stream) - : writer_(file_writer), +AvroFormatWriter::AvroFormatWriter(std::unique_ptr<::avro::DataFileWriterBase>&& file_writer, + const ::avro::ValidSchema& avro_schema, + const std::shared_ptr& data_type, + AvroOutputStreamImpl* avro_output_stream) + : writer_(std::move(file_writer)), avro_schema_(avro_schema), data_type_(data_type), - adaptor_(std::move(adaptor)), + metrics_(std::make_shared()), avro_output_stream_(avro_output_stream) {} Result> AvroFormatWriter::Create( @@ -60,12 +58,11 @@ Result> AvroFormatWriter::Create( PAIMON_ASSIGN_OR_RAISE(::avro::ValidSchema avro_schema, AvroSchemaConverter::ArrowSchemaToAvroSchema(schema)); AvroOutputStreamImpl* avro_output_stream = out.get(); - auto writer = std::make_shared<::avro::DataFileWriter<::avro::GenericDatum>>( - std::move(out), avro_schema, DEFAULT_SYNC_INTERVAL, codec); + auto writer = std::make_unique<::avro::DataFileWriterBase>(std::move(out), avro_schema, + DEFAULT_SYNC_INTERVAL, codec); auto data_type = arrow::struct_(schema->fields()); - auto adaptor = std::make_unique(data_type); - return std::unique_ptr(new AvroFormatWriter( - writer, avro_schema, data_type, std::move(adaptor), avro_output_stream)); + return std::unique_ptr( + new AvroFormatWriter(std::move(writer), avro_schema, data_type, avro_output_stream)); } catch (const ::avro::Exception& e) { return Status::Invalid(fmt::format("avro format writer create failed. {}", e.what())); } catch (const std::exception& e) { @@ -104,18 +101,23 @@ Status AvroFormatWriter::Finish() { } Result AvroFormatWriter::ReachTargetSize(bool suggested_check, int64_t target_size) const { - return Status::NotImplemented("not support yet"); + if (suggested_check) { + uint64_t current_size = writer_->getCurrentBlockStart(); + return current_size >= static_cast(target_size); + } + return false; } Status AvroFormatWriter::AddBatch(ArrowArray* batch) { assert(batch); PAIMON_ASSIGN_OR_RAISE_FROM_ARROW(std::shared_ptr arrow_array, arrow::ImportArray(batch, data_type_)); - PAIMON_ASSIGN_OR_RAISE(std::vector<::avro::GenericDatum> datums, - adaptor_->ConvertArrayToGenericDatums(arrow_array, avro_schema_)); try { - for (const auto& datum : datums) { - writer_->write(datum); + for (int64_t row_index = 0; row_index < arrow_array->length(); ++row_index) { + writer_->syncIfNeeded(); + PAIMON_RETURN_NOT_OK(AvroDirectEncoder::EncodeArrowToAvro( + avro_schema_.root(), *arrow_array, row_index, &writer_->encoder(), &encode_ctx_)); + writer_->incr(); } } catch (const ::avro::Exception& e) { return Status::Invalid(fmt::format("avro writer add batch failed. {}", e.what())); diff --git a/src/paimon/format/avro/avro_format_writer.h b/src/paimon/format/avro/avro_format_writer.h index 17f39a8b..e4b49548 100644 --- a/src/paimon/format/avro/avro_format_writer.h +++ b/src/paimon/format/avro/avro_format_writer.h @@ -23,7 +23,7 @@ #include "arrow/api.h" #include "avro/DataFile.hh" #include "avro/ValidSchema.hh" -#include "paimon/format/avro/avro_adaptor.h" +#include "paimon/format/avro/avro_direct_encoder.h" #include "paimon/format/avro/avro_output_stream_impl.h" #include "paimon/format/format_writer.h" #include "paimon/metrics.h" @@ -64,19 +64,20 @@ class AvroFormatWriter : public FormatWriter { } private: - static constexpr size_t DEFAULT_SYNC_INTERVAL = 16 * 1024; + static constexpr size_t DEFAULT_SYNC_INTERVAL = 64 * 1024; - AvroFormatWriter( - const std::shared_ptr<::avro::DataFileWriter<::avro::GenericDatum>>& file_writer, - const ::avro::ValidSchema& avro_schema, const std::shared_ptr& data_type, - std::unique_ptr adaptor, AvroOutputStreamImpl* avro_output_stream); + AvroFormatWriter(std::unique_ptr<::avro::DataFileWriterBase>&& file_writer, + const ::avro::ValidSchema& avro_schema, + const std::shared_ptr& data_type, + AvroOutputStreamImpl* avro_output_stream); - std::shared_ptr<::avro::DataFileWriter<::avro::GenericDatum>> writer_; + std::unique_ptr<::avro::DataFileWriterBase> writer_; ::avro::ValidSchema avro_schema_; std::shared_ptr data_type_; std::shared_ptr metrics_; - std::unique_ptr adaptor_; AvroOutputStreamImpl* avro_output_stream_; + // Encode context for reusing scratch buffers + AvroDirectEncoder::EncodeContext encode_ctx_; }; } // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_format_writer_test.cpp b/src/paimon/format/avro/avro_format_writer_test.cpp new file mode 100644 index 00000000..ec0b5651 --- /dev/null +++ b/src/paimon/format/avro/avro_format_writer_test.cpp @@ -0,0 +1,253 @@ +/* + * 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/format/avro/avro_format_writer.h" + +#include +#include +#include +#include + +#include "arrow/api.h" +#include "arrow/c/abi.h" +#include "arrow/c/bridge.h" +#include "arrow/io/file.h" +#include "arrow/memory_pool.h" +#include "gtest/gtest.h" +#include "paimon/common/utils/arrow/mem_utils.h" +#include "paimon/common/utils/path_util.h" +#include "paimon/format/avro/avro_file_batch_reader.h" +#include "paimon/format/file_format.h" +#include "paimon/format/file_format_factory.h" +#include "paimon/fs/file_system.h" +#include "paimon/fs/local/local_file_system.h" +#include "paimon/memory/memory_pool.h" +#include "paimon/metrics.h" +#include "paimon/record_batch.h" +#include "paimon/testing/utils/read_result_collector.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::avro::test { + +class AvroFormatWriterTest : public ::testing::Test { + public: + void SetUp() override { + dir_ = paimon::test::UniqueTestDirectory::Create(); + ASSERT_TRUE(dir_); + fs_ = std::make_shared(); + pool_ = GetDefaultPool(); + arrow_pool_ = GetArrowPool(pool_); + } + void TearDown() override {} + + std::pair, std::shared_ptr> PrepareArrowSchema() + const { + auto string_field = arrow::field("col1", arrow::utf8()); + auto int_field = arrow::field("col2", arrow::int32()); + auto bool_field = arrow::field("col3", arrow::boolean()); + auto struct_type = arrow::struct_({string_field, int_field, bool_field}); + return std::make_pair( + arrow::schema(arrow::FieldVector({string_field, int_field, bool_field})), struct_type); + } + + std::shared_ptr CreateFormatWriter(const std::shared_ptr& schema, + const std::shared_ptr& out, + int32_t batch_size) { + ::ArrowSchema c_schema; + EXPECT_TRUE(arrow::ExportSchema(*schema, &c_schema).ok()); + EXPECT_OK_AND_ASSIGN(auto file_format, FileFormatFactory::Get("avro", {})); + EXPECT_OK_AND_ASSIGN(auto writer_builder, + file_format->CreateWriterBuilder(&c_schema, batch_size)); + EXPECT_OK_AND_ASSIGN(std::shared_ptr writer, + writer_builder->Build(out, "zstd")); + return writer; + } + + std::shared_ptr PrepareArray(const std::shared_ptr& data_type, + int32_t record_batch_size, + int32_t offset = 0) const { + arrow::StructBuilder struct_builder( + data_type, arrow::default_memory_pool(), + {std::make_shared(), std::make_shared(), + std::make_shared()}); + auto string_builder = static_cast(struct_builder.field_builder(0)); + auto int_builder = static_cast(struct_builder.field_builder(1)); + auto bool_builder = static_cast(struct_builder.field_builder(2)); + for (int32_t i = 0 + offset; i < record_batch_size + offset; ++i) { + EXPECT_TRUE(struct_builder.Append().ok()); + EXPECT_TRUE(string_builder->Append("str_" + std::to_string(i)).ok()); + if (i % 3 == 0) { + // test null + EXPECT_TRUE(int_builder->AppendNull().ok()); + } else { + EXPECT_TRUE(int_builder->Append(i).ok()); + } + EXPECT_TRUE(bool_builder->Append(static_cast(i % 2)).ok()); + } + std::shared_ptr array; + EXPECT_TRUE(struct_builder.Finish(&array).ok()); + return array; + } + + void AddRecordBatchOnce(const std::shared_ptr& format_writer, + const std::shared_ptr& struct_type, + int32_t record_batch_size, int32_t offset) const { + auto array = PrepareArray(struct_type, record_batch_size, offset); + auto arrow_array = std::make_unique(); + ASSERT_TRUE(arrow::ExportArray(*array, arrow_array.get()).ok()); + auto batch = std::make_shared( + /*partition=*/std::map(), /*bucket=*/-1, + /*row_kinds=*/std::vector(), arrow_array.get()); + ASSERT_OK(format_writer->AddBatch(batch->GetData())); + } + + void CheckResult(const std::string& file_path, int32_t row_count) const { + ASSERT_OK_AND_ASSIGN(std::shared_ptr input_stream, fs_->Open(file_path)); + ASSERT_OK_AND_ASSIGN(auto file_reader, + AvroFileBatchReader::Create(input_stream, 1024, pool_)); + ASSERT_OK_AND_ASSIGN(uint64_t num_rows, file_reader->GetNumberOfRows()); + ASSERT_EQ(num_rows, row_count); + + ASSERT_OK_AND_ASSIGN(auto result_array, + ::paimon::test::ReadResultCollector::CollectResult(file_reader.get())); + const auto& struct_array = + std::static_pointer_cast(result_array->chunk(0)); + const auto& string_array = + std::static_pointer_cast(struct_array->field(0)); + ASSERT_TRUE(string_array); + const auto& int_array = std::static_pointer_cast(struct_array->field(1)); + ASSERT_TRUE(int_array); + const auto& bool_array = + std::static_pointer_cast(struct_array->field(2)); + ASSERT_TRUE(bool_array); + ASSERT_EQ(string_array->null_count(), 0); + ASSERT_EQ(int_array->null_count(), (row_count - 1) / 3 + 1); + ASSERT_EQ(bool_array->null_count(), 0); + + for (int32_t i = 0; i < row_count; i++) { + ASSERT_EQ("str_" + std::to_string(i), string_array->GetString(i)); + if (i % 3 == 0) { + ASSERT_TRUE(int_array->IsNull(i)); + } else { + ASSERT_FALSE(int_array->IsNull(i)); + ASSERT_EQ(i, int_array->Value(i)); + } + if (i % 2 == 0) { + ASSERT_EQ(false, bool_array->Value(i)); + } else { + ASSERT_EQ(true, bool_array->Value(i)); + } + } + } + + private: + std::unique_ptr dir_; + std::shared_ptr fs_; + std::shared_ptr pool_; + std::shared_ptr arrow_pool_; +}; + +TEST_F(AvroFormatWriterTest, TestWriteWithVariousBatchSize) { + auto schema_pair = PrepareArrowSchema(); + const auto& arrow_schema = schema_pair.first; + const auto& struct_type = schema_pair.second; + std::map options; + for (auto record_batch_size : {1, 2, 3, 5, 20}) { + for (auto batch_capacity : {1, 2, 3, 5, 20}) { + std::string file_name = + std::to_string(record_batch_size) + "_" + std::to_string(batch_capacity); + std::string file_path = PathUtil::JoinPath(dir_->Str(), file_name); + ASSERT_OK_AND_ASSIGN(std::shared_ptr out, + fs_->Create(file_path, /*overwrite=*/false)); + auto format_writer = CreateFormatWriter(arrow_schema, out, batch_capacity); + auto array = PrepareArray(struct_type, record_batch_size); + auto arrow_array = std::make_unique(); + ASSERT_TRUE(arrow::ExportArray(*array, arrow_array.get()).ok()); + + auto batch = std::make_shared( + /*partition=*/std::map(), /*bucket=*/-1, + /*row_kinds=*/std::vector(), arrow_array.get()); + ASSERT_OK(format_writer->AddBatch(batch->GetData())); + ASSERT_OK(format_writer->Flush()); + ASSERT_OK(format_writer->Finish()); + ASSERT_OK(out->Flush()); + ASSERT_OK(out->Close()); + CheckResult(file_path, record_batch_size); + } + } +} + +TEST_F(AvroFormatWriterTest, TestWriteMultipleTimes) { + // arrow array length = 6 + 10 + 15 + 6 = 37 + // avro batch capacity = 10 + auto schema_pair = PrepareArrowSchema(); + const auto& arrow_schema = schema_pair.first; + const auto& struct_type = schema_pair.second; + + std::string file_path = PathUtil::JoinPath(dir_->Str(), "write_multiple_times"); + ASSERT_OK_AND_ASSIGN(std::shared_ptr out, + fs_->Create(file_path, /*overwrite=*/false)); + auto format_writer = CreateFormatWriter(arrow_schema, out, /*batch_size=*/10); + + // add batch first time, 6 rows + AddRecordBatchOnce(format_writer, struct_type, 6, 0); + // add batch second times, 10 rows + AddRecordBatchOnce(format_writer, struct_type, 10, 6); + // add batch third times, 15 rows (expand internal batch) + AddRecordBatchOnce(format_writer, struct_type, 15, 16); + // add batch fourth times, 6 rows + AddRecordBatchOnce(format_writer, struct_type, 6, 31); + + ASSERT_OK(format_writer->Flush()); + ASSERT_OK(format_writer->Finish()); + ASSERT_OK(out->Flush()); + ASSERT_OK(out->Close()); + CheckResult(file_path, /*row_count=*/37); + auto metrics = format_writer->GetWriterMetrics(); + ASSERT_TRUE(metrics); +} + +TEST_F(AvroFormatWriterTest, TestGetEstimateLength) { + auto schema_pair = PrepareArrowSchema(); + const auto& arrow_schema = schema_pair.first; + const auto& struct_type = schema_pair.second; + + std::string file_path = PathUtil::JoinPath(dir_->Str(), "get_estimate_length"); + ASSERT_OK_AND_ASSIGN(std::shared_ptr out, + fs_->Create(file_path, /*overwrite=*/false)); + auto format_writer = CreateFormatWriter(arrow_schema, out, /*batch_size=*/1024); + + // add batch first time, 1 row + AddRecordBatchOnce(format_writer, struct_type, 1, 0); + ASSERT_OK_AND_ASSIGN(bool reach_targe_size, + format_writer->ReachTargetSize(/*suggested_check=*/true, + /*target_size=*/102400)); + ASSERT_FALSE(reach_targe_size); + + // add batch second times, 9998 rows + AddRecordBatchOnce(format_writer, struct_type, 9998, 1); + ASSERT_OK_AND_ASSIGN(reach_targe_size, format_writer->ReachTargetSize(/*suggested_check=*/true, + /*target_size=*/102400)); + ASSERT_FALSE(reach_targe_size); + + AddRecordBatchOnce(format_writer, struct_type, 100000, 9999); + ASSERT_OK_AND_ASSIGN(reach_targe_size, format_writer->ReachTargetSize(/*suggested_check=*/true, + /*target_size=*/102400)); + ASSERT_TRUE(reach_targe_size); + ASSERT_OK(format_writer->Finish()); +} + +} // namespace paimon::avro::test diff --git a/src/paimon/format/avro/avro_output_stream_impl.cpp b/src/paimon/format/avro/avro_output_stream_impl.cpp index 91e6c741..e3c1be16 100644 --- a/src/paimon/format/avro/avro_output_stream_impl.cpp +++ b/src/paimon/format/avro/avro_output_stream_impl.cpp @@ -71,7 +71,10 @@ void AvroOutputStreamImpl::FlushBuffer() { fmt::format("write failed, expected length: {}, actual write length: {}", length, write_len.value())); } - // TODO(jinli.zjw): call out_->Flush() ? + Status status = out_->Flush(); + if (!status.ok()) { + throw std::runtime_error("flush failed, status: " + status.ToString()); + } next_ = buffer_; available_ = buffer_size_; } diff --git a/src/paimon/format/avro/avro_stats_extractor.cpp b/src/paimon/format/avro/avro_stats_extractor.cpp new file mode 100644 index 00000000..6989979e --- /dev/null +++ b/src/paimon/format/avro/avro_stats_extractor.cpp @@ -0,0 +1,125 @@ +/* + * Copyright 2026-present Alibaba Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "paimon/format/avro/avro_stats_extractor.h" + +#include +#include +#include + +#include "arrow/api.h" +#include "arrow/c/abi.h" +#include "arrow/c/bridge.h" +#include "arrow/util/checked_cast.h" +#include "paimon/common/utils/arrow/status_utils.h" +#include "paimon/common/utils/date_time_utils.h" +#include "paimon/core/core_options.h" +#include "paimon/defs.h" +#include "paimon/format/avro/avro_file_format.h" +#include "paimon/status.h" + +namespace paimon { +class FileSystem; +class MemoryPool; +} // namespace paimon + +namespace paimon::avro { + +Result> +AvroStatsExtractor::ExtractWithFileInfoInternal(const std::shared_ptr& file_system, + const std::string& path, + const std::shared_ptr& pool, + bool with_file_info) const { + PAIMON_ASSIGN_OR_RAISE(CoreOptions core_options, CoreOptions::FromMap(options_)); + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr input_stream, file_system->Open(path)); + assert(input_stream); + auto avro_file_format = std::make_unique(options_); + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr avro_reader_builder, + avro_file_format->CreateReaderBuilder(core_options.GetReadBatchSize())); + PAIMON_ASSIGN_OR_RAISE( + std::unique_ptr avro_reader, + avro_reader_builder->WithMemoryPool(pool)->Build(std::move(input_stream))); + + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr<::ArrowSchema> c_schema, avro_reader->GetFileSchema()); + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW(std::shared_ptr arrow_schema, + arrow::ImportSchema(c_schema.get())); + ColumnStatsVector result_stats; + result_stats.reserve(arrow_schema->num_fields()); + for (const auto& arrow_field : arrow_schema->fields()) { + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr stats, + FetchColumnStatistics(arrow_field->type())); + result_stats.push_back(std::move(stats)); + } + if (!with_file_info) { + // Do not return file info if not needed, because GetNumberOfRows in avro reader need I/O + // and performance is poor. + return std::make_pair(result_stats, FileInfo(-1)); + } + PAIMON_ASSIGN_OR_RAISE(uint64_t num_rows, avro_reader->GetNumberOfRows()); + return std::make_pair(result_stats, FileInfo(num_rows)); +} + +Result> AvroStatsExtractor::FetchColumnStatistics( + const std::shared_ptr& type) const { + // TODO(jinli.zjw): support stats in avro + arrow::Type::type kind = type->id(); + switch (kind) { + case arrow::Type::type::BOOL: + return ColumnStats::CreateBooleanColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::INT8: + return ColumnStats::CreateTinyIntColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::INT16: + return ColumnStats::CreateSmallIntColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::INT32: + return ColumnStats::CreateIntColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::INT64: + return ColumnStats::CreateBigIntColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::FLOAT: + return ColumnStats::CreateFloatColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::DOUBLE: + return ColumnStats::CreateDoubleColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::BINARY: + return ColumnStats::CreateStringColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::STRING: + return ColumnStats::CreateStringColumnStats(std::nullopt, std::nullopt, std::nullopt); + + case arrow::Type::type::DATE32: + return ColumnStats::CreateDateColumnStats(std::nullopt, std::nullopt, std::nullopt); + case arrow::Type::type::TIMESTAMP: { + auto ts_type = arrow::internal::checked_pointer_cast<::arrow::TimestampType>(type); + int32_t precision = DateTimeUtils::GetPrecisionFromType(ts_type); + return ColumnStats::CreateTimestampColumnStats(std::nullopt, std::nullopt, std::nullopt, + precision); + } + case arrow::Type::type::DECIMAL128: { + auto decimal_type = + arrow::internal::checked_pointer_cast<::arrow::Decimal128Type>(type); + int32_t precision = decimal_type->precision(); + int32_t scale = decimal_type->scale(); + return ColumnStats::CreateDecimalColumnStats(std::nullopt, std::nullopt, std::nullopt, + precision, scale); + } + case arrow::Type::type::STRUCT: + return ColumnStats::CreateNestedColumnStats(FieldType::STRUCT, std::nullopt); + case arrow::Type::type::LIST: + return ColumnStats::CreateNestedColumnStats(FieldType::ARRAY, std::nullopt); + case arrow::Type::type::MAP: + return ColumnStats::CreateNestedColumnStats(FieldType::MAP, std::nullopt); + default: + return Status::Invalid("Unknown or unsupported arrow type: ", type->ToString()); + } +} +} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_stats_extractor.h b/src/paimon/format/avro/avro_stats_extractor.h new file mode 100644 index 00000000..18c49fb0 --- /dev/null +++ b/src/paimon/format/avro/avro_stats_extractor.h @@ -0,0 +1,71 @@ +/* + * Copyright 2026-present Alibaba Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include +#include + +#include "arrow/api.h" +#include "paimon/format/column_stats.h" +#include "paimon/format/format_stats_extractor.h" +#include "paimon/result.h" +#include "paimon/type_fwd.h" + +namespace arrow { +class DataType; +} // namespace arrow +namespace paimon { +class FileSystem; +class MemoryPool; +} // namespace paimon + +namespace paimon::avro { + +class AvroStatsExtractor : public FormatStatsExtractor { + public: + explicit AvroStatsExtractor(const std::map& options) + : options_(options) {} + + Result Extract(const std::shared_ptr& file_system, + const std::string& path, + const std::shared_ptr& pool) override { + PAIMON_ASSIGN_OR_RAISE(auto result, ExtractWithFileInfoInternal(file_system, path, pool, + /*with_file_info=*/false)); + return result.first; + } + + Result> ExtractWithFileInfo( + const std::shared_ptr& file_system, const std::string& path, + const std::shared_ptr& pool) override { + return ExtractWithFileInfoInternal(file_system, path, pool, /*with_file_info=*/true); + } + + private: + Result> ExtractWithFileInfoInternal( + const std::shared_ptr& file_system, const std::string& path, + const std::shared_ptr& pool, bool with_file_info) const; + + Result> FetchColumnStatistics( + const std::shared_ptr& type) const; + + private: + std::map options_; +}; + +} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_stats_extractor_test.cpp b/src/paimon/format/avro/avro_stats_extractor_test.cpp new file mode 100644 index 00000000..06197017 --- /dev/null +++ b/src/paimon/format/avro/avro_stats_extractor_test.cpp @@ -0,0 +1,218 @@ +/* + * 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/format/avro/avro_stats_extractor.h" + +#include + +#include "arrow/api.h" +#include "arrow/c/abi.h" +#include "arrow/c/bridge.h" +#include "arrow/ipc/json_simple.h" +#include "gtest/gtest.h" +#include "paimon/common/data/binary_row.h" +#include "paimon/common/utils/date_time_utils.h" +#include "paimon/core/stats/simple_stats.h" +#include "paimon/core/stats/simple_stats_converter.h" +#include "paimon/format/avro/avro_file_format.h" +#include "paimon/format/avro/avro_format_writer.h" +#include "paimon/format/file_format_factory.h" +#include "paimon/fs/file_system.h" +#include "paimon/fs/local/local_file_system.h" +#include "paimon/memory/memory_pool.h" +#include "paimon/status.h" +#include "paimon/testing/utils/testharness.h" + +namespace paimon::avro::test { + +class AvroStatsExtractorTest : public ::testing::Test { + public: + void SetUp() override {} + void TearDown() override {} + + void WriteAvroFile(const std::string& file_path, + const std::shared_ptr& src_chunk_array, + const std::shared_ptr& schema) const { + ::ArrowSchema c_schema; + ASSERT_TRUE(arrow::ExportSchema(*schema, &c_schema).ok()); + + ASSERT_OK_AND_ASSIGN(std::unique_ptr file_format, + FileFormatFactory::Get("avro", options_)); + ASSERT_OK_AND_ASSIGN(auto writer_builder, + file_format->CreateWriterBuilder(&c_schema, /*batch_size=*/1024)); + + auto fs = std::make_shared(); + ASSERT_OK_AND_ASSIGN(std::unique_ptr output_stream, + fs->Create(file_path, true)); + ASSERT_OK_AND_ASSIGN(auto writer, writer_builder->Build(std::move(output_stream), "null")); + + for (const auto& array : src_chunk_array->chunks()) { + ::ArrowArray c_array; + ASSERT_TRUE(arrow::ExportArray(*array, &c_array).ok()); + ASSERT_OK(writer->AddBatch(&c_array)); + } + ASSERT_OK(writer->Flush()); + ASSERT_OK(writer->Finish()); + + ASSERT_OK_AND_ASSIGN(auto file_status, fs->GetFileStatus(file_path)); + ASSERT_GT(file_status->GetLen(), 0); + } + + private: + std::map options_ = {{Options::FILE_FORMAT, "avro"}, + {Options::MANIFEST_FORMAT, "avro"}}; +}; + +TEST_F(AvroStatsExtractorTest, TestPrimitiveStatsExtractor) { + auto timezone = DateTimeUtils::GetLocalTimezoneName(); + arrow::FieldVector fields = { + arrow::field("f1", arrow::int8()), + arrow::field("f2", arrow::int16()), + arrow::field("f3", arrow::int32()), + arrow::field("f4", arrow::int64()), + arrow::field("f5", arrow::float32()), + arrow::field("f6", arrow::float64()), + arrow::field("f7", arrow::utf8()), + arrow::field("f8", arrow::binary()), + arrow::field("f9", arrow::date32()), + arrow::field("f10", arrow::timestamp(arrow::TimeUnit::NANO)), + arrow::field("f11", arrow::decimal128(5, 2)), + arrow::field("f12", arrow::boolean()), + arrow::field("f13", arrow::timestamp(arrow::TimeUnit::SECOND)), + arrow::field("f14", arrow::timestamp(arrow::TimeUnit::MILLI)), + arrow::field("f15", arrow::timestamp(arrow::TimeUnit::MICRO)), + arrow::field("f16", arrow::timestamp(arrow::TimeUnit::NANO)), + arrow::field("f17", arrow::timestamp(arrow::TimeUnit::SECOND, timezone)), + arrow::field("f18", arrow::timestamp(arrow::TimeUnit::MILLI, timezone)), + arrow::field("f19", arrow::timestamp(arrow::TimeUnit::MICRO, timezone)), + arrow::field("f20", arrow::timestamp(arrow::TimeUnit::NANO, timezone)), + }; + auto schema = std::make_shared(fields); + auto array = std::dynamic_pointer_cast( + arrow::ipc::internal::json::ArrayFromJSON(arrow::struct_({fields}), R"([ + [1, 11, 111, 1111, 1.1, 1.11, "Hello", "你好", 1234, "2033-05-18 03:33:20.0", "1.22", true, "2033-05-18 03:33:20", "2033-05-18 03:33:20.0", "2033-05-18 03:33:20.0", "2033-05-18 03:33:20.0", "2033-05-18 03:33:20", "2033-05-18 03:33:20.0", "2033-05-18 03:33:20.0", "2033-05-18 03:33:20.0"], + [2, 22, 222, 2222, 2.2, 2.22, "World", "世界", -1234, "1899-01-01 00:59:20.001001001", "2.22", false, "1899-01-01 00:59:20", "1899-01-01 00:59:20", "1899-01-01 00:59:20", "1899-01-01 00:59:20.001001001","1899-01-01 00:59:20", "1899-01-01 00:59:20", "1899-01-01 00:59:20", "1899-01-01 00:59:20.001001001"], + [null, null, 0, null, null, 0, null, null, null, null, null, null, null, null, null, null, null, null, null, null] + ])") + .ValueOrDie()); + auto src_chunk_array = std::make_shared(arrow::ArrayVector({array})); + + auto dir = paimon::test::UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + std::string file_path = dir->Str() + "/test.avro"; + WriteAvroFile(file_path, src_chunk_array, schema); + + AvroFileFormat format(options_); + ::ArrowSchema arrow_schema; + ASSERT_TRUE(arrow::ExportSchema(*schema, &arrow_schema).ok()); + ASSERT_OK_AND_ASSIGN(auto extractor, format.CreateStatsExtractor(&arrow_schema)); + auto fs = std::make_shared(); + ASSERT_OK_AND_ASSIGN(auto stats_with_info, + extractor->ExtractWithFileInfo(fs, file_path, GetDefaultPool())); + const auto& column_stats = stats_with_info.first; + const auto& file_stats = stats_with_info.second; + + ASSERT_EQ(column_stats.size(), 20); + for (const auto& stats : column_stats) { + ASSERT_EQ(stats->ToString(), "min null, max null, null count null"); + } + ASSERT_EQ(3, file_stats.GetRowCount()); +} + +TEST_F(AvroStatsExtractorTest, TestNestedType) { + arrow::FieldVector fields = { + arrow::field("f0", arrow::list(arrow::float32())), + arrow::field("f1", arrow::struct_({arrow::field("sub_f0", arrow::boolean()), + arrow::field("sub_f1", arrow::int64())}))}; + auto schema = arrow::schema(fields); + auto array = std::dynamic_pointer_cast( + arrow::ipc::internal::json::ArrayFromJSON(arrow::struct_({fields}), R"([ + [null, [true, 2]], + [[0.1, 0.3], [true, 1]], + [[1.1, 1.2], null] + ])") + .ValueOrDie()); + auto src_chunk_array = std::make_shared(arrow::ArrayVector({array})); + + auto dir = paimon::test::UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + std::string file_path = dir->Str() + "/test.avro"; + WriteAvroFile(file_path, src_chunk_array, schema); + + AvroStatsExtractor extractor(options_); + auto fs = std::make_shared(); + ASSERT_OK_AND_ASSIGN(auto results, extractor.Extract(fs, file_path, GetDefaultPool())); + + ASSERT_EQ(results.size(), 2); + for (const auto& stats : results) { + ASSERT_EQ(stats->ToString(), "min null, max null, null count null"); + } +} + +TEST_F(AvroStatsExtractorTest, TestNullForAllType) { + auto timezone = DateTimeUtils::GetLocalTimezoneName(); + arrow::FieldVector fields = { + arrow::field("f0", arrow::boolean()), + arrow::field("f1", arrow::int8()), + arrow::field("f2", arrow::int16()), + arrow::field("f3", arrow::int32()), + arrow::field("f4", arrow::int64()), + arrow::field("f5", arrow::float32()), + arrow::field("f6", arrow::float64()), + arrow::field("f7", arrow::utf8()), + arrow::field("f8", arrow::binary()), + arrow::field("f9", arrow::list(arrow::struct_({arrow::field("key", arrow::int8()), + arrow::field("value", arrow::int16())}))), + arrow::field("f10", arrow::list(arrow::float32())), + arrow::field("f11", arrow::struct_({arrow::field("f0", arrow::boolean()), + arrow::field("f1", arrow::int64())})), + arrow::field("f12", arrow::timestamp(arrow::TimeUnit::NANO)), + arrow::field("f13", arrow::date32()), + arrow::field("f14", arrow::decimal128(2, 2)), + arrow::field("f15", arrow::decimal128(30, 2)), + arrow::field("f16", arrow::timestamp(arrow::TimeUnit::SECOND)), + arrow::field("f17", arrow::timestamp(arrow::TimeUnit::MILLI)), + arrow::field("f18", arrow::timestamp(arrow::TimeUnit::MICRO)), + arrow::field("f19", arrow::timestamp(arrow::TimeUnit::NANO)), + arrow::field("f20", arrow::timestamp(arrow::TimeUnit::SECOND, timezone)), + arrow::field("f21", arrow::timestamp(arrow::TimeUnit::MILLI, timezone)), + arrow::field("f22", arrow::timestamp(arrow::TimeUnit::MICRO, timezone)), + arrow::field("f23", arrow::timestamp(arrow::TimeUnit::NANO, timezone)), + }; + auto schema = std::make_shared(fields); + auto src_array = std::dynamic_pointer_cast( + arrow::ipc::internal::json::ArrayFromJSON(arrow::struct_({fields}), R"([ + [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null] + ])") + .ValueOrDie()); + auto src_chunk_array = std::make_shared(arrow::ArrayVector({src_array})); + + auto dir = paimon::test::UniqueTestDirectory::Create(); + ASSERT_TRUE(dir); + std::string file_path = dir->Str() + "/test.avro"; + WriteAvroFile(file_path, src_chunk_array, schema); + + AvroStatsExtractor extractor(options_); + auto fs = std::make_shared(); + ASSERT_OK_AND_ASSIGN(auto column_stats, extractor.Extract(fs, file_path, GetDefaultPool())); + + ASSERT_OK_AND_ASSIGN(auto stats, + SimpleStatsConverter::ToBinary(column_stats, GetDefaultPool().get())); + ASSERT_EQ(stats.min_values_.HashCode(), 0xf890741a); + ASSERT_EQ(stats.max_values_.HashCode(), 0xf890741a); +} + +} // namespace paimon::avro::test diff --git a/src/paimon/format/blob/blob_file_batch_reader.cpp b/src/paimon/format/blob/blob_file_batch_reader.cpp index b1e8b672..d47e4173 100644 --- a/src/paimon/format/blob/blob_file_batch_reader.cpp +++ b/src/paimon/format/blob/blob_file_batch_reader.cpp @@ -133,13 +133,13 @@ Status BlobFileBatchReader::SetReadSchema(::ArrowSchema* read_schema, std::vector new_offsets(cardinality); std::vector new_row_indexes(cardinality); + PAIMON_ASSIGN_OR_RAISE(uint64_t total_rows, GetNumberOfRows()); RoaringBitmap32::Iterator iterator(*selection_bitmap); for (int32_t i = 0; i < cardinality; i++) { int32_t row_index = *iterator; - if (static_cast(row_index) >= GetNumberOfRows()) { - return Status::Invalid( - fmt::format("row index {} is out of bound of total row number {}", row_index, - GetNumberOfRows())); + if (static_cast(row_index) >= total_rows) { + return Status::Invalid(fmt::format( + "row index {} is out of bound of total row number {}", row_index, total_rows)); } ++iterator; new_lengths[i] = all_blob_lengths_[row_index]; @@ -231,7 +231,7 @@ Result BlobFileBatchReader::NextBatch() { return Status::Invalid("blob file batch reader is closed"); } if (current_pos_ >= target_blob_lengths_.size()) { - previous_batch_first_row_number_ = GetNumberOfRows(); + PAIMON_ASSIGN_OR_RAISE(previous_batch_first_row_number_, GetNumberOfRows()); return BatchReader::MakeEofBatch(); } int32_t left_rows = target_blob_lengths_.size() - current_pos_; diff --git a/src/paimon/format/blob/blob_file_batch_reader.h b/src/paimon/format/blob/blob_file_batch_reader.h index 29ecf68e..d2a76673 100644 --- a/src/paimon/format/blob/blob_file_batch_reader.h +++ b/src/paimon/format/blob/blob_file_batch_reader.h @@ -102,7 +102,7 @@ class BlobFileBatchReader : public FileBatchReader { return previous_batch_first_row_number_; } - uint64_t GetNumberOfRows() const override { + Result GetNumberOfRows() const override { return all_blob_lengths_.size(); } diff --git a/src/paimon/format/blob/blob_file_batch_reader_test.cpp b/src/paimon/format/blob/blob_file_batch_reader_test.cpp index d827c83e..c4376a92 100644 --- a/src/paimon/format/blob/blob_file_batch_reader_test.cpp +++ b/src/paimon/format/blob/blob_file_batch_reader_test.cpp @@ -167,7 +167,8 @@ TEST_F(BlobFileBatchReaderTest, TestRowNumbers) { /*batch_size=*/1, /*blob_as_descriptor=*/true, pool_)); ASSERT_OK(reader->SetReadSchema(&c_schema, nullptr, std::nullopt)); - ASSERT_EQ(3, reader->GetNumberOfRows()); + ASSERT_OK_AND_ASSIGN(auto number_of_rows, reader->GetNumberOfRows()); + ASSERT_EQ(3, number_of_rows); ASSERT_EQ(std::numeric_limits::max(), reader->GetPreviousBatchFirstRowNumber()); ASSERT_OK_AND_ASSIGN(auto batch1, reader->NextBatch()); ArrowArrayRelease(batch1.first.get()); @@ -206,7 +207,8 @@ TEST_F(BlobFileBatchReaderTest, TestRowNumbersWithBitmap) { RoaringBitmap32 roaring; roaring.Add(1); ASSERT_OK(reader->SetReadSchema(&c_schema, nullptr, roaring)); - ASSERT_EQ(3, reader->GetNumberOfRows()); + ASSERT_OK_AND_ASSIGN(auto number_of_rows, reader->GetNumberOfRows()); + ASSERT_EQ(3, number_of_rows); ASSERT_EQ(std::numeric_limits::max(), reader->GetPreviousBatchFirstRowNumber()); ASSERT_OK_AND_ASSIGN(auto batch1, reader->NextBatch()); ASSERT_EQ(1, reader->GetPreviousBatchFirstRowNumber()); @@ -283,7 +285,8 @@ TEST_P(BlobFileBatchReaderTest, EmptyFile) { /*batch_size=*/1, /*blob_as_descriptor=*/true, pool_)); ASSERT_OK(reader->SetReadSchema(&c_schema, nullptr, std::nullopt)); - ASSERT_EQ(0, reader->GetNumberOfRows()); + ASSERT_OK_AND_ASSIGN(auto number_of_rows, reader->GetNumberOfRows()); + ASSERT_EQ(0, number_of_rows); ASSERT_EQ(std::numeric_limits::max(), reader->GetPreviousBatchFirstRowNumber()); ASSERT_OK_AND_ASSIGN(auto batch, reader->NextBatch()); ASSERT_TRUE(BatchReader::IsEofBatch(batch)); diff --git a/src/paimon/format/blob/blob_stats_extractor.cpp b/src/paimon/format/blob/blob_stats_extractor.cpp index 3eba40f2..11057664 100644 --- a/src/paimon/format/blob/blob_stats_extractor.cpp +++ b/src/paimon/format/blob/blob_stats_extractor.cpp @@ -55,7 +55,8 @@ BlobStatsExtractor::ExtractWithFileInfo(const std::shared_ptr& file_ ColumnStatsVector result_stats; result_stats.push_back( ColumnStats::CreateStringColumnStats(std::nullopt, std::nullopt, /*null_count=*/0)); - return std::make_pair(result_stats, FileInfo(blob_reader->GetNumberOfRows())); + PAIMON_ASSIGN_OR_RAISE(uint64_t num_rows, blob_reader->GetNumberOfRows()); + return std::make_pair(result_stats, FileInfo(num_rows)); } } // namespace paimon::blob diff --git a/src/paimon/format/lance/lance_file_batch_reader.h b/src/paimon/format/lance/lance_file_batch_reader.h index ab8019ac..903c72af 100644 --- a/src/paimon/format/lance/lance_file_batch_reader.h +++ b/src/paimon/format/lance/lance_file_batch_reader.h @@ -47,7 +47,7 @@ class LanceFileBatchReader : public FileBatchReader { return -1; } - uint64_t GetNumberOfRows() const override { + Result GetNumberOfRows() const override { return num_rows_; } diff --git a/src/paimon/format/lance/lance_format_reader_writer_test.cpp b/src/paimon/format/lance/lance_format_reader_writer_test.cpp index 16603ec9..e76edc5a 100644 --- a/src/paimon/format/lance/lance_format_reader_writer_test.cpp +++ b/src/paimon/format/lance/lance_format_reader_writer_test.cpp @@ -130,7 +130,8 @@ class LanceFileReaderWriterTest : public ::testing::Test { ASSERT_OK_AND_ASSIGN(std::shared_ptr result, paimon::test::ReadResultCollector::CollectResult(reader.get())); ASSERT_TRUE(result->Equals(src_chunk_array)) << result->ToString(); - ASSERT_EQ(reader->GetNumberOfRows(), src_chunk_array->length()); + ASSERT_OK_AND_ASSIGN(uint64_t num_rows, reader->GetNumberOfRows()); + ASSERT_EQ(num_rows, src_chunk_array->length()); } } }; diff --git a/src/paimon/format/lance/lance_stats_extractor.cpp b/src/paimon/format/lance/lance_stats_extractor.cpp index 94004cda..c08ed4ac 100644 --- a/src/paimon/format/lance/lance_stats_extractor.cpp +++ b/src/paimon/format/lance/lance_stats_extractor.cpp @@ -59,7 +59,8 @@ LanceStatsExtractor::ExtractWithFileInfo(const std::shared_ptr& file FetchColumnStatistics(arrow_field->type())); result_stats.push_back(std::move(stats)); } - return std::make_pair(result_stats, FileInfo(lance_reader->GetNumberOfRows())); + PAIMON_ASSIGN_OR_RAISE(uint64_t num_rows, lance_reader->GetNumberOfRows()); + return std::make_pair(result_stats, FileInfo(num_rows)); } Result> LanceStatsExtractor::FetchColumnStatistics( diff --git a/src/paimon/format/orc/orc_file_batch_reader.h b/src/paimon/format/orc/orc_file_batch_reader.h index d5036813..ca4043b3 100644 --- a/src/paimon/format/orc/orc_file_batch_reader.h +++ b/src/paimon/format/orc/orc_file_batch_reader.h @@ -66,7 +66,7 @@ class OrcFileBatchReader : public PrefetchFileBatchReader { return reader_->GetRowNumber(); } - uint64_t GetNumberOfRows() const override { + Result GetNumberOfRows() const override { return reader_->GetNumberOfRows(); } @@ -78,7 +78,8 @@ class OrcFileBatchReader : public PrefetchFileBatchReader { Result>> GenReadRanges( bool* need_prefetch) const override { - return reader_->GenReadRanges(target_column_ids_, 0, GetNumberOfRows(), need_prefetch); + PAIMON_ASSIGN_OR_RAISE(uint64_t num_rows, GetNumberOfRows()); + return reader_->GenReadRanges(target_column_ids_, 0, num_rows, need_prefetch); } void Close() override { diff --git a/src/paimon/format/parquet/parquet_file_batch_reader.h b/src/paimon/format/parquet/parquet_file_batch_reader.h index f28ccb9c..6294eecd 100644 --- a/src/paimon/format/parquet/parquet_file_batch_reader.h +++ b/src/paimon/format/parquet/parquet_file_batch_reader.h @@ -89,7 +89,7 @@ class ParquetFileBatchReader : public PrefetchFileBatchReader { return reader_->GetPreviousBatchFirstRowNumber(); } - uint64_t GetNumberOfRows() const override { + Result GetNumberOfRows() const override { assert(reader_); return reader_->GetNumberOfRows(); } diff --git a/src/paimon/format/parquet/parquet_format_writer_test.cpp b/src/paimon/format/parquet/parquet_format_writer_test.cpp index de3d0098..2c3a39f3 100644 --- a/src/paimon/format/parquet/parquet_format_writer_test.cpp +++ b/src/paimon/format/parquet/parquet_format_writer_test.cpp @@ -286,7 +286,7 @@ TEST_F(ParquetFormatWriterTest, TestGetEstimateLength) { AddRecordBatchOnce(format_writer, struct_type, 100000, 9999); ASSERT_OK_AND_ASSIGN(uint64_t estimate_len3, format_writer->GetEstimateLength()); ASSERT_GT(estimate_len3, estimate_len2); - ASSERT_TRUE(format_writer->Finish().ok()); + ASSERT_OK(format_writer->Finish()); } TEST_F(ParquetFormatWriterTest, TestTimestampType) { diff --git a/src/paimon/testing/mock/mock_file_batch_reader.h b/src/paimon/testing/mock/mock_file_batch_reader.h index 398c0520..19a89a26 100644 --- a/src/paimon/testing/mock/mock_file_batch_reader.h +++ b/src/paimon/testing/mock/mock_file_batch_reader.h @@ -82,7 +82,7 @@ class MockFileBatchReader : public PrefetchFileBatchReader { Result>> GenReadRanges( bool* need_prefetch) const override { uint64_t begin_row_num = 0; - uint64_t end_row_num = GetNumberOfRows(); + PAIMON_ASSIGN_OR_RAISE(uint64_t end_row_num, GetNumberOfRows()); std::vector> read_ranges; for (uint64_t begin = begin_row_num; begin < end_row_num; begin += batch_size_) { uint64_t end = std::min(begin + batch_size_, end_row_num); @@ -145,7 +145,7 @@ class MockFileBatchReader : public PrefetchFileBatchReader { std::shared_ptr GetReaderMetrics() const override { auto metrics = std::make_shared(); - metrics->SetCounter("mock.number.of.rows", GetNumberOfRows()); + metrics->SetCounter("mock.number.of.rows", GetNumberOfRows().value_or(-1)); return metrics; } @@ -153,7 +153,7 @@ class MockFileBatchReader : public PrefetchFileBatchReader { return previous_batch_first_row_num_; } - uint64_t GetNumberOfRows() const override { + Result GetNumberOfRows() const override { return data_ ? data_->length() : 0; } uint64_t GetNextRowToRead() const override { diff --git a/test/inte/blob_table_inte_test.cpp b/test/inte/blob_table_inte_test.cpp index 727690a1..125c50e6 100644 --- a/test/inte/blob_table_inte_test.cpp +++ b/test/inte/blob_table_inte_test.cpp @@ -249,6 +249,9 @@ std::vector GetTestValuesForBlobTableInteTest() { #endif #ifdef PAIMON_ENABLE_LANCE values.emplace_back("lance"); +#endif +#ifdef PAIMON_ENABLE_AVRO + values.emplace_back("avro"); #endif return values; } @@ -963,8 +966,8 @@ TEST_P(BlobTableInteTest, TestPartitionWithPredicate) { } TEST_P(BlobTableInteTest, TestPredicate) { - if (GetParam() == "lance") { - // lance does not have stats + if (GetParam() == "lance" || GetParam() == "avro") { + // lance and avro do not have stats return; } CreateTable(); @@ -1108,7 +1111,7 @@ TEST_P(BlobTableInteTest, TestIOException) { TEST_P(BlobTableInteTest, TestReadTableWithDenseStats) { auto file_format = GetParam(); - if (file_format == "lance") { + if (file_format == "lance" || file_format == "avro") { return; } std::string table_path = @@ -1172,7 +1175,7 @@ TEST_P(BlobTableInteTest, TestReadTableWithDenseStats) { TEST_P(BlobTableInteTest, TestDataEvolutionAndAlterTable) { auto file_format = GetParam(); - if (file_format == "lance") { + if (file_format == "lance" || file_format == "avro") { return; } std::string table_path = paimon::test::GetDataDir() + file_format + diff --git a/test/inte/data_evolution_table_test.cpp b/test/inte/data_evolution_table_test.cpp index 6ac0f414..e650448c 100644 --- a/test/inte/data_evolution_table_test.cpp +++ b/test/inte/data_evolution_table_test.cpp @@ -1010,7 +1010,7 @@ TEST_P(DataEvolutionTableTest, TestWithPartitionWithoutPartitionFieldsInFile) { TEST_P(DataEvolutionTableTest, TestPartitionWithPredicate) { auto file_format = GetParam(); - if (file_format == "lance") { + if (file_format == "lance" || file_format == "avro") { return; } std::vector partition_keys = {"f1"}; @@ -1183,7 +1183,7 @@ TEST_P(DataEvolutionTableTest, TestPartitionWithPredicate) { TEST_P(DataEvolutionTableTest, TestAlterTable) { auto file_format = GetParam(); - if (file_format == "lance") { + if (file_format == "lance" || file_format == "avro") { return; } std::string table_path = paimon::test::GetDataDir() + file_format + @@ -1280,7 +1280,7 @@ TEST_P(DataEvolutionTableTest, TestAlterTable) { TEST_P(DataEvolutionTableTest, TestReadCompactFiles) { auto file_format = GetParam(); - if (file_format == "lance") { + if (file_format == "lance" || file_format == "avro") { return; } std::string table_path = @@ -1310,7 +1310,7 @@ TEST_P(DataEvolutionTableTest, TestReadCompactFiles) { TEST_P(DataEvolutionTableTest, TestReadTableWithDenseStats) { auto file_format = GetParam(); - if (file_format == "lance") { + if (file_format == "lance" || file_format == "avro") { return; } std::string table_path = paimon::test::GetDataDir() + file_format + @@ -1391,7 +1391,7 @@ TEST_P(DataEvolutionTableTest, TestReadTableWithDenseStats) { TEST_P(DataEvolutionTableTest, TestScanAndReadWithIndex) { auto file_format = GetParam(); - if (file_format == "lance") { + if (file_format == "lance" || file_format == "avro") { return; } // only f2 has index @@ -1531,8 +1531,8 @@ TEST_P(DataEvolutionTableTest, TestScanAndReadWithIndex) { } TEST_P(DataEvolutionTableTest, TestPredicate) { - if (GetParam() == "lance") { - // lance does not have stats + if (GetParam() == "lance" || GetParam() == "avro") { + // lance and avro do not have stats return; } CreateTable(); @@ -1837,8 +1837,8 @@ TEST_P(DataEvolutionTableTest, TestWithRowIds) { /*predicate=*/nullptr, /*row_ranges=*/row_ranges)); } - if (GetParam() == "lance") { - // as lance does not support stats + if (GetParam() == "lance" || GetParam() == "avro") { + // as lance and avro do not support stats return; } { @@ -1905,6 +1905,9 @@ std::vector GetTestValuesForDataEvolutionTableTest() { #endif #ifdef PAIMON_ENABLE_LANCE values.emplace_back("lance"); +#endif +#ifdef PAIMON_ENABLE_AVRO + values.emplace_back("avro"); #endif return values; } diff --git a/test/inte/global_index_test.cpp b/test/inte/global_index_test.cpp index e820a8fe..3eceb17a 100644 --- a/test/inte/global_index_test.cpp +++ b/test/inte/global_index_test.cpp @@ -434,7 +434,7 @@ TEST_P(GlobalIndexTest, TestWriteIndexWithPartition) { #endif TEST_P(GlobalIndexTest, TestScanIndex) { - if (file_format_ == "lance") { + if (file_format_ == "lance" || file_format_ == "avro") { return; } @@ -624,7 +624,7 @@ TEST_P(GlobalIndexTest, TestScanIndex) { } TEST_P(GlobalIndexTest, TestScanIndexWithSpecificSnapshot) { - if (file_format_ == "lance") { + if (file_format_ == "lance" || file_format_ == "avro") { return; } @@ -679,7 +679,7 @@ TEST_P(GlobalIndexTest, TestScanIndexWithSpecificSnapshot) { } TEST_P(GlobalIndexTest, TestScanIndexWithSpecificSnapshotWithNoIndex) { - if (file_format_ == "lance") { + if (file_format_ == "lance" || file_format_ == "avro") { return; } @@ -711,7 +711,7 @@ TEST_P(GlobalIndexTest, TestScanIndexWithSpecificSnapshotWithNoIndex) { } TEST_P(GlobalIndexTest, TestScanIndexWithRange) { - if (file_format_ == "lance") { + if (file_format_ == "lance" || file_format_ == "avro") { return; } @@ -772,7 +772,7 @@ TEST_P(GlobalIndexTest, TestScanIndexWithRange) { } TEST_P(GlobalIndexTest, TestScanIndexWithPartition) { - if (file_format_ == "lance") { + if (file_format_ == "lance" || file_format_ == "avro") { return; } @@ -836,7 +836,7 @@ TEST_P(GlobalIndexTest, TestScanIndexWithPartition) { } TEST_P(GlobalIndexTest, TestScanUnregisteredIndex) { - if (file_format_ == "lance") { + if (file_format_ == "lance" || file_format_ == "avro") { return; } auto factory_creator = FactoryCreator::GetInstance(); @@ -2191,6 +2191,10 @@ std::vector GetTestValuesForGlobalIndexTest() { #ifdef PAIMON_ENABLE_LANCE values.emplace_back("lance", false); values.emplace_back("lance", true); +#endif +#ifdef PAIMON_ENABLE_AVRO + values.emplace_back("avro", false); + values.emplace_back("avro", true); #endif return values; } diff --git a/test/inte/scan_and_read_inte_test.cpp b/test/inte/scan_and_read_inte_test.cpp index 8c38c28b..8e867acf 100644 --- a/test/inte/scan_and_read_inte_test.cpp +++ b/test/inte/scan_and_read_inte_test.cpp @@ -171,6 +171,19 @@ class ScanAndReadInteTest : public testing::Test, DataField(3, arrow::field("f3", arrow::float64()))})); }; +std::vector> GetTestValuesForScanAndReadInteTest() { + std::vector> values = {{"parquet", false}, {"parquet", true}}; +#ifdef PAIMON_ENABLE_ORC + values.emplace_back("orc", false); + values.emplace_back("orc", true); +#endif + return values; +} + +INSTANTIATE_TEST_SUITE_P(FileFormatAndEnablePaimonPrefetch, ScanAndReadInteTest, + ::testing::ValuesIn(std::vector>( + GetTestValuesForScanAndReadInteTest()))); + TEST_P(ScanAndReadInteTest, TestWithAppendSnapshotIOException) { auto [file_format, enable_prefetch] = GetParam(); std::string table_path = GetDataDir() + "/" + file_format + "/append_09.db/append_09"; @@ -546,30 +559,30 @@ TEST_P(ScanAndReadInteTest, TestJavaPaimon1WithAppendSnapshotOfNestedType) { } // test pk with dv -TEST_P(ScanAndReadInteTest, TestWithPKWithDvBatchScanSnapshot6) { - auto [file_format, enable_prefetch] = GetParam(); - std::string table_path = GetDataDir() + "/" + file_format + - "/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/"; +TEST_F(ScanAndReadInteTest, TestWithPKWithDvBatchScanSnapshot6) { + auto check_result = [&](const std::string& file_format) { + std::string table_path = GetDataDir() + "/" + file_format + + "/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/"; - // normal batch scan case for pk+dv, all data in level 0 is filtered out - ScanContextBuilder scan_context_builder(table_path); - scan_context_builder.AddOption(Options::SCAN_SNAPSHOT_ID, "6"); - ASSERT_OK_AND_ASSIGN(auto scan_context, scan_context_builder.Finish()); - ASSERT_OK_AND_ASSIGN(auto table_scan, TableScan::Create(std::move(scan_context))); + // normal batch scan case for pk+dv, all data in level 0 is filtered out + ScanContextBuilder scan_context_builder(table_path); + scan_context_builder.AddOption(Options::SCAN_SNAPSHOT_ID, "6"); + ASSERT_OK_AND_ASSIGN(auto scan_context, scan_context_builder.Finish()); + ASSERT_OK_AND_ASSIGN(auto table_scan, TableScan::Create(std::move(scan_context))); - ReadContextBuilder read_context_builder(table_path); - AddReadOptionsForPrefetch(&read_context_builder); - ASSERT_OK_AND_ASSIGN(auto read_context, read_context_builder.Finish()); - ASSERT_OK_AND_ASSIGN(auto table_read, TableRead::Create(std::move(read_context))); + ReadContextBuilder read_context_builder(table_path); + ASSERT_OK_AND_ASSIGN(auto read_context, read_context_builder.Finish()); + ASSERT_OK_AND_ASSIGN(auto table_read, TableRead::Create(std::move(read_context))); - ASSERT_OK_AND_ASSIGN(auto result_plan, table_scan->CreatePlan()); - ASSERT_EQ(result_plan->SnapshotId().value(), 6); - ASSERT_OK_AND_ASSIGN(auto batch_reader, table_read->CreateReader(result_plan->Splits())); - ASSERT_OK_AND_ASSIGN(auto read_result, ReadResultCollector::CollectResult(batch_reader.get())); + ASSERT_OK_AND_ASSIGN(auto result_plan, table_scan->CreatePlan()); + ASSERT_EQ(result_plan->SnapshotId().value(), 6); + ASSERT_OK_AND_ASSIGN(auto batch_reader, table_read->CreateReader(result_plan->Splits())); + ASSERT_OK_AND_ASSIGN(auto read_result, + ReadResultCollector::CollectResult(batch_reader.get())); - // check result - auto expected = std::make_shared( - arrow::ipc::internal::json::ArrayFromJSON(arrow_data_type_, R"([ + // check result + auto expected = std::make_shared( + arrow::ipc::internal::json::ArrayFromJSON(arrow_data_type_, R"([ [0, "Two roads diverged in a wood, and I took the one less traveled by, And that has made all the difference.", 10, 1, 11.0], [0, "Alice", 10, 1, 19.1], [0, "Alex", 10, 0, 16.1], @@ -579,9 +592,14 @@ TEST_P(ScanAndReadInteTest, TestWithPKWithDvBatchScanSnapshot6) { [0, "Lucy", 20, 1, 14.1], [0, "Paul", 20, 1, 18.1] ])") - .ValueOrDie()); - ASSERT_TRUE(expected); - ASSERT_TRUE(expected->Equals(read_result)) << read_result->ToString(); + .ValueOrDie()); + ASSERT_TRUE(expected); + ASSERT_TRUE(expected->Equals(read_result)) << read_result->ToString(); + }; + for (auto [file_format, enable_prefetch] : GetTestValuesForScanAndReadInteTest()) { + check_result(file_format); + } + check_result("avro"); } TEST_P(ScanAndReadInteTest, TestWithPKWithDvBatchScanSnapshot1) { @@ -2567,8 +2585,7 @@ TEST_P(ScanAndReadInteTest, TestCastTimestampType) { } #ifdef PAIMON_ENABLE_AVRO -// TODO(zjw): remove DISABLED_ when avro write is ready -TEST_F(ScanAndReadInteTest, DISABLED_TestAvroWithAppendTable) { +TEST_F(ScanAndReadInteTest, TestAvroWithAppendTable) { auto read_data = [](int64_t snapshot_id, const std::string& result_json) { std::string table_path = GetDataDir() + "/avro/append_multiple.db/append_multiple"; // scan @@ -2620,7 +2637,8 @@ TEST_F(ScanAndReadInteTest, DISABLED_TestAvroWithAppendTable) { arrow::ipc::internal::json::ArrayFromJSON(arrow::struct_(fields), result_json) .ValueOrDie()); ASSERT_TRUE(expected); - ASSERT_TRUE(expected->Equals(read_result)) << read_result->ToString(); + ASSERT_TRUE(expected->Equals(read_result)) + << "read_result: " << read_result->ToString() << "expected: " << expected->ToString(); }; read_data(1, R"([ @@ -2631,13 +2649,17 @@ TEST_F(ScanAndReadInteTest, DISABLED_TestAvroWithAppendTable) { ])"); read_data(2, R"([ -[0, 6, 10, 1, 100, 6.0, 4.0, "six", "fff", 123, "123.45", "1970-01-02 00:00:00", "1970-01-06 00:00:00.000", "1970-01-06 00:00:00.000000", "1970-01-06 00:00:00", "1970-01-06 00:00:00.000", "1970-01-06 00:00:00.000000",[[["key",123]],[1,2,3]]], -[0, 5, 10, 0, 100, 5.0, 2.0, null, "eee", 123, "123.45", "1970-01-01 00:00:00", "1970-01-05 00:00:00.000", "1970-01-05 00:00:00.000000", "1970-01-05 00:00:00", "1970-01-05 00:00:00.000", "1970-01-05 00:00:00.000000",[[["key",123]],[1,2,3]]], -[0, 7, 11, 0, 100, 7.0, 6.0, "seven", "ggg", 123, "123.45", "1970-01-03 00:00:00", "1970-01-07 00:00:00.000", "1970-01-07 00:00:00.000000", "1970-01-07 00:00:00", "1970-01-07 00:00:00.000", "1970-01-07 00:00:00.000000",[[["key",123]],[1,2,3]]] +[0, 2, 10, 1, 100, 2.0, 2.0, "two", "bbb", 123, "123.45", "1970-01-02 00:00:00", "1970-01-02 00:00:00.000", "1970-01-02 00:00:00.000000", "1970-01-02 00:00:00", "1970-01-02 00:00:00.000", "1970-01-02 00:00:00.000000",[[["key",123]],[1,2,3]]], +[0, 6, 10, 1, 100, 6.0, 4.0, "six", "fff", 123, "123.45", "1970-01-06 00:00:00", "1970-01-06 00:00:00.000", "1970-01-06 00:00:00.000000", "1970-01-06 00:00:00", "1970-01-06 00:00:00.000", "1970-01-06 00:00:00.000000",[[["key",123]],[1,2,3]]], +[0, 1, 10, 0, 100, 1.0, 1.0, "one", "aaa", 123, "123.45", "1970-01-01 00:00:00", "1970-01-01 00:00:00.000", "1970-01-01 00:00:00.000000", "1970-01-01 00:00:00", "1970-01-01 00:00:00.000", "1970-01-01 00:00:00.000000",[[["key",123]],[1,2,3]]], +[0, 5, 10, 0, 100, 5.0, 2.0, null, "eee", 123, "123.45", "1970-01-05 00:00:00", "1970-01-05 00:00:00.000", "1970-01-05 00:00:00.000000", "1970-01-05 00:00:00", "1970-01-05 00:00:00.000", "1970-01-05 00:00:00.000000",[[["key",123]],[1,2,3]]], +[0, 3, 11, 0, 100, null, 3.0, "three", "ccc", 123, "123.45", "1970-01-03 00:00:00", "1970-01-03 00:00:00.000", "1970-01-03 00:00:00.000000", "1970-01-03 00:00:00", "1970-01-03 00:00:00.000", "1970-01-03 00:00:00.000000",[[["key",123]],[1,2,3]]], +[0, 4, 11, 0, 100, 4.0, null, "four", "ddd", 123, "123.45", "1970-01-04 00:00:00", "1970-01-04 00:00:00.000", "1970-01-04 00:00:00.000000", "1970-01-04 00:00:00", "1970-01-04 00:00:00.000", "1970-01-04 00:00:00.000000",[[["key",123]],[1,2,3]]], +[0, 7, 11, 0, 100, 7.0, 6.0, "seven", "ggg", 123, "123.45", "1970-01-07 00:00:00", "1970-01-07 00:00:00.000", "1970-01-07 00:00:00.000000", "1970-01-07 00:00:00", "1970-01-07 00:00:00.000", "1970-01-07 00:00:00.000000",[[["key",123]],[1,2,3]]] ])"); } -TEST_F(ScanAndReadInteTest, DISABLED_TestAvroWithPkTable) { +TEST_F(ScanAndReadInteTest, TestAvroWithPkTable) { auto read_data = [](int64_t snapshot_id, const std::string& result_json) { std::string table_path = GetDataDir() + "/avro/pk_with_multiple_type.db/pk_with_multiple_type"; @@ -2701,17 +2723,4 @@ TEST_F(ScanAndReadInteTest, DISABLED_TestAvroWithPkTable) { } #endif -std::vector> GetTestValuesForScanAndReadInteTest() { - std::vector> values = {{"parquet", false}, {"parquet", true}}; -#ifdef PAIMON_ENABLE_ORC - values.emplace_back("orc", false); - values.emplace_back("orc", true); -#endif - return values; -} - -INSTANTIATE_TEST_SUITE_P(FileFormatAndEnablePaimonPrefetch, ScanAndReadInteTest, - ::testing::ValuesIn(std::vector>( - GetTestValuesForScanAndReadInteTest()))); - } // namespace paimon::test diff --git a/test/inte/write_and_read_inte_test.cpp b/test/inte/write_and_read_inte_test.cpp index a0d81739..e860d10d 100644 --- a/test/inte/write_and_read_inte_test.cpp +++ b/test/inte/write_and_read_inte_test.cpp @@ -75,7 +75,7 @@ TEST_P(WriteAndReadInteTest, TestAppendSimple) { auto [file_format, file_system] = GetParam(); // manifest and file format are upper case std::map options = { - {Options::MANIFEST_FORMAT, "ORC"}, + {Options::MANIFEST_FORMAT, "AVRO"}, {Options::FILE_FORMAT, StringUtils::ToUpperCase(file_format)}, {Options::TARGET_FILE_SIZE, "1024"}, {Options::BUCKET, "-1"}, @@ -126,7 +126,7 @@ TEST_P(WriteAndReadInteTest, TestPKSimple) { auto schema = arrow::schema(fields); auto [file_format, file_system] = GetParam(); std::map options = { - {Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, file_format}, + {Options::MANIFEST_FORMAT, "avro"}, {Options::FILE_FORMAT, file_format}, {Options::TARGET_FILE_SIZE, "1024"}, {Options::BUCKET, "1"}, {Options::FILE_SYSTEM, file_system}, {"orc.read.enable-lazy-decoding", "true"}, {"orc.dictionary-key-size-threshold", "1"}, @@ -192,7 +192,7 @@ TEST_P(WriteAndReadInteTest, TestNestedType) { auto schema = arrow::schema(fields); auto [file_format, file_system] = GetParam(); std::map options = { - {Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, file_format}, + {Options::MANIFEST_FORMAT, "avro"}, {Options::FILE_FORMAT, file_format}, {Options::TARGET_FILE_SIZE, "1024"}, {Options::BUCKET, "-1"}, {Options::FILE_SYSTEM, file_system}, }; @@ -440,7 +440,7 @@ TEST_P(WriteAndReadInteTest, TestAppendTimestampType) { auto schema = arrow::schema(fields); auto [file_format, file_system] = GetParam(); std::map options = { - {Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, file_format}, + {Options::MANIFEST_FORMAT, "avro"}, {Options::FILE_FORMAT, file_format}, {Options::TARGET_FILE_SIZE, "1024"}, {Options::BUCKET, "-1"}, {Options::FILE_SYSTEM, file_system}, {"orc.timestamp-ltz.legacy.type", "false"}}; if (file_system == "jindo") { @@ -493,7 +493,7 @@ TEST_P(WriteAndReadInteTest, TestPkTimestampType) { auto schema = arrow::schema(fields); auto [file_format, file_system] = GetParam(); std::map options = { - {Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, file_format}, + {Options::MANIFEST_FORMAT, "avro"}, {Options::FILE_FORMAT, file_format}, {Options::TARGET_FILE_SIZE, "1024"}, {Options::BUCKET, "1"}, {Options::FILE_SYSTEM, file_system}, {"orc.timestamp-ltz.legacy.type", "false"}}; if (file_system == "jindo") { @@ -543,7 +543,7 @@ TEST_P(WriteAndReadInteTest, TestPKWithSequenceFieldInPKField) { auto schema = arrow::schema(fields); auto [file_format, file_system] = GetParam(); std::map options = { - {Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, file_format}, + {Options::MANIFEST_FORMAT, "avro"}, {Options::FILE_FORMAT, file_format}, {Options::TARGET_FILE_SIZE, "1024"}, {Options::BUCKET, "1"}, {Options::FILE_SYSTEM, file_system}, {Options::SEQUENCE_FIELD, "p2"}, {"orc.read.enable-lazy-decoding", "true"}, {"orc.dictionary-key-size-threshold", "1"}, @@ -608,7 +608,7 @@ TEST_P(WriteAndReadInteTest, TestPKWithSequenceFieldPartialInPKField) { auto schema = arrow::schema(fields); auto [file_format, file_system] = GetParam(); std::map options = { - {Options::MANIFEST_FORMAT, "orc"}, {Options::FILE_FORMAT, file_format}, + {Options::MANIFEST_FORMAT, "avro"}, {Options::FILE_FORMAT, file_format}, {Options::TARGET_FILE_SIZE, "1024"}, {Options::BUCKET, "1"}, {Options::FILE_SYSTEM, file_system}, {Options::SEQUENCE_FIELD, "p2,f1"}, {"orc.read.enable-lazy-decoding", "true"}, {"orc.dictionary-key-size-threshold", "1"}, @@ -671,6 +671,9 @@ std::vector> GetTestValuesForWriteAndReadInt #endif #ifdef PAIMON_ENABLE_LANCE values.emplace_back("lance", "local"); +#endif +#ifdef PAIMON_ENABLE_AVRO + values.emplace_back("avro", "local"); #endif return values; } diff --git a/test/inte/write_inte_test.cpp b/test/inte/write_inte_test.cpp index 1914d461..22e0cd27 100644 --- a/test/inte/write_inte_test.cpp +++ b/test/inte/write_inte_test.cpp @@ -196,10 +196,10 @@ class WriteInteTest : public testing::Test, public ::testing::WithParamInterface std::shared_ptr ReconstructDataFileMeta( const std::shared_ptr& file_meta) const { - if (GetParam() != "lance") { + if (GetParam() != "lance" && GetParam() != "avro") { return file_meta; } - // for lance format, all stats is null + // for lance and avro format, all stats is null auto new_meta = std::make_shared( file_meta->file_name, file_meta->file_size, file_meta->row_count, file_meta->min_key, file_meta->max_key, file_meta->key_stats, file_meta->value_stats, @@ -261,6 +261,9 @@ std::vector GetTestValuesForWriteInteTest() { #endif #ifdef PAIMON_ENABLE_LANCE values.emplace_back("lance"); +#endif +#ifdef PAIMON_ENABLE_AVRO + values.emplace_back("avro"); #endif return values; } @@ -688,7 +691,7 @@ TEST_P(WriteInteTest, TestAppendTableWriteWithComplexType) { arrow::field("f2", arrow::list(arrow::float32())), arrow::field("f3", arrow::struct_({arrow::field("f0", arrow::boolean()), arrow::field("f1", arrow::int64())})), - arrow::field("f4", arrow::timestamp(arrow::TimeUnit::NANO)), + arrow::field("f4", arrow::timestamp(arrow::TimeUnit::MILLI)), arrow::field("f5", arrow::date32()), arrow::field("f6", arrow::decimal128(2, 2))}; auto schema = arrow::schema(fields); @@ -705,12 +708,12 @@ TEST_P(WriteInteTest, TestAppendTableWriteWithComplexType) { int64_t commit_identifier = 0; std::string data_1 = R"([ - [[[0, 0]], [0.1, 0.2], [true, 2], "1970-01-01 00:02:03.123123", 2456, "0.22"], - [[[0, 1]], [0.1, 0.3], [true, 1], "1970-01-01 00:02:03.999999", 24, "0.28"], - [[[10, 10]], [1.1, 1.2], [false, 12], "1970-01-01 00:02:03.123123", 2456, "0.22"], - [[[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123123", 245, "0.12"], + [[[0, 0]], [0.1, 0.2], [true, 2], "1970-01-01 00:02:03.123", 2456, "0.22"], + [[[0, 1]], [0.1, 0.3], [true, 1], "1970-01-01 00:02:03.999", 24, "0.28"], + [[[10, 10]], [1.1, 1.2], [false, 12], "1970-01-01 00:02:03.123", 2456, "0.22"], + [[[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123", 245, "0.12"], [[[1, 64], [2, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.0", 24, "0.78"], - [[[11, 64], [12, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.123123", 24, "0.78"] + [[[11, 64], [12, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.123", 24, "0.78"] ])"; ASSERT_OK_AND_ASSIGN(std::unique_ptr batch_1, TestHelper::MakeRecordBatch( @@ -719,43 +722,14 @@ TEST_P(WriteInteTest, TestAppendTableWriteWithComplexType) { RecordBatch::RowKind::INSERT, RecordBatch::RowKind::INSERT, RecordBatch::RowKind::INSERT, RecordBatch::RowKind::INSERT})); - SimpleStats value_stats = SimpleStats::EmptyStats(); - if (file_format != "parquet") { - value_stats = BinaryRowGenerator::GenerateStats( - {NullType(), NullType(), NullType(), TimestampType(Timestamp(0, 0), 9), - static_cast(24), Decimal(2, 2, DecimalUtils::StrToInt128("12").value())}, - {NullType(), NullType(), NullType(), TimestampType(Timestamp(123999, 999000), 9), - static_cast(2456), Decimal(2, 2, DecimalUtils::StrToInt128("78").value())}, - std::vector({0, 0, 0, 0, 0, 0}), pool_.get()); - } else { - BinaryRow min_row(6); - BinaryRowWriter min_writer(&min_row, 0, pool_.get()); - min_writer.Reset(); - min_writer.SetNullAt(0); - min_writer.SetNullAt(1); - min_writer.SetNullAt(2); - min_writer.WriteTimestamp(3, std::nullopt, Timestamp::MAX_PRECISION); - min_writer.WriteInt(4, 24); - Decimal decimal(2, 2, DecimalUtils::StrToInt128("12").value()); - min_writer.WriteDecimal(5, decimal, 2); - min_writer.Complete(); - - BinaryRow max_row(6); - BinaryRowWriter max_writer(&max_row, 0, pool_.get()); - max_writer.Reset(); - max_writer.SetNullAt(0); - max_writer.SetNullAt(1); - max_writer.SetNullAt(2); - max_writer.WriteTimestamp(3, std::nullopt, Timestamp::MAX_PRECISION); - max_writer.WriteInt(4, 2456); - Decimal decimal1(2, 2, DecimalUtils::StrToInt128("78").value()); - max_writer.WriteDecimal(5, decimal1, 2); - max_writer.Complete(); - - auto null_counts = - BinaryRowGenerator::FromLongArrayWithNull({-1, -1, -1, -1, 0, 0}, pool_.get()); - value_stats = SimpleStats(min_row, max_row, null_counts); - } + SimpleStats value_stats = BinaryRowGenerator::GenerateStats( + {NullType(), NullType(), NullType(), TimestampType(Timestamp(0, 0), 3), + static_cast(24), Decimal(2, 2, DecimalUtils::StrToInt128("12").value())}, + {NullType(), NullType(), NullType(), TimestampType(Timestamp(123999, 000000), 3), + static_cast(2456), Decimal(2, 2, DecimalUtils::StrToInt128("78").value())}, + file_format == "parquet" ? std::vector({-1, -1, -1, 0, 0, 0}) + : std::vector({0, 0, 0, 0, 0, 0}), + pool_.get()); auto file_meta = std::make_shared( "data-xxx.xxx", /*file_size=*/543, @@ -795,12 +769,12 @@ TEST_P(WriteInteTest, TestAppendTableWriteWithComplexType) { helper->NewScan(StartupMode::LatestFull(), /*snapshot_id=*/std::nullopt)); ASSERT_EQ(data_splits_1.size(), 1); std::string expected_data_1 = R"([ - [0, [[0, 0]], [0.1, 0.2], [true, 2], "1970-01-01 00:02:03.123123", 2456, "0.22"], - [0, [[0, 1]], [0.1, 0.3], [true, 1], "1970-01-01 00:02:03.999999", 24, "0.28"], - [0, [[10, 10]], [1.1, 1.2], [false, 12], "1970-01-01 00:02:03.123123", 2456, "0.22"], - [0, [[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123123", 245, "0.12"], + [0, [[0, 0]], [0.1, 0.2], [true, 2], "1970-01-01 00:02:03.123", 2456, "0.22"], + [0, [[0, 1]], [0.1, 0.3], [true, 1], "1970-01-01 00:02:03.999", 24, "0.28"], + [0, [[10, 10]], [1.1, 1.2], [false, 12], "1970-01-01 00:02:03.123", 2456, "0.22"], + [0, [[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123", 245, "0.12"], [0, [[1, 64], [2, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.0", 24, "0.78"], - [0, [[11, 64], [12, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.123123", 24, "0.78"] + [0, [[11, 64], [12, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.123", 24, "0.78"] ])"; ASSERT_OK_AND_ASSIGN(bool success, @@ -808,10 +782,10 @@ TEST_P(WriteInteTest, TestAppendTableWriteWithComplexType) { ASSERT_TRUE(success); std::string data_2 = R"([ - [[[10, 11]], [1.1, 1.3], [true, 1], "1970-01-01 00:02:03.999999", 24, "0.28"], - [[[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123123", 245, "0.12"], + [[[10, 11]], [1.1, 1.3], [true, 1], "1970-01-01 00:02:03.999", 24, "0.28"], + [[[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123", 245, "0.12"], [[[10, 20], [20, 30]], [2.0, 3.0], [true, 2], "1970-01-01 00:00:00.0", 24, "0.78"], - [[[11, 61], [21, 31]], [2.1, 3.1], [false, 1], "1970-01-01 00:00:00.123123", 24, "0.78"] + [[[11, 61], [21, 31]], [2.1, 3.1], [false, 1], "1970-01-01 00:00:00.123", 24, "0.78"] ])"; ASSERT_OK_AND_ASSIGN(std::unique_ptr batch_2, TestHelper::MakeRecordBatch( @@ -833,6 +807,7 @@ TEST_P(WriteInteTest, TestAppendTableWriteWithComplexType) { /*value_stats_cols=*/std::nullopt, /*external_path=*/std::nullopt, /*first_row_id=*/std::nullopt, /*write_cols=*/std::nullopt); + file_meta_2 = ReconstructDataFileMeta(file_meta_2); DataIncrement data_increment_2({file_meta_2}, {}, {}); std::shared_ptr expected_commit_message_2 = std::make_shared( BinaryRow::EmptyRow(), /*bucket=*/0, @@ -852,10 +827,10 @@ TEST_P(WriteInteTest, TestAppendTableWriteWithComplexType) { ASSERT_OK_AND_ASSIGN(std::vector> data_splits_2, helper->Scan()); ASSERT_EQ(data_splits_2.size(), 1); std::string expected_data_2 = R"([ - [0, [[10, 11]], [1.1, 1.3], [true, 1], "1970-01-01 00:02:03.999999", 24, "0.28"], - [0, [[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123123", 245, "0.12"], + [0, [[10, 11]], [1.1, 1.3], [true, 1], "1970-01-01 00:02:03.999", 24, "0.28"], + [0, [[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123", 245, "0.12"], [0, [[10, 20], [20, 30]], [2.0, 3.0], [true, 2], "1970-01-01 00:00:00.0", 24, "0.78"], - [0, [[11, 61], [21, 31]], [2.1, 3.1], [false, 1], "1970-01-01 00:00:00.123123", 24, "0.78"] + [0, [[11, 61], [21, 31]], [2.1, 3.1], [false, 1], "1970-01-01 00:00:00.123", 24, "0.78"] ])"; ASSERT_OK_AND_ASSIGN(success, helper->ReadAndCheckResult(data_type, data_splits_2, expected_data_2)); @@ -1542,7 +1517,7 @@ TEST_P(WriteInteTest, TestPkTableWriteWithComplexType) { arrow::field("f2", arrow::list(arrow::float32())), arrow::field("f3", arrow::struct_({arrow::field("f0", arrow::boolean()), arrow::field("f1", arrow::int64())})), - arrow::field("f4", arrow::timestamp(arrow::TimeUnit::NANO)), + arrow::field("f4", arrow::timestamp(arrow::TimeUnit::MILLI)), arrow::field("f5", arrow::date32()), arrow::field("f6", arrow::decimal128(2, 2))}; auto schema = arrow::schema(fields); @@ -1559,12 +1534,12 @@ TEST_P(WriteInteTest, TestPkTableWriteWithComplexType) { /*is_streaming_mode=*/true)); int64_t commit_identifier = 0; std::string data_1 = R"([ - [[[0, 0]], [0.1, 0.2], [true, 2], "1970-01-01 00:02:03.123123", 2456, "0.22"], - [[[0, 1]], [0.1, 0.3], [true, 1], "1970-01-01 00:02:03.999999", 24, "0.28"], - [[[10, 10]], [1.1, 1.2], [false, 12], "1970-01-01 00:02:03.123123", 2456, "0.22"], - [[[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123123", 245, "0.12"], + [[[0, 0]], [0.1, 0.2], [true, 2], "1970-01-01 00:02:03.123", 2456, "0.22"], + [[[0, 1]], [0.1, 0.3], [true, 1], "1970-01-01 00:02:03.999", 24, "0.28"], + [[[10, 10]], [1.1, 1.2], [false, 12], "1970-01-01 00:02:03.123", 2456, "0.22"], + [[[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123", 245, "0.12"], [[[1, 64], [2, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.0", 24, "0.78"], - [[[11, 64], [12, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.123123", 24, "0.78"] + [[[11, 64], [12, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.123", 24, "0.78"] ])"; ASSERT_OK_AND_ASSIGN(std::unique_ptr batch_1, TestHelper::MakeRecordBatch( @@ -1575,80 +1550,26 @@ TEST_P(WriteInteTest, TestPkTableWriteWithComplexType) { auto min_key = BinaryRowGenerator::GenerateRow( {Decimal(2, 2, DecimalUtils::StrToInt128("12").value()), - TimestampType(Timestamp(123123, 123000), 9), static_cast(245)}, + TimestampType(Timestamp(123123, 000000), 3), static_cast(245)}, pool_.get()); auto max_key = BinaryRowGenerator::GenerateRow( {Decimal(2, 2, DecimalUtils::StrToInt128("78").value()), - TimestampType(Timestamp(123, 123000), 9), static_cast(24)}, + TimestampType(Timestamp(123, 000000), 3), static_cast(24)}, + pool_.get()); + SimpleStats key_stats = BinaryRowGenerator::GenerateStats( + {Decimal(2, 2, DecimalUtils::StrToInt128("12").value()), TimestampType(Timestamp(0, 0), 3), + static_cast(24)}, + {Decimal(2, 2, DecimalUtils::StrToInt128("78").value()), + TimestampType(Timestamp(123999, 000000), 3), static_cast(2456)}, + std::vector({0, 0, 0}), pool_.get()); + SimpleStats value_stats = BinaryRowGenerator::GenerateStats( + {NullType(), NullType(), NullType(), TimestampType(Timestamp(0, 0), 3), + static_cast(24), Decimal(2, 2, DecimalUtils::StrToInt128("12").value())}, + {NullType(), NullType(), NullType(), TimestampType(Timestamp(123999, 000000), 3), + static_cast(2456), Decimal(2, 2, DecimalUtils::StrToInt128("78").value())}, + file_format == "parquet" ? std::vector({-1, -1, -1, 0, 0, 0}) + : std::vector({0, 0, 0, 0, 0, 0}), pool_.get()); - SimpleStats key_stats = SimpleStats::EmptyStats(); - if (file_format != "parquet") { - key_stats = BinaryRowGenerator::GenerateStats( - {Decimal(2, 2, DecimalUtils::StrToInt128("12").value()), - TimestampType(Timestamp(0, 0), 9), static_cast(24)}, - {Decimal(2, 2, DecimalUtils::StrToInt128("78").value()), - TimestampType(Timestamp(123999, 999000), 9), static_cast(2456)}, - std::vector({0, 0, 0}), pool_.get()); - } else { - BinaryRow min_row(3); - BinaryRowWriter min_writer(&min_row, 0, pool_.get()); - min_writer.Reset(); - Decimal decimal(2, 2, DecimalUtils::StrToInt128("12").value()); - min_writer.WriteDecimal(0, decimal, 2); - min_writer.WriteTimestamp(1, std::nullopt, Timestamp::MAX_PRECISION); - min_writer.WriteInt(2, 24); - min_writer.Complete(); - - BinaryRow max_row(3); - BinaryRowWriter max_writer(&max_row, 0, pool_.get()); - max_writer.Reset(); - Decimal decimal1(2, 2, DecimalUtils::StrToInt128("78").value()); - max_writer.WriteDecimal(0, decimal1, 2); - max_writer.WriteTimestamp(1, std::nullopt, Timestamp::MAX_PRECISION); - max_writer.WriteInt(2, 2456); - max_writer.Complete(); - - auto null_counts = BinaryRowGenerator::FromLongArrayWithNull({0, -1, 0}, pool_.get()); - key_stats = SimpleStats(min_row, max_row, null_counts); - } - - SimpleStats value_stats = SimpleStats::EmptyStats(); - if (file_format != "parquet") { - value_stats = BinaryRowGenerator::GenerateStats( - {NullType(), NullType(), NullType(), TimestampType(Timestamp(0, 0), 9), - static_cast(24), Decimal(2, 2, DecimalUtils::StrToInt128("12").value())}, - {NullType(), NullType(), NullType(), TimestampType(Timestamp(123999, 999000), 9), - static_cast(2456), Decimal(2, 2, DecimalUtils::StrToInt128("78").value())}, - std::vector({0, 0, 0, 0, 0, 0}), pool_.get()); - } else { - BinaryRow min_row(6); - BinaryRowWriter min_writer(&min_row, 0, pool_.get()); - min_writer.Reset(); - min_writer.SetNullAt(0); - min_writer.SetNullAt(1); - min_writer.SetNullAt(2); - min_writer.WriteTimestamp(3, std::nullopt, Timestamp::MAX_PRECISION); - min_writer.WriteInt(4, 24); - Decimal decimal(2, 2, DecimalUtils::StrToInt128("12").value()); - min_writer.WriteDecimal(5, decimal, 2); - min_writer.Complete(); - - BinaryRow max_row(6); - BinaryRowWriter max_writer(&max_row, 0, pool_.get()); - max_writer.Reset(); - max_writer.SetNullAt(0); - max_writer.SetNullAt(1); - max_writer.SetNullAt(2); - max_writer.WriteTimestamp(3, std::nullopt, Timestamp::MAX_PRECISION); - max_writer.WriteInt(4, 2456); - Decimal decimal1(2, 2, DecimalUtils::StrToInt128("78").value()); - max_writer.WriteDecimal(5, decimal1, 2); - max_writer.Complete(); - - auto null_counts = - BinaryRowGenerator::FromLongArrayWithNull({-1, -1, -1, -1, 0, 0}, pool_.get()); - value_stats = SimpleStats(min_row, max_row, null_counts); - } auto file_meta = std::make_shared( "data-xxx.xxx", /*file_size=*/543, @@ -1686,21 +1607,21 @@ TEST_P(WriteInteTest, TestPkTableWriteWithComplexType) { helper->NewScan(StartupMode::LatestFull(), /*snapshot_id=*/std::nullopt)); ASSERT_EQ(data_splits_1.size(), 1); std::string expected_data_1 = R"([ - [0, [[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123123", 245, "0.12"], - [0, [[10, 10]], [1.1, 1.2], [false, 12], "1970-01-01 00:02:03.123123", 2456, "0.22"], - [0, [[0, 1]], [0.1, 0.3], [true, 1], "1970-01-01 00:02:03.999999", 24, "0.28"], + [0, [[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123", 245, "0.12"], + [0, [[10, 10]], [1.1, 1.2], [false, 12], "1970-01-01 00:02:03.123", 2456, "0.22"], + [0, [[0, 1]], [0.1, 0.3], [true, 1], "1970-01-01 00:02:03.999", 24, "0.28"], [0, [[1, 64], [2, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.0", 24, "0.78"], - [0, [[11, 64], [12, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.123123", 24, "0.78"] + [0, [[11, 64], [12, 32]], [2.2, 3.2], [true, 2], "1970-01-01 00:00:00.123", 24, "0.78"] ])"; ASSERT_OK_AND_ASSIGN(bool success, helper->ReadAndCheckResult(data_type, data_splits_1, expected_data_1)); ASSERT_TRUE(success); std::string data_2 = R"([ - [[[10, 11]], [1.1, 1.3], [true, 1], "1970-01-01 00:02:03.999999", 24, "0.28"], - [[[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123123", 245, "0.12"], + [[[10, 11]], [1.1, 1.3], [true, 1], "1970-01-01 00:02:03.999", 24, "0.28"], + [[[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123", 245, "0.12"], [[[10, 20], [20, 30]], [2.0, 3.0], [true, 2], "1970-01-01 00:00:00.0", 24, "0.78"], - [[[11, 61], [21, 31]], [2.1, 3.1], [false, 1], "1970-01-01 00:00:00.123123", 24, "0.78"] + [[[11, 61], [21, 31]], [2.1, 3.1], [false, 1], "1970-01-01 00:00:00.123", 24, "0.78"] ])"; ASSERT_OK_AND_ASSIGN( std::unique_ptr batch_2, @@ -1711,11 +1632,11 @@ TEST_P(WriteInteTest, TestPkTableWriteWithComplexType) { auto min_key_2 = BinaryRowGenerator::GenerateRow( {Decimal(2, 2, DecimalUtils::StrToInt128("12").value()), - TimestampType(Timestamp(123123, 123000), 9), static_cast(245)}, + TimestampType(Timestamp(123123, 000000), 3), static_cast(245)}, pool_.get()); auto max_key_2 = BinaryRowGenerator::GenerateRow( {Decimal(2, 2, DecimalUtils::StrToInt128("78").value()), - TimestampType(Timestamp(123, 123000), 9), static_cast(24)}, + TimestampType(Timestamp(123, 000000), 3), static_cast(24)}, pool_.get()); auto key_stats_2 = key_stats; key_stats_2.max_values_.SetInt(2, 245); @@ -1754,10 +1675,10 @@ TEST_P(WriteInteTest, TestPkTableWriteWithComplexType) { ASSERT_OK_AND_ASSIGN(std::vector> data_splits_2, helper->Scan()); ASSERT_EQ(data_splits_2.size(), 1); std::string expected_data_2 = R"([ - [3, [[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123123", 245, "0.12"], - [0, [[10, 11]], [1.1, 1.3], [true, 1], "1970-01-01 00:02:03.999999", 24, "0.28"], + [3, [[127, 32767], [-128, -32768]], [1.1, 1.2], [false, 2222], "1970-01-01 00:02:03.123", 245, "0.12"], + [0, [[10, 11]], [1.1, 1.3], [true, 1], "1970-01-01 00:02:03.999", 24, "0.28"], [1, [[10, 20], [20, 30]], [2.0, 3.0], [true, 2], "1970-01-01 00:00:00.0", 24, "0.78"], - [2, [[11, 61], [21, 31]], [2.1, 3.1], [false, 1], "1970-01-01 00:00:00.123123", 24, "0.78"] + [2, [[11, 61], [21, 31]], [2.1, 3.1], [false, 1], "1970-01-01 00:00:00.123", 24, "0.78"] ])"; ASSERT_OK_AND_ASSIGN(success, @@ -2434,7 +2355,7 @@ TEST_P(WriteInteTest, TestWriteAndCommitIOException) { TEST_P(WriteInteTest, TestWriteWithFieldId) { auto file_format = GetParam(); - if (file_format == "lance") { + if (file_format == "lance" || file_format == "avro") { return; } // prepare write schema and write data @@ -3375,7 +3296,7 @@ TEST_P(WriteInteTest, TestAppendTableWithAllNull) { // test compatible with java ASSERT_EQ(stats.min_values_.HashCode(), 0xc7883013); ASSERT_EQ(stats.max_values_.HashCode(), 0xc7883013); - if (GetParam() != "parquet") { + if (GetParam() != "parquet" && GetParam() != "avro") { ASSERT_EQ(stats.null_counts_.HashCode(), 0x5ddc482d); } } diff --git a/test/test_data/avro/append_multiple.db/append_multiple/README b/test/test_data/avro/append_multiple.db/append_multiple/README index 58d086c9..822d3c56 100644 --- a/test/test_data/avro/append_multiple.db/append_multiple/README +++ b/test/test_data/avro/append_multiple.db/append_multiple/README @@ -9,6 +9,6 @@ Add: 3, 11, 0, 100, null, 3.0, "three", "ccc", 123, "123.45", "1970-01-03 00:00: Add: 4, 11, 0, 100, 4.0, null, "four", "ddd", 123, "123.45", "1970-01-04 00:00:00", "1970-01-04 00:00:00.000", "1970-01-04 00:00:00.000000", "1970-01-04 00:00:00", "1970-01-04 00:00:00.000", "1970-01-04 00:00:00.000000",[[["key",123]],[1,2,3]] snapshot-2 (3 data files) -Add: 5, 10, 0, 100, 5.0, 2.0, null, "eee", 123, "123.45", "1970-01-01 00:00:00", "1970-01-05 00:00:00.000", "1970-01-05 00:00:00.000000", "1970-01-05 00:00:00", "1970-01-05 00:00:00.000", "1970-01-05 00:00:00.000000",[[["key",123]],[1,2,3]] -Add: 6, 10, 1, 100, 6.0, 4.0, "six", "fff", 123, "123.45", "1970-01-02 00:00:00", "1970-01-06 00:00:00.000", "1970-01-06 00:00:00.000000", "1970-01-06 00:00:00", "1970-01-06 00:00:00.000", "1970-01-06 00:00:00.000000",[[["key",123]],[1,2,3]] -Add: 7, 11, 0, 100, 7.0, 6.0, "seven", "ggg", 123, "123.45", "1970-01-03 00:00:00", "1970-01-07 00:00:00.000", "1970-01-07 00:00:00.000000", "1970-01-07 00:00:00", "1970-01-07 00:00:00.000", "1970-01-07 00:00:00.000000",[[["key",123]],[1,2,3]] \ No newline at end of file +Add: 5, 10, 0, 100, 5.0, 2.0, null, "eee", 123, "123.45", "1970-01-05 00:00:00", "1970-01-05 00:00:00.000", "1970-01-05 00:00:00.000000", "1970-01-05 00:00:00", "1970-01-05 00:00:00.000", "1970-01-05 00:00:00.000000",[[["key",123]],[1,2,3]] +Add: 6, 10, 1, 100, 6.0, 4.0, "six", "fff", 123, "123.45", "1970-01-06 00:00:00", "1970-01-06 00:00:00.000", "1970-01-06 00:00:00.000000", "1970-01-06 00:00:00", "1970-01-06 00:00:00.000", "1970-01-06 00:00:00.000000",[[["key",123]],[1,2,3]] +Add: 7, 11, 0, 100, 7.0, 6.0, "seven", "ggg", 123, "123.45", "1970-01-07 00:00:00", "1970-01-07 00:00:00.000", "1970-01-07 00:00:00.000000", "1970-01-07 00:00:00", "1970-01-07 00:00:00.000", "1970-01-07 00:00:00.000000",[[["key",123]],[1,2,3]] \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/README b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/README new file mode 100644 index 00000000..cff5b876 --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/README @@ -0,0 +1,36 @@ +f0:string f1:int f2:int f3:double +partition key: f1 +primary key: f0 f1 f2 +bucket key: f2 +bucket count: 2 +deletion-vectors.enabled = true + +Msgs: +Add: "Two roads diverged in a wood, and I took the one less traveled by, And that has made all the difference.", 10, 1, 11.0 +Add: "Alice", 10, 1, 11.1 + +Add: "Bob", 10, 0, 12.1 +Add: "Emily", 10, 0, 13.1 +Add: "Tony", 10, 0, 14.1 +Add: "Alex", 10, 0, 16.1 +Add: "David", 10, 0, 17.1 + +Add: "Lucy", 20, 1, 14.1 +Add: "Paul", 20, 1, 18.1 +Commit - snapshot 1 +Compact - snapshot 2 + +DELETE: "Tony", 10, 0, 14.1 +UPDATE_AFTER: "Alice", 10, 1, 19.1 +Commit - snapshot 3 +Compact - snapshot 4 + +Add: "Skye", 10, 0, 21.0 +Add: "Marco", 10, 0, 21.1 +UPDATE_AFTER: "Alex", 10, 0, 21.2 +Commit - snapshot 5 + +Add: "Skye2", 10, 0, 31.0 +Add: "Marco2", 10, 0, 31.1 +DELETE: "Alex", 10, 0, 31.2 +Commit - snapshot 6 diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-0/data-32c29de9-0c48-4fd7-ab3d-1978fcb721a5-0.avro b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-0/data-32c29de9-0c48-4fd7-ab3d-1978fcb721a5-0.avro new file mode 100644 index 00000000..e61855ae Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-0/data-32c29de9-0c48-4fd7-ab3d-1978fcb721a5-0.avro differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-0/data-32c29de9-0c48-4fd7-ab3d-1978fcb721a5-1.avro b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-0/data-32c29de9-0c48-4fd7-ab3d-1978fcb721a5-1.avro new file mode 100644 index 00000000..914f885b Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-0/data-32c29de9-0c48-4fd7-ab3d-1978fcb721a5-1.avro differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-7d312c16-b116-42d0-887e-c7853e4a2df5-0.avro b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-7d312c16-b116-42d0-887e-c7853e4a2df5-0.avro new file mode 100644 index 00000000..5da195bd Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-7d312c16-b116-42d0-887e-c7853e4a2df5-0.avro differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-7d312c16-b116-42d0-887e-c7853e4a2df5-1.avro b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-7d312c16-b116-42d0-887e-c7853e4a2df5-1.avro new file mode 100644 index 00000000..1ca0ba1d Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-7d312c16-b116-42d0-887e-c7853e4a2df5-1.avro differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-b156a829-1fa3-4612-bd31-6e896a8a24cc-0.avro b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-b156a829-1fa3-4612-bd31-6e896a8a24cc-0.avro new file mode 100644 index 00000000..28d72be8 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-b156a829-1fa3-4612-bd31-6e896a8a24cc-0.avro differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-b156a829-1fa3-4612-bd31-6e896a8a24cc-1.avro b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-b156a829-1fa3-4612-bd31-6e896a8a24cc-1.avro new file mode 100644 index 00000000..71f37d84 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=10/bucket-1/data-b156a829-1fa3-4612-bd31-6e896a8a24cc-1.avro differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=20/bucket-0/data-502ebf05-03a4-4c6e-aa31-15e9b6c0429d-0.avro b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=20/bucket-0/data-502ebf05-03a4-4c6e-aa31-15e9b6c0429d-0.avro new file mode 100644 index 00000000..af3eab4c Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/f1=20/bucket-0/data-502ebf05-03a4-4c6e-aa31-15e9b6c0429d-0.avro differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/index/index-3be0d042-692a-41cc-8d24-449ebfd0370a-0 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/index/index-3be0d042-692a-41cc-8d24-449ebfd0370a-0 new file mode 100644 index 00000000..dc94633e Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/index/index-3be0d042-692a-41cc-8d24-449ebfd0370a-0 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/index/index-3be0d042-692a-41cc-8d24-449ebfd0370a-1 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/index/index-3be0d042-692a-41cc-8d24-449ebfd0370a-1 new file mode 100644 index 00000000..3bb00480 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/index/index-3be0d042-692a-41cc-8d24-449ebfd0370a-1 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/index-manifest-a944b7c9-e594-4123-b581-4d57b9d2faa8-0 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/index-manifest-a944b7c9-e594-4123-b581-4d57b9d2faa8-0 new file mode 100644 index 00000000..911bfe5c Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/index-manifest-a944b7c9-e594-4123-b581-4d57b9d2faa8-0 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-b04fa92d-d4c1-452d-81e8-ea226867ea1c-0 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-b04fa92d-d4c1-452d-81e8-ea226867ea1c-0 new file mode 100644 index 00000000..02b349ef Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-b04fa92d-d4c1-452d-81e8-ea226867ea1c-0 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-b04fa92d-d4c1-452d-81e8-ea226867ea1c-1 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-b04fa92d-d4c1-452d-81e8-ea226867ea1c-1 new file mode 100644 index 00000000..6578ca57 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-b04fa92d-d4c1-452d-81e8-ea226867ea1c-1 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-0 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-0 new file mode 100644 index 00000000..d97a7e41 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-0 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-1 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-1 new file mode 100644 index 00000000..95f5c51c Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-1 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-2 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-2 new file mode 100644 index 00000000..c87dc45a Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-2 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-3 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-3 new file mode 100644 index 00000000..a0403ea7 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-d4ac5a14-906b-489b-9a8b-cc7bc6406c0f-3 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-0 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-0 new file mode 100644 index 00000000..1009d0d1 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-0 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-1 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-1 new file mode 100644 index 00000000..38158aec Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-1 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-2 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-2 new file mode 100644 index 00000000..d7ae8e68 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-2 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-3 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-3 new file mode 100644 index 00000000..28750ed1 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-3 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-0 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-0 new file mode 100644 index 00000000..9e648f13 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-0 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-1 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-1 new file mode 100644 index 00000000..5dcc3f90 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-1 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-2 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-2 new file mode 100644 index 00000000..72de9cbc Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-2 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-3 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-3 new file mode 100644 index 00000000..c865f3e8 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-3 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-4 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-4 new file mode 100644 index 00000000..3be9ac39 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-4 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-5 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-5 new file mode 100644 index 00000000..bd5fdb82 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-5 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-6 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-6 new file mode 100644 index 00000000..67eb1d81 Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-6 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-7 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-7 new file mode 100644 index 00000000..ff59028d Binary files /dev/null and b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/manifest/manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-7 differ diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/schema/schema-0 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/schema/schema-0 new file mode 100644 index 00000000..c519d6c6 --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/schema/schema-0 @@ -0,0 +1,33 @@ +{ + "version" : 3, + "id" : 0, + "fields" : [ { + "id" : 0, + "name" : "f0", + "type" : "STRING NOT NULL" + }, { + "id" : 1, + "name" : "f1", + "type" : "INT NOT NULL" + }, { + "id" : 2, + "name" : "f2", + "type" : "INT NOT NULL" + }, { + "id" : 3, + "name" : "f3", + "type" : "DOUBLE" + } ], + "highestFieldId" : 3, + "partitionKeys" : [ "f1" ], + "primaryKeys" : [ "f0", "f1", "f2" ], + "options" : { + "bucket" : "2", + "bucket-key" : "f2", + "manifest.format" : "avro", + "file.format" : "avro", + "deletion-vectors.enabled" : "true", + "commit.force-compact" : "true" + }, + "timeMillis" : 1770109651519 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/EARLIEST b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/EARLIEST new file mode 100644 index 00000000..56a6051c --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/LATEST b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/LATEST new file mode 100644 index 00000000..62f94575 --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/LATEST @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-1 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-1 new file mode 100644 index 00000000..e420a1cc --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-1 @@ -0,0 +1,16 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-0", + "baseManifestListSize" : 1006, + "deltaManifestList" : "manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-1", + "deltaManifestListSize" : 1122, + "commitUser" : "b9854fe1-77cd-46c7-868a-ff7909d748a2", + "commitIdentifier" : 1, + "commitKind" : "APPEND", + "timeMillis" : 1770109652554, + "totalRecordCount" : 9, + "deltaRecordCount" : 9, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-2 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-2 new file mode 100644 index 00000000..bd0dcbaf --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-2 @@ -0,0 +1,16 @@ +{ + "version" : 3, + "id" : 2, + "schemaId" : 0, + "baseManifestList" : "manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-2", + "baseManifestListSize" : 1122, + "deltaManifestList" : "manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-3", + "deltaManifestListSize" : 1120, + "commitUser" : "b9854fe1-77cd-46c7-868a-ff7909d748a2", + "commitIdentifier" : 1, + "commitKind" : "COMPACT", + "timeMillis" : 1770109652716, + "totalRecordCount" : 9, + "deltaRecordCount" : 0, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-3 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-3 new file mode 100644 index 00000000..5a446bea --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-3 @@ -0,0 +1,16 @@ +{ + "version" : 3, + "id" : 3, + "schemaId" : 0, + "baseManifestList" : "manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-4", + "baseManifestListSize" : 1134, + "deltaManifestList" : "manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-5", + "deltaManifestListSize" : 1116, + "commitUser" : "b9854fe1-77cd-46c7-868a-ff7909d748a2", + "commitIdentifier" : 2, + "commitKind" : "APPEND", + "timeMillis" : 1770109652784, + "totalRecordCount" : 11, + "deltaRecordCount" : 2, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-4 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-4 new file mode 100644 index 00000000..7f8452f8 --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-4 @@ -0,0 +1,17 @@ +{ + "version" : 3, + "id" : 4, + "schemaId" : 0, + "baseManifestList" : "manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-6", + "baseManifestListSize" : 1148, + "deltaManifestList" : "manifest-list-c771cbf4-0a50-40bc-bf71-917a3f82d8b0-7", + "deltaManifestListSize" : 1117, + "indexManifest" : "index-manifest-a944b7c9-e594-4123-b581-4d57b9d2faa8-0", + "commitUser" : "b9854fe1-77cd-46c7-868a-ff7909d748a2", + "commitIdentifier" : 2, + "commitKind" : "COMPACT", + "timeMillis" : 1770109652807, + "totalRecordCount" : 10, + "deltaRecordCount" : -1, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-5 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-5 new file mode 100644 index 00000000..19b34d07 --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-5 @@ -0,0 +1,17 @@ +{ + "version" : 3, + "id" : 5, + "schemaId" : 0, + "baseManifestList" : "manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-0", + "baseManifestListSize" : 1159, + "deltaManifestList" : "manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-1", + "deltaManifestListSize" : 1115, + "indexManifest" : "index-manifest-a944b7c9-e594-4123-b581-4d57b9d2faa8-0", + "commitUser" : "fa1dcf7d-445f-4939-9087-1471f606c82b", + "commitIdentifier" : 3, + "commitKind" : "APPEND", + "timeMillis" : 1770109652838, + "totalRecordCount" : 13, + "deltaRecordCount" : 3, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-6 b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-6 new file mode 100644 index 00000000..7214e127 --- /dev/null +++ b/test/test_data/avro/pk_table_scan_and_read_dv.db/pk_table_scan_and_read_dv/snapshot/snapshot-6 @@ -0,0 +1,17 @@ +{ + "version" : 3, + "id" : 6, + "schemaId" : 0, + "baseManifestList" : "manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-2", + "baseManifestListSize" : 1194, + "deltaManifestList" : "manifest-list-68019c51-e09d-44cd-aaf0-74f9e5174cc1-3", + "deltaManifestListSize" : 1116, + "indexManifest" : "index-manifest-a944b7c9-e594-4123-b581-4d57b9d2faa8-0", + "commitUser" : "fa1dcf7d-445f-4939-9087-1471f606c82b", + "commitIdentifier" : 4, + "commitKind" : "APPEND", + "timeMillis" : 1770109652848, + "totalRecordCount" : 16, + "deltaRecordCount" : 3, + "nextRowId" : 0 +} \ No newline at end of file