diff --git a/LICENSE b/LICENSE index 8b5c0b97..18605853 100644 --- a/LICENSE +++ b/LICENSE @@ -263,6 +263,9 @@ This product includes code from Apache Iceberg C++. * .devcontainer/devcontainer.json.template * CI utilities: * .pre-commit-config.yaml +* Avro direct decoder/encoder: + * src/paimon/format/avro/avro_direct_decoder.cpp + * src/paimon/format/avro/avro_direct_decoder.h Copyright: 2024-2025 The Apache Software Foundation. Home page: https://iceberg.apache.org/ diff --git a/src/paimon/CMakeLists.txt b/src/paimon/CMakeLists.txt index ead27e63..33c45ba1 100644 --- a/src/paimon/CMakeLists.txt +++ b/src/paimon/CMakeLists.txt @@ -386,6 +386,7 @@ if(PAIMON_BUILD_TESTS) common/types/data_type_json_parser_test.cpp common/types/row_kind_test.cpp common/types/data_type_test.cpp + common/utils/arrow/arrow_utils_test.cpp common/utils/arrow/mem_utils_test.cpp common/utils/arrow/status_utils_test.cpp common/utils/concurrent_hash_map_test.cpp diff --git a/src/paimon/common/utils/arrow/arrow_utils.h b/src/paimon/common/utils/arrow/arrow_utils.h index 995b3fff..5c23ec83 100644 --- a/src/paimon/common/utils/arrow/arrow_utils.h +++ b/src/paimon/common/utils/arrow/arrow_utils.h @@ -16,14 +16,41 @@ #pragma once -#include "arrow/c/helpers.h" +#include + +#include "arrow/api.h" +#include "fmt/format.h" +#include "paimon/result.h" namespace paimon { -inline void ArrowArrayInit(struct ArrowArray* array) { - ArrowArrayMarkReleased(array); -} -inline void ArrowSchemaInit(struct ArrowSchema* schema) { - ArrowSchemaMarkReleased(schema); -} +class ArrowUtils { + public: + ArrowUtils() = delete; + ~ArrowUtils() = delete; + + static Result> DataTypeToSchema( + const std::shared_ptr<::arrow::DataType>& data_type) { + if (data_type->id() != arrow::Type::STRUCT) { + return Status::Invalid(fmt::format("Expected struct data type, actual data type: {}", + data_type->ToString())); + } + const auto& struct_type = std::static_pointer_cast(data_type); + return std::make_shared(struct_type->fields()); + } + + static std::vector 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); + target_to_src_mapping.push_back(src_field_idx); + } + return target_to_src_mapping; + } +}; + } // namespace paimon diff --git a/src/paimon/common/utils/arrow/arrow_utils_test.cpp b/src/paimon/common/utils/arrow/arrow_utils_test.cpp new file mode 100644 index 00000000..afc86d10 --- /dev/null +++ b/src/paimon/common/utils/arrow/arrow_utils_test.cpp @@ -0,0 +1,47 @@ +/* + * 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/common/utils/arrow/arrow_utils.h" + +#include "arrow/api.h" +#include "gtest/gtest.h" +#include "paimon/common/types/data_field.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); + + 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); +} + +} // namespace paimon::test diff --git a/src/paimon/common/utils/status.cpp b/src/paimon/common/utils/status.cpp index 5f21b08b..0ee1a533 100644 --- a/src/paimon/common/utils/status.cpp +++ b/src/paimon/common/utils/status.cpp @@ -142,7 +142,7 @@ void Status::AddContextLine(const char* filename, int line, const char* function const char* expr) { assert(!ok() && "Cannot add context line to ok status"); std::stringstream ss; - ss << "\nIn " << filename << ", line " << line << ", function: " << function_name + ss << "\nIn " << filename << ":" << line << ", function: " << function_name << ", code: " << expr; state_->msg += ss.str(); } diff --git a/src/paimon/core/io/row_to_arrow_array_converter.h b/src/paimon/core/io/row_to_arrow_array_converter.h index fd8b307b..9d8ea932 100644 --- a/src/paimon/core/io/row_to_arrow_array_converter.h +++ b/src/paimon/core/io/row_to_arrow_array_converter.h @@ -167,7 +167,7 @@ Status RowToArrowArrayConverter::Reserve(arrow::ArrayBuilder* array_builde PAIMON_ASSIGN_OR_RAISE(auto* struct_builder, CastToTypedBuilder(array_builder)); for (int32_t i = 0; i < struct_builder->num_fields(); i++) { - // reserve item builder in map + // reserve item builder in struct PAIMON_RETURN_NOT_OK(Reserve(struct_builder->field_builder(i), idx)); } break; diff --git a/src/paimon/core/io/single_file_writer.h b/src/paimon/core/io/single_file_writer.h index ca4f19b3..862772eb 100644 --- a/src/paimon/core/io/single_file_writer.h +++ b/src/paimon/core/io/single_file_writer.h @@ -175,7 +175,7 @@ Status SingleFileWriter::Write(T record) { } } else { ArrowArray array; - ArrowArrayInit(&array); + ArrowArrayMarkReleased(&array); // reset array ScopeGuard inner_guard([&array]() { ArrowArrayRelease(&array); }); PAIMON_RETURN_NOT_OK(converter_(std::move(record), &array)); record_count = array.length; diff --git a/src/paimon/core/operation/abstract_split_read.cpp b/src/paimon/core/operation/abstract_split_read.cpp index d3be9353..8fa3f0c8 100644 --- a/src/paimon/core/operation/abstract_split_read.cpp +++ b/src/paimon/core/operation/abstract_split_read.cpp @@ -147,7 +147,9 @@ Result> AbstractSplitRead::CreateFileBatchReade // lance do not support stream build with input stream return reader_builder->Build(data_file_path); } - if (context_->EnablePrefetch() && file_format_identifier != "blob") { + // TODO(zhanyu.fyh): orc format support prefetch + if (context_->EnablePrefetch() && file_format_identifier != "blob" && + file_format_identifier != "avro") { PAIMON_ASSIGN_OR_RAISE( std::unique_ptr prefetch_reader, PrefetchFileBatchReaderImpl::Create( diff --git a/src/paimon/core/operation/merge_file_split_read.cpp b/src/paimon/core/operation/merge_file_split_read.cpp index 9de26f34..c6688e07 100644 --- a/src/paimon/core/operation/merge_file_split_read.cpp +++ b/src/paimon/core/operation/merge_file_split_read.cpp @@ -32,6 +32,7 @@ #include "paimon/common/reader/concat_batch_reader.h" #include "paimon/common/table/special_fields.h" #include "paimon/common/types/data_field.h" +#include "paimon/common/utils/arrow/arrow_utils.h" #include "paimon/common/utils/arrow/status_utils.h" #include "paimon/core/core_options.h" #include "paimon/core/deletionvectors/apply_deletion_vector_batch_reader.h" @@ -105,7 +106,8 @@ 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 = CreateProjection(context->GetReadSchema(), value_schema); + std::vector projection = + ArrowUtils::CreateProjection(value_schema, context->GetReadSchema()->fields()); return std::unique_ptr(new MergeFileSplitRead( path_factory, context, @@ -368,19 +370,6 @@ Result> MergeFileSplitRead::GenerateKeyPredicates( return PredicateUtils::ExcludePredicateWithFields(predicate, non_primary_keys); } -std::vector MergeFileSplitRead::CreateProjection( - const std::shared_ptr& raw_read_schema, - const std::shared_ptr& value_schema) { - std::vector target_to_src_mapping; - target_to_src_mapping.reserve(raw_read_schema->num_fields()); - for (const auto& field : raw_read_schema->fields()) { - auto src_field_idx = value_schema->GetFieldIndex(field->name()); - assert(src_field_idx >= 0); - target_to_src_mapping.push_back(src_field_idx); - } - return target_to_src_mapping; -} - Result> MergeFileSplitRead::CreateReaderForSection( const std::vector& section, const std::string& bucket_path, const BinaryRow& partition, diff --git a/src/paimon/core/operation/merge_file_split_read.h b/src/paimon/core/operation/merge_file_split_read.h index fc20c6ac..483e4742 100644 --- a/src/paimon/core/operation/merge_file_split_read.h +++ b/src/paimon/core/operation/merge_file_split_read.h @@ -154,10 +154,6 @@ class MergeFileSplitRead : public AbstractSplitRead { static Result> GenerateKeyPredicates( const std::shared_ptr& predicate, const TableSchema& table_schema); - static std::vector CreateProjection( - const std::shared_ptr& raw_read_schema, - const std::shared_ptr& value_schema); - private: int32_t key_arity_; // schema of value member in KeyValue object diff --git a/src/paimon/core/operation/merge_file_split_read_test.cpp b/src/paimon/core/operation/merge_file_split_read_test.cpp index 5aafc2f5..09ed81a7 100644 --- a/src/paimon/core/operation/merge_file_split_read_test.cpp +++ b/src/paimon/core/operation/merge_file_split_read_test.cpp @@ -553,28 +553,6 @@ TEST_F(MergeFileSplitReadTest, TestGenerateKeyPredicates2) { ASSERT_FALSE(result); } -TEST_F(MergeFileSplitReadTest, TestCreateProjection) { - std::vector raw_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 raw_read_schema = DataField::ConvertDataFieldsToArrowSchema(raw_read_fields); - - std::vector value_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 value_schema = DataField::ConvertDataFieldsToArrowSchema(value_fields); - - auto projection = MergeFileSplitRead::CreateProjection(raw_read_schema, value_schema); - std::vector expected_projection = {1, 2, 3, 4, 5}; - ASSERT_EQ(projection, expected_projection); -} - TEST_P(MergeFileSplitReadTest, TestSimple) { std::string path = paimon::test::GetDataDir() + "/parquet/pk_table_with_mor.db/pk_table_with_mor"; diff --git a/src/paimon/format/avro/CMakeLists.txt b/src/paimon/format/avro/CMakeLists.txt index 0811b2cf..97c4113a 100644 --- a/src/paimon/format/avro/CMakeLists.txt +++ b/src/paimon/format/avro/CMakeLists.txt @@ -16,15 +16,13 @@ if(PAIMON_ENABLE_AVRO) set(PAIMON_AVRO_FILE_FORMAT avro_adaptor.cpp - avro_array_data_getter.cpp + avro_direct_decoder.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_record_converter.cpp - avro_record_data_getter.cpp avro_schema_converter.cpp) add_paimon_lib(paimon_avro_file_format @@ -55,10 +53,8 @@ if(PAIMON_ENABLE_AVRO) avro_file_batch_reader_test.cpp avro_file_format_test.cpp avro_input_stream_impl_test.cpp - avro_record_converter_test.cpp avro_schema_converter_test.cpp avro_writer_builder_test.cpp - avro_array_data_getter_test.cpp EXTRA_INCLUDES ${AVRO_INCLUDE_DIR} STATIC_LINK_LIBS diff --git a/src/paimon/format/avro/avro_adaptor_test.cpp b/src/paimon/format/avro/avro_adaptor_test.cpp index 0cc8140c..34cf6624 100644 --- a/src/paimon/format/avro/avro_adaptor_test.cpp +++ b/src/paimon/format/avro/avro_adaptor_test.cpp @@ -27,7 +27,6 @@ #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_record_converter.h" #include "paimon/format/avro/avro_schema_converter.h" #include "paimon/memory/memory_pool.h" #include "paimon/status.h" @@ -62,17 +61,6 @@ TEST(AvroAdaptorTest, Simple) { ASSERT_OK_AND_ASSIGN(std::vector<::avro::GenericDatum> datums, adaptor.ConvertArrayToGenericDatums(array, avro_schema)); ASSERT_EQ(4, datums.size()); - ASSERT_OK_AND_ASSIGN(auto record_converter, - AvroRecordConverter::Create(data_type, GetDefaultPool())); - auto read_batch_result = record_converter->NextBatch(datums); - ASSERT_OK(read_batch_result); - auto [c_array, c_schema] = std::move(read_batch_result).value(); - - auto arrow_array = arrow::ImportArray(c_array.get(), c_schema.get()).ValueOrDie(); - auto arrow_pool = GetArrowPool(GetDefaultPool()); - ASSERT_OK_AND_ASSIGN(arrow_array, ManifestMetaReader::AlignArrayWithSchema( - arrow_array, data_type, arrow_pool.get())); - ASSERT_TRUE(array->Equals(arrow_array)); } } // namespace paimon::avro::test diff --git a/src/paimon/format/avro/avro_array_data_getter.cpp b/src/paimon/format/avro/avro_array_data_getter.cpp deleted file mode 100644 index 1801247b..00000000 --- a/src/paimon/format/avro/avro_array_data_getter.cpp +++ /dev/null @@ -1,91 +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_array_data_getter.h" - -#include "avro/GenericDatum.hh" -#include "paimon/format/avro/avro_datum_data_getter.h" - -namespace paimon::avro { - -bool AvroArrayDataGetter::IsNullAt(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::IsNullAt(array_.value()[pos]); -} -bool AvroArrayDataGetter::GetBoolean(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetBoolean(array_.value()[pos]); -} -char AvroArrayDataGetter::GetByte(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetByte(array_.value()[pos]); -} -int16_t AvroArrayDataGetter::GetShort(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetShort(array_.value()[pos]); -} -int32_t AvroArrayDataGetter::GetInt(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetInt(array_.value()[pos]); -} -int32_t AvroArrayDataGetter::GetDate(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetDate(array_.value()[pos]); -} -int64_t AvroArrayDataGetter::GetLong(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetLong(array_.value()[pos]); -} -float AvroArrayDataGetter::GetFloat(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetFloat(array_.value()[pos]); -} -double AvroArrayDataGetter::GetDouble(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetDouble(array_.value()[pos]); -} -BinaryString AvroArrayDataGetter::GetString(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetString(array_.value()[pos], pool_); -} -std::string_view AvroArrayDataGetter::GetStringView(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetStringView(array_.value()[pos]); -} -Decimal AvroArrayDataGetter::GetDecimal(int32_t pos, int32_t precision, int32_t scale) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetDecimal(array_.value()[pos], precision, scale, pool_); -} -Timestamp AvroArrayDataGetter::GetTimestamp(int32_t pos, int32_t precision) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetTimestamp(array_.value()[pos], precision); -} -std::shared_ptr AvroArrayDataGetter::GetBinary(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetBinary(array_.value()[pos], pool_); -} - -std::shared_ptr AvroArrayDataGetter::GetArray(int32_t pos) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetArray(array_.value()[pos], pool_); -} - -std::shared_ptr AvroArrayDataGetter::GetRow(int32_t pos, int32_t num_fields) const { - assert(pos < Size()); - return AvroDatumDataGetter::GetRow(array_.value()[pos], num_fields, pool_); -} - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_array_data_getter.h b/src/paimon/format/avro/avro_array_data_getter.h deleted file mode 100644 index cd102073..00000000 --- a/src/paimon/format/avro/avro_array_data_getter.h +++ /dev/null @@ -1,105 +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 "avro/GenericDatum.hh" -#include "paimon/common/data/binary_string.h" -#include "paimon/common/data/internal_array.h" -#include "paimon/common/data/internal_row.h" -#include "paimon/data/decimal.h" -#include "paimon/data/timestamp.h" -#include "paimon/memory/bytes.h" -#include "paimon/result.h" -#include "paimon/status.h" - -namespace paimon { -class InternalMap; -class MemoryPool; -} // namespace paimon - -namespace paimon::avro { - -class AvroArrayDataGetter : public InternalArray { - public: - AvroArrayDataGetter(const ::avro::GenericArray& array, const std::shared_ptr& pool) - : array_(array), pool_(pool) {} - - bool IsNullAt(int32_t pos) const override; - bool GetBoolean(int32_t pos) const override; - char GetByte(int32_t pos) const override; - int16_t GetShort(int32_t pos) const override; - int32_t GetInt(int32_t pos) const override; - int32_t GetDate(int32_t pos) const override; - int64_t GetLong(int32_t pos) const override; - float GetFloat(int32_t pos) const override; - double GetDouble(int32_t pos) const override; - BinaryString GetString(int32_t pos) const override; - std::string_view GetStringView(int32_t pos) const override; - Decimal GetDecimal(int32_t pos, int32_t precision, int32_t scale) const override; - Timestamp GetTimestamp(int32_t pos, int32_t precision) const override; - std::shared_ptr GetBinary(int32_t pos) const override; - std::shared_ptr GetArray(int32_t pos) const override; - std::shared_ptr GetRow(int32_t pos, int32_t num_fields) const override; - - int32_t Size() const override { - return array_.value().size(); - } - std::shared_ptr GetMap(int32_t pos) const override { - assert(false); - return nullptr; - } - Result> ToBooleanArray() const override { - assert(false); - return Status::NotImplemented("not implemented"); - } - Result> ToByteArray() const override { - assert(false); - return Status::NotImplemented("not implemented"); - } - Result> ToShortArray() const override { - assert(false); - return Status::NotImplemented("not implemented"); - } - Result> ToIntArray() const override { - assert(false); - return Status::NotImplemented("not implemented"); - } - Result> ToLongArray() const override { - assert(false); - return Status::NotImplemented("not implemented"); - } - Result> ToFloatArray() const override { - assert(false); - return Status::NotImplemented("not implemented"); - } - Result> ToDoubleArray() const override { - assert(false); - return Status::NotImplemented("not implemented"); - } - - private: - const ::avro::GenericArray& array_; - std::shared_ptr pool_; -}; - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_array_data_getter_test.cpp b/src/paimon/format/avro/avro_array_data_getter_test.cpp deleted file mode 100644 index 7de5b38e..00000000 --- a/src/paimon/format/avro/avro_array_data_getter_test.cpp +++ /dev/null @@ -1,96 +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_array_data_getter.h" - -#include -#include - -#include "avro/GenericDatum.hh" -#include "avro/Schema.hh" -#include "gtest/gtest.h" -#include "paimon/memory/memory_pool.h" - -namespace paimon::avro::test { - -TEST(AvroArrayDataGetterTest, TestBasic) { - { - ::avro::GenericArray bool_array(::avro::ArraySchema(::avro::BoolSchema()).root()); - bool_array.value().push_back(true); - bool_array.value().push_back(false); - AvroArrayDataGetter getter(bool_array, GetDefaultPool()); - ASSERT_EQ(true, getter.GetBoolean(0)); - ASSERT_EQ(false, getter.GetBoolean(1)); - } - { - ::avro::GenericArray char_array(::avro::ArraySchema(::avro::IntSchema()).root()); - char_array.value().push_back(std::numeric_limits::max()); - char_array.value().push_back(std::numeric_limits::min()); - AvroArrayDataGetter getter(char_array, GetDefaultPool()); - ASSERT_EQ(std::numeric_limits::max(), getter.GetByte(0)); - ASSERT_EQ(std::numeric_limits::min(), getter.GetByte(1)); - } - { - ::avro::GenericArray short_array(::avro::ArraySchema(::avro::IntSchema()).root()); - short_array.value().push_back(std::numeric_limits::max()); - short_array.value().push_back(std::numeric_limits::min()); - AvroArrayDataGetter getter(short_array, GetDefaultPool()); - ASSERT_EQ(std::numeric_limits::max(), getter.GetShort(0)); - ASSERT_EQ(std::numeric_limits::min(), getter.GetShort(1)); - } - { - ::avro::GenericArray int_array(::avro::ArraySchema(::avro::IntSchema()).root()); - int_array.value().push_back(std::numeric_limits::max()); - int_array.value().push_back(std::numeric_limits::min()); - AvroArrayDataGetter getter(int_array, GetDefaultPool()); - ASSERT_EQ(std::numeric_limits::max(), getter.GetInt(0)); - ASSERT_EQ(std::numeric_limits::min(), getter.GetInt(1)); - } - { - ::avro::GenericArray long_array(::avro::ArraySchema(::avro::LongSchema()).root()); - long_array.value().push_back(std::numeric_limits::max()); - long_array.value().push_back(std::numeric_limits::min()); - AvroArrayDataGetter getter(long_array, GetDefaultPool()); - ASSERT_EQ(std::numeric_limits::max(), getter.GetLong(0)); - ASSERT_EQ(std::numeric_limits::min(), getter.GetLong(1)); - } - { - ::avro::GenericArray float_array(::avro::ArraySchema(::avro::FloatSchema()).root()); - float_array.value().push_back(std::numeric_limits::max()); - float_array.value().push_back(std::numeric_limits::min()); - AvroArrayDataGetter getter(float_array, GetDefaultPool()); - ASSERT_EQ(std::numeric_limits::max(), getter.GetFloat(0)); - ASSERT_EQ(std::numeric_limits::min(), getter.GetFloat(1)); - } - { - ::avro::GenericArray double_array(::avro::ArraySchema(::avro::DoubleSchema()).root()); - double_array.value().push_back(std::numeric_limits::max()); - double_array.value().push_back(std::numeric_limits::min()); - AvroArrayDataGetter getter(double_array, GetDefaultPool()); - ASSERT_EQ(std::numeric_limits::max(), getter.GetDouble(0)); - ASSERT_EQ(std::numeric_limits::min(), getter.GetDouble(1)); - } - { - ::avro::GenericArray string_array(::avro::ArraySchema(::avro::StringSchema()).root()); - string_array.value().push_back(std::string("apple")); - string_array.value().push_back(std::string("banana")); - AvroArrayDataGetter getter(string_array, GetDefaultPool()); - ASSERT_EQ("apple", getter.GetString(0).ToString()); - ASSERT_EQ("banana", getter.GetString(1).ToString()); - } -} - -} // namespace paimon::avro::test diff --git a/src/paimon/format/avro/avro_datum_data_getter.h b/src/paimon/format/avro/avro_datum_data_getter.h deleted file mode 100644 index 1f354f00..00000000 --- a/src/paimon/format/avro/avro_datum_data_getter.h +++ /dev/null @@ -1,149 +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 "avro/GenericDatum.hh" -#include "paimon/common/data/internal_array.h" -#include "paimon/common/data/internal_row.h" -#include "paimon/common/utils/date_time_utils.h" -#include "paimon/format/avro/avro_array_data_getter.h" -#include "paimon/format/avro/avro_record_data_getter.h" - -namespace paimon::avro { - -class AvroDatumDataGetter { - public: - static bool IsNullAt(const ::avro::GenericDatum& datum) { - return (datum.type() == ::avro::AVRO_NULL); - } - static bool GetBoolean(const ::avro::GenericDatum& datum) { - assert(datum.type() == ::avro::AVRO_BOOL); - return datum.value(); - } - static char GetByte(const ::avro::GenericDatum& datum) { - assert(datum.type() == ::avro::AVRO_INT); - return datum.value(); - } - static int16_t GetShort(const ::avro::GenericDatum& datum) { - assert(datum.type() == ::avro::AVRO_INT); - return datum.value(); - } - static int32_t GetInt(const ::avro::GenericDatum& datum) { - assert(datum.type() == ::avro::AVRO_INT); - return datum.value(); - } - static int32_t GetDate(const ::avro::GenericDatum& datum) { - assert(datum.type() == ::avro::AVRO_INT); - return datum.value(); - } - static int64_t GetLong(const ::avro::GenericDatum& datum) { - assert(datum.type() == ::avro::AVRO_LONG); - return datum.value(); - } - static float GetFloat(const ::avro::GenericDatum& datum) { - assert(datum.type() == ::avro::AVRO_FLOAT); - return datum.value(); - } - static double GetDouble(const ::avro::GenericDatum& datum) { - assert(datum.type() == ::avro::AVRO_DOUBLE); - return datum.value(); - } - static BinaryString GetString(const ::avro::GenericDatum& datum, - const std::shared_ptr& pool) { - assert(datum.type() == ::avro::AVRO_STRING); - return BinaryString::FromString(datum.value(), pool.get()); - } - static std::string_view GetStringView(const ::avro::GenericDatum& datum) { - if (datum.type() == ::avro::AVRO_STRING) { - return {datum.value()}; - } else if (datum.type() == ::avro::AVRO_BYTES) { - const auto& binary = datum.value>(); - return {reinterpret_cast(binary.data()), binary.size()}; - } else { - assert(false); - return {""}; - } - } - static Decimal GetDecimal(const ::avro::GenericDatum& datum, int32_t precision, int32_t scale, - const std::shared_ptr& pool) { - assert(datum.type() == ::avro::AVRO_BYTES); - auto logical_type = datum.logicalType(); - switch (logical_type.type()) { - case ::avro::LogicalType::DECIMAL: { - auto bytes = GetBinary(datum, pool); - assert(logical_type.precision() == precision && logical_type.scale() == scale); - return Decimal::FromUnscaledBytes(precision, scale, bytes.get()); - } - default: - assert(false); - return Decimal::FromUnscaledLong(0, 0, 0); - } - } - static Timestamp GetTimestamp(const ::avro::GenericDatum& datum, int32_t precision) { - assert(datum.type() == ::avro::AVRO_LONG); - switch (datum.logicalType().type()) { - case ::avro::LogicalType::TIMESTAMP_MILLIS: - case ::avro::LogicalType::LOCAL_TIMESTAMP_MILLIS: - return Timestamp(/*millisecond=*/datum.value(), /*nano_of_millisecond=*/0); - case ::avro::LogicalType::TIMESTAMP_MICROS: - case ::avro::LogicalType::LOCAL_TIMESTAMP_MICROS: { - auto [milliseconds, nanoseconds] = DateTimeUtils::TimestampConverter( - datum.value(), DateTimeUtils::MICROSECOND, DateTimeUtils::MILLISECOND, - DateTimeUtils::NANOSECOND); - return Timestamp(milliseconds, nanoseconds); - } - case ::avro::LogicalType::TIMESTAMP_NANOS: - case ::avro::LogicalType::LOCAL_TIMESTAMP_NANOS: { - assert(false); // Java Avro do not support TIMESTAMP_NANOS, should not call this - auto [milliseconds, nanoseconds] = DateTimeUtils::TimestampConverter( - datum.value(), DateTimeUtils::NANOSECOND, DateTimeUtils::MILLISECOND, - DateTimeUtils::NANOSECOND); - return Timestamp(milliseconds, nanoseconds); - } - default: - assert(false); // do not have TIMESTAMP_SECONDS/LOCAL_TIMESTAMP_SECONDS - return Timestamp(/*millisecond=*/0, /*nano_of_millisecond=*/0); - } - } - - static std::shared_ptr GetBinary(const ::avro::GenericDatum& datum, - const std::shared_ptr& pool) { - assert(datum.type() == ::avro::AVRO_BYTES); - const auto& binary = datum.value>(); - return std::make_shared( - std::string(reinterpret_cast(binary.data()), binary.size()), pool.get()); - } - static std::shared_ptr GetArray(const ::avro::GenericDatum& datum, - const std::shared_ptr& pool) { - assert(datum.type() == ::avro::AVRO_ARRAY); - return std::make_shared(datum.value<::avro::GenericArray>(), pool); - } - static std::shared_ptr GetRow(const ::avro::GenericDatum& datum, - int32_t num_fields, - const std::shared_ptr& pool) { - assert(datum.type() == ::avro::AVRO_RECORD); - const auto& record = datum.value<::avro::GenericRecord>(); - assert(record.fieldCount() == static_cast(num_fields)); - return std::make_shared(record, pool); - } -}; - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_direct_decoder.cpp b/src/paimon/format/avro/avro_direct_decoder.cpp new file mode 100644 index 00000000..dc47fe41 --- /dev/null +++ b/src/paimon/format/avro/avro_direct_decoder.cpp @@ -0,0 +1,449 @@ +/* + * 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_decoder.cc + +#include "paimon/format/avro/avro_direct_decoder.h" + +#include "arrow/api.h" +#include "arrow/util/checked_cast.h" +#include "avro/Decoder.hh" +#include "avro/Node.hh" +#include "avro/Types.hh" +#include "paimon/common/utils/arrow/status_utils.h" +#include "paimon/common/utils/date_time_utils.h" +#include "paimon/format/avro/avro_utils.h" + +namespace paimon::avro { + +namespace { + +/// Forward declaration for mutual recursion. +Status DecodeFieldToBuilder(const ::avro::NodePtr& avro_node, + const std::optional>& projection, + ::avro::Decoder* decoder, arrow::ArrayBuilder* array_builder, + AvroDirectDecoder::DecodeContext* ctx); + +/// \brief Skip an Avro value based on its schema without decoding +Status SkipAvroValue(const ::avro::NodePtr& avro_node, ::avro::Decoder* decoder) { + switch (avro_node->type()) { + case ::avro::AVRO_NULL: + decoder->decodeNull(); + return Status::OK(); + + case ::avro::AVRO_BOOL: + decoder->decodeBool(); + return Status::OK(); + + case ::avro::AVRO_INT: + decoder->decodeInt(); + return Status::OK(); + + case ::avro::AVRO_LONG: + decoder->decodeLong(); + return Status::OK(); + + case ::avro::AVRO_FLOAT: + decoder->decodeFloat(); + return Status::OK(); + + case ::avro::AVRO_DOUBLE: + decoder->decodeDouble(); + return Status::OK(); + + case ::avro::AVRO_STRING: + decoder->skipString(); + return Status::OK(); + + case ::avro::AVRO_BYTES: + 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) { + PAIMON_RETURN_NOT_OK(SkipAvroValue(avro_node->leafAt(i), 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 + int64_t block_count = decoder->skipArray(); + while (block_count > 0) { + for (int64_t i = 0; i < block_count; ++i) { + PAIMON_RETURN_NOT_OK(SkipAvroValue(element_node, decoder)); + } + block_count = decoder->arrayNext(); + } + return Status::OK(); + } + + case ::avro::AVRO_MAP: { + const auto& value_node = avro_node->leafAt(1); + // skipMap() returns count like mapStart(), must handle all blocks + int64_t block_count = decoder->skipMap(); + while (block_count > 0) { + for (int64_t i = 0; i < block_count; ++i) { + decoder->skipString(); // Skip key (always string in Avro maps) + PAIMON_RETURN_NOT_OK(SkipAvroValue(value_node, decoder)); + } + block_count = decoder->mapNext(); + } + return Status::OK(); + } + + case ::avro::AVRO_UNION: { + const size_t branch_index = decoder->decodeUnionIndex(); + // Validate branch index + const size_t num_branches = avro_node->leaves(); + if (branch_index >= num_branches) { + return Status::Invalid(fmt::format("Union branch index {} out of range [0, {})", + branch_index, num_branches)); + } + return SkipAvroValue(avro_node->leafAt(branch_index), decoder); + } + + default: + return Status::Invalid(fmt::format("Unsupported Avro type for skipping: {}", + AvroUtils::ToString(avro_node))); + } +} + +/// Decode Avro record directly to Arrow struct builder. +Status DecodeStructToBuilder(const ::avro::NodePtr& avro_node, + const std::optional>& projection, + ::avro::Decoder* decoder, arrow::ArrayBuilder* array_builder, + AvroDirectDecoder::DecodeContext* ctx) { + if (avro_node->type() != ::avro::AVRO_RECORD) { + return Status::Invalid( + fmt::format("Expected Avro record, got type: {}", AvroUtils::ToString(avro_node))); + } + + auto* struct_builder = arrow::internal::checked_cast(array_builder); + PAIMON_RETURN_NOT_OK_FROM_ARROW(struct_builder->Append()); + + size_t skipped_fields = 0; + // Read all Avro fields in order (must maintain decoder position) + for (size_t avro_idx = 0; avro_idx < avro_node->leaves(); ++avro_idx) { + if (projection && projection->find(avro_idx) == projection->end()) { + skipped_fields++; + PAIMON_RETURN_NOT_OK(SkipAvroValue(avro_node->leafAt(avro_idx), decoder)); + } else { + // Decode this field + const auto& avro_field_node = avro_node->leafAt(avro_idx); + auto* field_builder = struct_builder->field_builder(avro_idx - skipped_fields); + PAIMON_RETURN_NOT_OK(DecodeFieldToBuilder(avro_field_node, /*projection=*/std::nullopt, + decoder, field_builder, ctx)); + } + } + + return Status::OK(); +} + +/// Decode Avro array directly to Arrow list builder. +Status DecodeListToBuilder(const ::avro::NodePtr& avro_node, ::avro::Decoder* decoder, + arrow::ArrayBuilder* array_builder, + AvroDirectDecoder::DecodeContext* ctx) { + if (avro_node->type() != ::avro::AVRO_ARRAY) { + return Status::Invalid( + fmt::format("Expected Avro array, got type: {}", AvroUtils::ToString(avro_node))); + } + + auto* list_builder = arrow::internal::checked_cast(array_builder); + PAIMON_RETURN_NOT_OK_FROM_ARROW(list_builder->Append()); + + auto* value_builder = list_builder->value_builder(); + const auto& element_node = avro_node->leafAt(0); + + // Read array block count + int64_t block_count = decoder->arrayStart(); + while (block_count != 0) { + for (int64_t i = 0; i < block_count; ++i) { + PAIMON_RETURN_NOT_OK(DecodeFieldToBuilder(element_node, /*projection=*/std::nullopt, + decoder, value_builder, ctx)); + } + block_count = decoder->arrayNext(); + } + + return Status::OK(); +} + +/// Decode Avro map directly to Arrow map builder. +Status DecodeMapToBuilder(const ::avro::NodePtr& avro_node, ::avro::Decoder* decoder, + arrow::ArrayBuilder* array_builder, + AvroDirectDecoder::DecodeContext* ctx) { + auto* map_builder = arrow::internal::checked_cast(array_builder); + + if (avro_node->type() == ::avro::AVRO_MAP) { + // Handle regular Avro map: map + const auto& key_node = avro_node->leafAt(0); + const auto& value_node = avro_node->leafAt(1); + + PAIMON_RETURN_NOT_OK_FROM_ARROW(map_builder->Append()); + auto* key_builder = map_builder->key_builder(); + auto* item_builder = map_builder->item_builder(); + + // Read map block count + int64_t block_count = decoder->mapStart(); + while (block_count != 0) { + for (int64_t i = 0; i < block_count; ++i) { + PAIMON_RETURN_NOT_OK(DecodeFieldToBuilder(key_node, /*projection=*/std::nullopt, + decoder, key_builder, ctx)); + PAIMON_RETURN_NOT_OK(DecodeFieldToBuilder(value_node, /*projection=*/std::nullopt, + decoder, item_builder, ctx)); + } + block_count = decoder->mapNext(); + } + return Status::OK(); + } else if (avro_node->type() == ::avro::AVRO_ARRAY && AvroUtils::HasMapLogicalType(avro_node)) { + // Handle array-based map: list> + PAIMON_RETURN_NOT_OK_FROM_ARROW(map_builder->Append()); + auto* key_builder = map_builder->key_builder(); + auto* item_builder = map_builder->item_builder(); + + const auto& record_node = avro_node->leafAt(0); + if (record_node->type() != ::avro::AVRO_RECORD || record_node->leaves() != 2) { + return Status::Invalid( + fmt::format("Array-based map must contain records with exactly 2 fields, got: {}", + AvroUtils::ToString(record_node))); + } + const auto& key_node = record_node->leafAt(0); + const auto& value_node = record_node->leafAt(1); + + // Read array block count + int64_t block_count = decoder->arrayStart(); + while (block_count != 0) { + for (int64_t i = 0; i < block_count; ++i) { + PAIMON_RETURN_NOT_OK(DecodeFieldToBuilder(key_node, /*projection=*/std::nullopt, + decoder, key_builder, ctx)); + PAIMON_RETURN_NOT_OK(DecodeFieldToBuilder(value_node, /*projection=*/std::nullopt, + decoder, item_builder, ctx)); + } + block_count = decoder->arrayNext(); + } + return Status::OK(); + } else { + return Status::Invalid( + fmt::format("Expected Avro map or array with map logical type, got: {}", + AvroUtils::ToString(avro_node))); + } +} + +/// Decode Avro data directly to Arrow array builder. +Status DecodeAvroValueToBuilder(const ::avro::NodePtr& avro_node, + const std::optional>& projection, + ::avro::Decoder* decoder, arrow::ArrayBuilder* array_builder, + AvroDirectDecoder::DecodeContext* ctx) { + auto type = avro_node->type(); + auto logical_type = avro_node->logicalType(); + + switch (type) { + case ::avro::AVRO_BOOL: { + auto* builder = arrow::internal::checked_cast(array_builder); + bool value = decoder->decodeBool(); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + + case ::avro::AVRO_INT: { + int32_t value = decoder->decodeInt(); + auto arrow_type = array_builder->type(); + switch (arrow_type->id()) { + case arrow::Type::INT8: { + auto* builder = + arrow::internal::checked_cast(array_builder); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + case arrow::Type::INT16: { + auto* builder = + arrow::internal::checked_cast(array_builder); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + case arrow::Type::INT32: { + auto* builder = + arrow::internal::checked_cast(array_builder); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + case arrow::Type::DATE32: { + if (logical_type.type() != ::avro::LogicalType::Type::DATE) { + return Status::TypeError( + fmt::format("Unexpected avro type [{}] with arrow type [{}].", type, + arrow_type->ToString())); + } + auto* builder = + arrow::internal::checked_cast(array_builder); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + default: + return Status::TypeError( + fmt::format("Unexpected avro type [{}] with arrow type [{}].", type, + arrow_type->ToString())); + } + } + + case ::avro::AVRO_LONG: { + int64_t value = decoder->decodeLong(); + switch (logical_type.type()) { + case ::avro::LogicalType::Type::NONE: { + auto* builder = + arrow::internal::checked_cast(array_builder); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + case ::avro::LogicalType::Type::TIMESTAMP_MILLIS: + case ::avro::LogicalType::Type::TIMESTAMP_MICROS: + case ::avro::LogicalType::Type::TIMESTAMP_NANOS: + case ::avro::LogicalType::Type::LOCAL_TIMESTAMP_MILLIS: + case ::avro::LogicalType::Type::LOCAL_TIMESTAMP_MICROS: + case ::avro::LogicalType::Type::LOCAL_TIMESTAMP_NANOS: { + auto* builder = + arrow::internal::checked_cast(array_builder); + auto ts_type = + arrow::internal::checked_cast(builder->type().get()); + // for arrow second, we need to convert it from avro millisecond + if (ts_type->unit() == arrow::TimeUnit::type::SECOND) { + value /= DateTimeUtils::CONVERSION_FACTORS[DateTimeUtils::MILLISECOND]; + } + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + default: + return Status::TypeError( + fmt::format("Unexpected avro type [{}] with arrow type [{}].", type, + array_builder->type()->ToString())); + } + } + + case ::avro::AVRO_FLOAT: { + auto* builder = arrow::internal::checked_cast(array_builder); + float value = decoder->decodeFloat(); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + case ::avro::AVRO_DOUBLE: { + auto* builder = arrow::internal::checked_cast(array_builder); + double value = decoder->decodeDouble(); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(value)); + return Status::OK(); + } + case ::avro::AVRO_STRING: { + auto* builder = arrow::internal::checked_cast(array_builder); + decoder->decodeString(ctx->string_scratch); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(ctx->string_scratch)); + return Status::OK(); + } + + case ::avro::AVRO_BYTES: { + decoder->decodeBytes(ctx->bytes_scratch); + switch (logical_type.type()) { + case ::avro::LogicalType::Type::NONE: { + auto* builder = + arrow::internal::checked_cast(array_builder); + PAIMON_RETURN_NOT_OK_FROM_ARROW( + builder->Append(ctx->bytes_scratch.data(), + static_cast(ctx->bytes_scratch.size()))); + return Status::OK(); + } + case ::avro::LogicalType::Type::DECIMAL: { + auto* builder = + arrow::internal::checked_cast(array_builder); + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW( + arrow::Decimal128 decimal, + arrow::Decimal128::FromBigEndian(ctx->bytes_scratch.data(), + ctx->bytes_scratch.size())); + PAIMON_RETURN_NOT_OK_FROM_ARROW(builder->Append(decimal)); + return Status::OK(); + } + default: + return Status::TypeError( + fmt::format("Unexpected avro type [{}] with arrow type [{}].", type, + array_builder->type()->ToString())); + } + } + + case ::avro::AVRO_RECORD: { + return DecodeStructToBuilder(avro_node, projection, decoder, array_builder, ctx); + } + case ::avro::AVRO_ARRAY: { + if (AvroUtils::HasMapLogicalType(avro_node)) { + return DecodeMapToBuilder(avro_node, decoder, array_builder, ctx); + } else { + return DecodeListToBuilder(avro_node, decoder, array_builder, ctx); + } + } + case ::avro::AVRO_MAP: { + return DecodeMapToBuilder(avro_node, decoder, array_builder, ctx); + } + default: + return Status::Invalid(fmt::format("Unsupported avro type: {}", type)); + } +} + +Status DecodeFieldToBuilder(const ::avro::NodePtr& avro_node, + const std::optional>& projection, + ::avro::Decoder* decoder, arrow::ArrayBuilder* array_builder, + AvroDirectDecoder::DecodeContext* ctx) { + if (avro_node->type() == ::avro::AVRO_UNION) { + const size_t branch_index = decoder->decodeUnionIndex(); + + // Validate branch index + const size_t num_branches = avro_node->leaves(); + if (branch_index >= num_branches) { + return Status::Invalid(fmt::format("Union branch index {} out of range [0, {})", + branch_index, num_branches)); + } + + const auto& branch_node = avro_node->leafAt(branch_index); + if (branch_node->type() == ::avro::AVRO_NULL) { + PAIMON_RETURN_NOT_OK_FROM_ARROW(array_builder->AppendNull()); + return Status::OK(); + } else { + return DecodeFieldToBuilder(branch_node, projection, decoder, array_builder, ctx); + } + } + + return DecodeAvroValueToBuilder(avro_node, projection, decoder, array_builder, ctx); +} + +} // namespace + +Status AvroDirectDecoder::DecodeAvroToBuilder(const ::avro::NodePtr& avro_node, + const std::optional>& projection, + ::avro::Decoder* decoder, + arrow::ArrayBuilder* array_builder, + DecodeContext* ctx) { + return DecodeFieldToBuilder(avro_node, projection, decoder, array_builder, ctx); +} + +} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_direct_decoder.h b/src/paimon/format/avro/avro_direct_decoder.h new file mode 100644 index 00000000..a5bb7cde --- /dev/null +++ b/src/paimon/format/avro/avro_direct_decoder.h @@ -0,0 +1,63 @@ +/* + * 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_decoder_internal.h + +#pragma once + +#include + +#include "arrow/array/builder_base.h" +#include "avro/Decoder.hh" +#include "avro/Node.hh" +#include "paimon/status.h" + +namespace paimon::avro { + +class AvroDirectDecoder { + public: + /// Context for reusing scratch buffers during Avro decoding + /// + /// Avoids frequent small allocations by reusing temporary buffers across multiple decode + /// operations. This is particularly important for string, binary, and decimal data types. + struct DecodeContext { + // Scratch buffer for string decoding (reused across rows) + std::string string_scratch; + // Scratch buffer for binary/decimal data (reused across rows) + std::vector bytes_scratch; + }; + + /// Directly decode Avro data to Arrow array builders without GenericDatum + /// + /// Eliminates the GenericDatum intermediate layer by directly calling Avro decoder + /// methods and immediately appending to Arrow builders. + /// + /// @param avro_node The Avro schema node for the data being decoded + /// @param decoder The Avro decoder positioned at the data to read + /// @param array_builder The Arrow array builder to append decoded data to + /// @param ctx Decode context for reusing scratch buffers + /// @return Status indicating success, or an error status + static Status DecodeAvroToBuilder(const ::avro::NodePtr& avro_node, + const std::optional>& projection, + ::avro::Decoder* decoder, arrow::ArrayBuilder* array_builder, + DecodeContext* 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 c5c02a45..3d068d99 100644 --- a/src/paimon/format/avro/avro_file_batch_reader.cpp +++ b/src/paimon/format/avro/avro_file_batch_reader.cpp @@ -18,24 +18,34 @@ #include #include -#include #include "arrow/c/bridge.h" -#include "avro/Generic.hh" -#include "avro/GenericDatum.hh" #include "fmt/format.h" +#include "paimon/common/metrics/metrics_impl.h" +#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/format/avro/avro_input_stream_impl.h" #include "paimon/format/avro/avro_schema_converter.h" #include "paimon/reader/batch_reader.h" namespace paimon::avro { -AvroFileBatchReader::AvroFileBatchReader( - std::unique_ptr<::avro::DataFileReader<::avro::GenericDatum>>&& reader, - std::unique_ptr&& record_converter, int32_t batch_size) - : reader_(std::move(reader)), - record_converter_(std::move(record_converter)), - batch_size_(batch_size) {} +AvroFileBatchReader::AvroFileBatchReader(const std::shared_ptr& input_stream, + const std::shared_ptr<::arrow::DataType>& file_data_type, + std::unique_ptr<::avro::DataFileReaderBase>&& reader, + std::unique_ptr&& array_builder, + std::unique_ptr&& arrow_pool, + int32_t batch_size, + const std::shared_ptr& pool) + : pool_(pool), + arrow_pool_(std::move(arrow_pool)), + input_stream_(input_stream), + file_data_type_(file_data_type), + reader_(std::move(reader)), + array_builder_(std::move(array_builder)), + batch_size_(batch_size), + metrics_(std::make_shared()) {} AvroFileBatchReader::~AvroFileBatchReader() { DoClose(); @@ -49,44 +59,68 @@ void AvroFileBatchReader::DoClose() { } Result> AvroFileBatchReader::Create( - std::unique_ptr<::avro::DataFileReader<::avro::GenericDatum>>&& reader, int32_t batch_size, + const std::shared_ptr& input_stream, int32_t batch_size, const std::shared_ptr& pool) { if (batch_size <= 0) { return Status::Invalid( fmt::format("invalid batch size {}, must be larger than 0", batch_size)); } - const auto& avro_read_schema = reader->readerSchema(); - PAIMON_ASSIGN_OR_RAISE(std::shared_ptr<::arrow::DataType> arrow_data_type, - AvroSchemaConverter::AvroSchemaToArrowDataType(avro_read_schema)); - PAIMON_ASSIGN_OR_RAISE(std::unique_ptr record_converter, - AvroRecordConverter::Create(arrow_data_type, pool)); + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr<::avro::DataFileReaderBase> reader, + CreateDataFileReader(input_stream, pool)); + const auto& avro_file_schema = reader->dataSchema(); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr<::arrow::DataType> file_data_type, + AvroSchemaConverter::AvroSchemaToArrowDataType(avro_file_schema)); + auto arrow_pool = GetArrowPool(pool); + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW(std::unique_ptr array_builder, + arrow::MakeBuilder(file_data_type, arrow_pool.get())); return std::unique_ptr( - new AvroFileBatchReader(std::move(reader), std::move(record_converter), batch_size)); + new AvroFileBatchReader(input_stream, file_data_type, std::move(reader), + std::move(array_builder), std::move(arrow_pool), batch_size, pool)); +} + +Result> AvroFileBatchReader::CreateDataFileReader( + const std::shared_ptr& input_stream, const std::shared_ptr& pool) { + PAIMON_RETURN_NOT_OK(input_stream->Seek(0, SeekOrigin::FS_SEEK_SET)); + try { + PAIMON_ASSIGN_OR_RAISE(std::unique_ptr<::avro::InputStream> in, + AvroInputStreamImpl::Create(input_stream, BUFFER_SIZE, pool)); + auto reader = std::make_unique<::avro::DataFileReaderBase>(std::move(in)); + reader->init(); + return reader; + } catch (const ::avro::Exception& e) { + return Status::Invalid(fmt::format("build avro reader failed. {}", e.what())); + } catch (const std::exception& e) { + return Status::Invalid(fmt::format("build avro reader failed. {}", e.what())); + } catch (...) { + return Status::Invalid("build avro reader failed. unknown error"); + } } Result AvroFileBatchReader::NextBatch() { - std::vector<::avro::GenericDatum> datums; - datums.reserve(batch_size_); + if (next_row_to_read_ == std::numeric_limits::max()) { + next_row_to_read_ = 0; + } try { - for (int32_t i = 0; i < batch_size_; i++) { - ::avro::GenericDatum datum(reader_->readerSchema()); - if (!reader_->read(datum)) { - // reach eof + while (array_builder_->length() < batch_size_) { + if (!reader_->hasMore()) { break; } - if (datum.type() != ::avro::AVRO_RECORD) { - return Status::Invalid( - fmt::format("avro reader next batch failed. invalid datum type: {}", - ::avro::toString(datum.type()))); - } - datums.emplace_back(datum); + reader_->decr(); + PAIMON_RETURN_NOT_OK(AvroDirectDecoder::DecodeAvroToBuilder( + reader_->dataSchema().root(), read_fields_projection_, &reader_->decoder(), + array_builder_.get(), &decode_context_)); } - if (datums.empty()) { + previous_first_row_ = next_row_to_read_; + next_row_to_read_ += array_builder_->length(); + if (array_builder_->length() == 0) { return BatchReader::MakeEofBatch(); } - // TODO(jinli.zjw) when support SetReadSchema(), may need convert file timestamp (milli) to - // target read type timestamp(second) - return record_converter_->NextBatch(datums); + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW(std::shared_ptr array, + array_builder_->Finish()); + std::unique_ptr c_array = std::make_unique(); + std::unique_ptr c_schema = std::make_unique(); + PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportArray(*array, c_array.get(), c_schema.get())); + return make_pair(std::move(c_array), std::move(c_schema)); } catch (const ::avro::Exception& e) { return Status::Invalid(fmt::format("avro reader next batch failed. {}", e.what())); } catch (const std::exception& e) { @@ -99,28 +133,49 @@ Result AvroFileBatchReader::NextBatch() { Status AvroFileBatchReader::SetReadSchema(::ArrowSchema* read_schema, const std::shared_ptr& predicate, const std::optional& selection_bitmap) { - assert(false); - return Status::NotImplemented("avro reader not support set read schema"); + if (!read_schema) { + return Status::Invalid("SetReadSchema failed: read schema cannot be nullptr"); + } + // TODO(menglingda.mld): support predicate + if (selection_bitmap) { + // TODO(menglingda.mld): support bitmap + } + previous_first_row_ = std::numeric_limits::max(); + next_row_to_read_ = std::numeric_limits::max(); + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW(std::shared_ptr arrow_read_schema, + arrow::ImportSchema(read_schema)); + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr file_schema, + ArrowUtils::DataTypeToSchema(file_data_type_)); + PAIMON_ASSIGN_OR_RAISE(read_fields_projection_, + CalculateReadFieldsProjection(file_schema, arrow_read_schema->fields())); + array_builder_->Reset(); + std::shared_ptr<::arrow::DataType> read_data_type = arrow::struct_(arrow_read_schema->fields()); + PAIMON_ASSIGN_OR_RAISE_FROM_ARROW(array_builder_, + arrow::MakeBuilder(read_data_type, arrow_pool_.get())); + return Status::OK(); +} + +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); + int32_t prev_index = -1; + for (auto& index : projection) { + if (index <= prev_index) { + return Status::Invalid( + "SetReadSchema failed: read schema fields order is different from file schema"); + } + prev_index = index; + projection_set.insert(index); + } + return projection_set; } Result> AvroFileBatchReader::GetFileSchema() const { assert(reader_); - try { - const auto& avro_file_schema = reader_->dataSchema(); - bool nullable = false; - PAIMON_ASSIGN_OR_RAISE( - std::shared_ptr arrow_file_type, - AvroSchemaConverter::GetArrowType(avro_file_schema.root(), &nullable)); - auto c_schema = std::make_unique<::ArrowSchema>(); - PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportType(*arrow_file_type, c_schema.get())); - return c_schema; - } catch (const ::avro::Exception& e) { - return Status::Invalid(fmt::format("get file schema failed. {}", e.what())); - } catch (const std::exception& e) { - return Status::Invalid(fmt::format("get file schema batch failed. {}", e.what())); - } catch (...) { - return Status::Invalid("get file schema failed. unknown error"); - } + auto c_schema = std::make_unique<::ArrowSchema>(); + PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::ExportType(*file_data_type_, c_schema.get())); + return c_schema; } } // 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 08645b92..495c4a2a 100644 --- a/src/paimon/format/avro/avro_file_batch_reader.h +++ b/src/paimon/format/avro/avro_file_batch_reader.h @@ -17,12 +17,14 @@ #pragma once #include +#include #include #include #include "avro/DataFile.hh" -#include "paimon/format/avro/avro_record_converter.h" +#include "paimon/format/avro/avro_direct_decoder.h" #include "paimon/memory/memory_pool.h" +#include "paimon/metrics.h" #include "paimon/reader/file_batch_reader.h" #include "paimon/result.h" @@ -31,7 +33,7 @@ namespace paimon::avro { class AvroFileBatchReader : public FileBatchReader { public: static Result> Create( - std::unique_ptr<::avro::DataFileReader<::avro::GenericDatum>>&& reader, int32_t batch_size, + const std::shared_ptr& input_stream, int32_t batch_size, const std::shared_ptr& pool); ~AvroFileBatchReader() override; @@ -44,8 +46,7 @@ class AvroFileBatchReader : public FileBatchReader { const std::optional& selection_bitmap) override; uint64_t GetPreviousBatchFirstRowNumber() const override { - assert(false); - return -1; + return previous_first_row_; } uint64_t GetNumberOfRows() const override { @@ -54,8 +55,7 @@ class AvroFileBatchReader : public FileBatchReader { } std::shared_ptr GetReaderMetrics() const override { - assert(false); - return nullptr; + return metrics_; } void Close() override { @@ -69,14 +69,35 @@ class AvroFileBatchReader : public FileBatchReader { private: void DoClose(); - AvroFileBatchReader(std::unique_ptr<::avro::DataFileReader<::avro::GenericDatum>>&& reader, - std::unique_ptr&& record_converter, - int32_t batch_size); - - std::unique_ptr<::avro::DataFileReader<::avro::GenericDatum>> reader_; - std::unique_ptr record_converter_; + static Result> CreateDataFileReader( + const std::shared_ptr& input_stream, const std::shared_ptr& pool); + + static Result> CalculateReadFieldsProjection( + const std::shared_ptr<::arrow::Schema>& file_schema, const arrow::FieldVector& read_fields); + + AvroFileBatchReader(const std::shared_ptr& input_stream, + const std::shared_ptr<::arrow::DataType>& file_data_type, + std::unique_ptr<::avro::DataFileReaderBase>&& reader, + std::unique_ptr&& array_builder, + std::unique_ptr&& arrow_pool, int32_t batch_size, + const std::shared_ptr& pool); + + static constexpr size_t BUFFER_SIZE = 1024 * 1024; // 1M + + std::shared_ptr pool_; + std::unique_ptr arrow_pool_; + std::shared_ptr input_stream_; + std::shared_ptr<::arrow::DataType> file_data_type_; + std::unique_ptr<::avro::DataFileReaderBase> reader_; + std::unique_ptr array_builder_; + std::optional> read_fields_projection_; + uint64_t previous_first_row_ = std::numeric_limits::max(); + uint64_t next_row_to_read_ = std::numeric_limits::max(); const int32_t batch_size_; bool close_ = false; + std::shared_ptr metrics_; + // Decode context for reusing scratch buffers + AvroDirectDecoder::DecodeContext decode_context_; }; } // namespace paimon::avro 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 6ab2639c..5f03eade 100644 --- a/src/paimon/format/avro/avro_file_batch_reader_test.cpp +++ b/src/paimon/format/avro/avro_file_batch_reader_test.cpp @@ -215,30 +215,27 @@ TEST_P(AvroFileBatchReaderTest, TestReadTimestampTypes) { ASSERT_OK_AND_ASSIGN(std::shared_ptr in, fs_->Open(path)); ASSERT_OK_AND_ASSIGN(auto batch_reader, reader_builder->Build(in)); - // check file schema - ASSERT_OK_AND_ASSIGN(auto c_file_schema, batch_reader->GetFileSchema()); - auto result_file_schema = arrow::ImportSchema(c_file_schema.get()).ValueOr(nullptr); - ASSERT_TRUE(result_file_schema); auto timezone = DateTimeUtils::GetLocalTimezoneName(); - arrow::FieldVector fields = { - arrow::field("ts_sec", arrow::timestamp(arrow::TimeUnit::MILLI)), + arrow::FieldVector read_fields = { + arrow::field("ts_sec", arrow::timestamp(arrow::TimeUnit::SECOND)), arrow::field("ts_milli", arrow::timestamp(arrow::TimeUnit::MILLI)), arrow::field("ts_micro", arrow::timestamp(arrow::TimeUnit::MICRO)), - arrow::field("ts_tz_sec", arrow::timestamp(arrow::TimeUnit::MILLI, timezone)), + arrow::field("ts_tz_sec", arrow::timestamp(arrow::TimeUnit::SECOND, timezone)), arrow::field("ts_tz_milli", arrow::timestamp(arrow::TimeUnit::MILLI, timezone)), arrow::field("ts_tz_micro", arrow::timestamp(arrow::TimeUnit::MICRO, timezone)), }; - auto expected_file_schema = arrow::schema(fields); - ASSERT_TRUE(result_file_schema->Equals(expected_file_schema)) << result_file_schema->ToString(); + auto read_schema = arrow::schema(read_fields); + std::unique_ptr c_schema = std::make_unique(); + ASSERT_TRUE(arrow::ExportSchema(*read_schema, c_schema.get()).ok()); + EXPECT_OK(batch_reader->SetReadSchema(c_schema.get(), /*predicate=*/nullptr, + /*selection_bitmap=*/std::nullopt)); // check array ASSERT_OK_AND_ASSIGN(auto result_array, ::paimon::test::ReadResultCollector::CollectResult(batch_reader.get())); - // TODO(jinli.zjw) after support SetReadSchema, need change ts_sec/ts_tz_sec type from milli - // to second std::shared_ptr expected_array; auto array_status = - arrow::ipc::internal::json::ChunkedArrayFromJSON(arrow::struct_(fields), {R"([ + arrow::ipc::internal::json::ChunkedArrayFromJSON(arrow::struct_(read_fields), {R"([ ["1970-01-01T00:00:01","1970-01-01T00:00:00.001","1970-01-01T00:00:00.000001","1970-01-01T00:00:02","1970-01-01T00:00:00.002","1970-01-01T00:00:00.000002"], [null,"1970-01-01T00:00:00.003",null,null,"1970-01-01T00:00:00.004",null], ["1970-01-01T00:00:05",null,"1970-01-01T00:00:00.000005","1970-01-01T00:00:06",null,"1970-01-01T00:00:00.000006"] @@ -249,6 +246,104 @@ TEST_P(AvroFileBatchReaderTest, TestReadTimestampTypes) { ASSERT_TRUE(expected_array->Equals(result_array)); } +TEST_P(AvroFileBatchReaderTest, TestReadMapTypes) { + std::string path = paimon::test::GetDataDir() + + "/avro/append_with_multiple_map.db/" + "append_with_multiple_map/bucket-0/" + "data-72442742-e49e-48a4-a736-a2475aac2d2c-0.avro"; + + ASSERT_OK_AND_ASSIGN(auto reader_builder, + file_format_->CreateReaderBuilder(/*batch_size=*/1024)); + ASSERT_OK_AND_ASSIGN(std::shared_ptr in, fs_->Open(path)); + ASSERT_OK_AND_ASSIGN(auto batch_reader, reader_builder->Build(in)); + + arrow::FieldVector read_fields = { + arrow::field("f0", arrow::map(arrow::int32(), arrow::int32())), + arrow::field("f1", arrow::map(arrow::float64(), arrow::float64())), + arrow::field("f2", arrow::map(arrow::utf8(), arrow::utf8())), + arrow::field("f3", arrow::map(arrow::utf8(), arrow::binary())), + arrow::field("f4", arrow::map(arrow::timestamp(arrow::TimeUnit::MICRO), + arrow::timestamp(arrow::TimeUnit::MICRO))), + arrow::field("f5", arrow::map(arrow::utf8(), arrow::list(arrow::float64()))), + arrow::field("f6", arrow::map(arrow::utf8(), arrow::map(arrow::float64(), arrow::utf8()))), + arrow::field("f7", arrow::map(arrow::int64(), + arrow::struct_({field("f0", arrow::int32()), + field("f1", arrow::utf8()), + field("f2", arrow::decimal128(5, 2))})))}; + auto read_schema = arrow::schema(read_fields); + std::unique_ptr c_schema = std::make_unique(); + ASSERT_TRUE(arrow::ExportSchema(*read_schema, c_schema.get()).ok()); + EXPECT_OK(batch_reader->SetReadSchema(c_schema.get(), /*predicate=*/nullptr, + /*selection_bitmap=*/std::nullopt)); + + // check array + ASSERT_OK_AND_ASSIGN(auto result_array, + ::paimon::test::ReadResultCollector::CollectResult(batch_reader.get())); + std::shared_ptr expected_array; + auto array_status = + arrow::ipc::internal::json::ChunkedArrayFromJSON(arrow::struct_(read_fields), {R"([ + [ + [[1,10],[2,20]], + [[1.1,10.1],[2.2,20.2]], + [["key1","val1"],["key2","val2"]], + [["123456","abcdef"]], + [["2023-01-01 12:00:00.123000","2023-01-01 12:00:00.123000"],["2023-01-02 13:30:00.456000","2023-01-02 13:30:00.456000"]], + [["arr_key",[1.5, 2.5, 3.5]]], + [["outer_key",[[99.9,"nested_val"]]]], + [[1000, [42, "row_str", "123.45"]]] + ] + ])"}, + &expected_array); + ASSERT_TRUE(array_status.ok()) << array_status.ToString(); + ASSERT_TRUE(result_array->Equals(expected_array)) << result_array->ToString() << std::endl; + ASSERT_TRUE(expected_array->Equals(result_array)); +} + +TEST_P(AvroFileBatchReaderTest, TestReadRowNumbers) { + std::string path = paimon::test::GetDataDir() + + "/avro/append_simple.db/" + "append_simple/bucket-0/" + "data-d7d1c416-6e34-4834-af87-341d09418f0c-0.avro"; + + ASSERT_OK_AND_ASSIGN(auto reader_builder, file_format_->CreateReaderBuilder(/*batch_size=*/1)); + ASSERT_OK_AND_ASSIGN(std::shared_ptr in, fs_->Open(path)); + ASSERT_OK_AND_ASSIGN(auto reader, reader_builder->Build(in)); + + arrow::FieldVector read_fields = { + arrow::field("f0", arrow::int32()), arrow::field("f1", arrow::float64()), + arrow::field("f2", arrow::utf8()), + arrow::field("f3", + arrow::struct_({arrow::field("f0", arrow::map(arrow::utf8(), arrow::int32())), + arrow::field("f1", arrow::list(arrow::int32()))}))}; + + auto read_schema = arrow::schema(read_fields); + std::unique_ptr c_schema = std::make_unique(); + ASSERT_TRUE(arrow::ExportSchema(*read_schema, c_schema.get()).ok()); + EXPECT_OK(reader->SetReadSchema(c_schema.get(), /*predicate=*/nullptr, + /*selection_bitmap=*/std::nullopt)); + + ASSERT_EQ(std::numeric_limits::max(), reader->GetPreviousBatchFirstRowNumber()); + ASSERT_OK_AND_ASSIGN(auto batch1, reader->NextBatch()); + ArrowArrayRelease(batch1.first.get()); + ArrowSchemaRelease(batch1.second.get()); + ASSERT_EQ(0, reader->GetPreviousBatchFirstRowNumber()); + ASSERT_OK_AND_ASSIGN(auto batch2, reader->NextBatch()); + ASSERT_EQ(1, reader->GetPreviousBatchFirstRowNumber()); + ArrowArrayRelease(batch2.first.get()); + ArrowSchemaRelease(batch2.second.get()); + ASSERT_OK_AND_ASSIGN(auto batch3, reader->NextBatch()); + ASSERT_EQ(2, reader->GetPreviousBatchFirstRowNumber()); + ArrowArrayRelease(batch3.first.get()); + ArrowSchemaRelease(batch3.second.get()); + ASSERT_OK_AND_ASSIGN(auto batch4, reader->NextBatch()); + ASSERT_EQ(3, reader->GetPreviousBatchFirstRowNumber()); + ArrowArrayRelease(batch4.first.get()); + ArrowSchemaRelease(batch4.second.get()); + ASSERT_OK_AND_ASSIGN(auto batch5, reader->NextBatch()); + ASSERT_EQ(4, reader->GetPreviousBatchFirstRowNumber()); + ASSERT_TRUE(BatchReader::IsEofBatch(batch5)); +} + 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 bfdeacce..0d9e7908 100644 --- a/src/paimon/format/avro/avro_file_format.cpp +++ b/src/paimon/format/avro/avro_file_format.cpp @@ -16,6 +16,7 @@ #include "paimon/format/avro/avro_file_format.h" +#include #include #include "arrow/c/bridge.h" diff --git a/src/paimon/format/avro/avro_file_format_factory.cpp b/src/paimon/format/avro/avro_file_format_factory.cpp index 365417e4..e11be40a 100644 --- a/src/paimon/format/avro/avro_file_format_factory.cpp +++ b/src/paimon/format/avro/avro_file_format_factory.cpp @@ -30,6 +30,11 @@ Result> AvroFileFormatFactory::Create( return std::make_unique(options); } +static __attribute__((constructor)) void AvroFileFormatFactoryRegisterLogicalTypes() { + ::avro::CustomLogicalTypeRegistry::instance().registerType( + "map", [](const std::string&) { return std::make_shared(); }); +} + REGISTER_PAIMON_FACTORY(AvroFileFormatFactory); } // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_file_format_factory.h b/src/paimon/format/avro/avro_file_format_factory.h index 64daf6ac..c01360fa 100644 --- a/src/paimon/format/avro/avro_file_format_factory.h +++ b/src/paimon/format/avro/avro_file_format_factory.h @@ -20,12 +20,17 @@ #include #include +#include "avro/LogicalType.hh" #include "paimon/format/file_format.h" #include "paimon/format/file_format_factory.h" #include "paimon/result.h" namespace paimon::avro { +struct MapLogicalType : public ::avro::CustomLogicalType { + MapLogicalType() : ::avro::CustomLogicalType("map") {} +}; + class AvroFileFormatFactory : public FileFormatFactory { public: static const char IDENTIFIER[]; diff --git a/src/paimon/format/avro/avro_format_writer.cpp b/src/paimon/format/avro/avro_format_writer.cpp index 7b041e31..b19f3881 100644 --- a/src/paimon/format/avro/avro_format_writer.cpp +++ b/src/paimon/format/avro/avro_format_writer.cpp @@ -39,9 +39,6 @@ namespace arrow { class Array; } // namespace arrow -namespace avro { -class OutputStream; -} // namespace avro struct ArrowArray; namespace paimon::avro { @@ -49,28 +46,30 @@ 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) + std::unique_ptr adaptor, AvroOutputStreamImpl* avro_output_stream) : writer_(file_writer), avro_schema_(avro_schema), data_type_(data_type), - adaptor_(std::move(adaptor)) {} + adaptor_(std::move(adaptor)), + avro_output_stream_(avro_output_stream) {} Result> AvroFormatWriter::Create( - std::unique_ptr<::avro::OutputStream> out, const std::shared_ptr& schema, + std::unique_ptr out, const std::shared_ptr& schema, const ::avro::Codec codec) { try { 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 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))); + return std::unique_ptr(new AvroFormatWriter( + writer, avro_schema, data_type, std::move(adaptor), 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) { - return Status::Invalid("avro format writer create failed: {}", e.what()); + return Status::Invalid(fmt::format("avro format writer create failed: {}", e.what())); } catch (...) { return Status::Invalid("avro format writer create failed: unknown exception"); } @@ -82,7 +81,7 @@ Status AvroFormatWriter::Flush() { } catch (const ::avro::Exception& e) { return Status::Invalid(fmt::format("avro writer flush failed. {}", e.what())); } catch (const std::exception& e) { - return Status::Invalid("avro writer flush failed: {}", e.what()); + return Status::Invalid(fmt::format("avro writer flush failed: {}", e.what())); } catch (...) { return Status::Invalid("avro writer flush failed: unknown exception"); } @@ -92,11 +91,12 @@ Status AvroFormatWriter::Flush() { Status AvroFormatWriter::Finish() { try { + avro_output_stream_->FlushBuffer(); // we need flush buffer before close writer writer_->close(); } catch (const ::avro::Exception& e) { return Status::Invalid(fmt::format("avro writer close failed. {}", e.what())); } catch (const std::exception& e) { - return Status::Invalid("avro writer close failed: {}", e.what()); + return Status::Invalid(fmt::format("avro writer close failed: {}", e.what())); } catch (...) { return Status::Invalid("avro writer close failed: unknown exception"); } @@ -120,7 +120,7 @@ Status AvroFormatWriter::AddBatch(ArrowArray* batch) { } catch (const ::avro::Exception& e) { return Status::Invalid(fmt::format("avro writer add batch failed. {}", e.what())); } catch (const std::exception& e) { - return Status::Invalid("avro writer add batch failed: {}", e.what()); + return Status::Invalid(fmt::format("avro writer add batch failed: {}", e.what())); } catch (...) { return Status::Invalid("avro writer add batch failed: unknown exception"); } diff --git a/src/paimon/format/avro/avro_format_writer.h b/src/paimon/format/avro/avro_format_writer.h index 2701c230..17f39a8b 100644 --- a/src/paimon/format/avro/avro_format_writer.h +++ b/src/paimon/format/avro/avro_format_writer.h @@ -22,9 +22,9 @@ #include "arrow/api.h" #include "avro/DataFile.hh" -#include "avro/Stream.hh" #include "avro/ValidSchema.hh" #include "paimon/format/avro/avro_adaptor.h" +#include "paimon/format/avro/avro_output_stream_impl.h" #include "paimon/format/format_writer.h" #include "paimon/metrics.h" #include "paimon/result.h" @@ -36,7 +36,6 @@ class Schema; } // namespace arrow namespace avro { class GenericDatum; -class OutputStream; } // namespace avro namespace paimon { class Metrics; @@ -49,7 +48,7 @@ namespace paimon::avro { class AvroFormatWriter : public FormatWriter { public: static Result> Create( - std::unique_ptr<::avro::OutputStream> out, const std::shared_ptr& schema, + std::unique_ptr out, const std::shared_ptr& schema, const ::avro::Codec codec); Status AddBatch(ArrowArray* batch) override; @@ -70,13 +69,14 @@ class AvroFormatWriter : public FormatWriter { 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); + std::unique_ptr adaptor, AvroOutputStreamImpl* avro_output_stream); std::shared_ptr<::avro::DataFileWriter<::avro::GenericDatum>> writer_; ::avro::ValidSchema avro_schema_; std::shared_ptr data_type_; std::shared_ptr metrics_; std::unique_ptr adaptor_; + AvroOutputStreamImpl* avro_output_stream_; }; } // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_output_stream_impl.cpp b/src/paimon/format/avro/avro_output_stream_impl.cpp index a590bf69..91e6c741 100644 --- a/src/paimon/format/avro/avro_output_stream_impl.cpp +++ b/src/paimon/format/avro/avro_output_stream_impl.cpp @@ -44,7 +44,7 @@ AvroOutputStreamImpl::~AvroOutputStreamImpl() { bool AvroOutputStreamImpl::next(uint8_t** data, size_t* len) { if (available_ == 0) { - flush(); + FlushBuffer(); } *data = next_; *len = available_; @@ -60,7 +60,7 @@ void AvroOutputStreamImpl::backup(size_t len) { byte_count_ -= len; } -void AvroOutputStreamImpl::flush() { +void AvroOutputStreamImpl::FlushBuffer() { size_t length = buffer_size_ - available_; Result write_len = out_->Write(reinterpret_cast(buffer_), length); if (!write_len.ok()) { @@ -76,4 +76,13 @@ void AvroOutputStreamImpl::flush() { available_ = buffer_size_; } +void AvroOutputStreamImpl::flush() { + // In avro-java impl, there is an option to control flush frequency. + // See: https://github.com/apache/avro/commit/35750393891c40f0ceb925a852162ec764bcae6c + // + // However, in the avro-cpp impl, there is no such option. Calling flush() too frequently + // generates many small I/O operations, affecting write performance, so we make + // ::avro::OutputStream's flush() do nothing +} + } // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_output_stream_impl.h b/src/paimon/format/avro/avro_output_stream_impl.h index 389a7eb8..d0dfe919 100644 --- a/src/paimon/format/avro/avro_output_stream_impl.h +++ b/src/paimon/format/avro/avro_output_stream_impl.h @@ -44,6 +44,8 @@ class AvroOutputStreamImpl : public ::avro::OutputStream { return byte_count_; } + void FlushBuffer(); + private: std::shared_ptr pool_; const size_t buffer_size_; diff --git a/src/paimon/format/avro/avro_reader_builder.h b/src/paimon/format/avro/avro_reader_builder.h index 48c9fc92..aa0afd7a 100644 --- a/src/paimon/format/avro/avro_reader_builder.h +++ b/src/paimon/format/avro/avro_reader_builder.h @@ -42,19 +42,7 @@ class AvroReaderBuilder : public ReaderBuilder { Result> Build( const std::shared_ptr& path) const override { - try { - PAIMON_ASSIGN_OR_RAISE(std::unique_ptr<::avro::InputStream> in, - AvroInputStreamImpl::Create(path, BUFFER_SIZE, pool_)); - auto data_file_reader = - std::make_unique<::avro::DataFileReader<::avro::GenericDatum>>(std::move(in)); - return AvroFileBatchReader::Create(std::move(data_file_reader), batch_size_, pool_); - } catch (const ::avro::Exception& e) { - return Status::Invalid(fmt::format("build avro reader failed. {}", e.what())); - } catch (const std::exception& e) { - return Status::Invalid(fmt::format("build avro reader failed. {}", e.what())); - } catch (...) { - return Status::Invalid("build avro reader failed. unknown error"); - } + return AvroFileBatchReader::Create(path, batch_size_, pool_); } Result> Build(const std::string& path) const override { @@ -62,8 +50,6 @@ class AvroReaderBuilder : public ReaderBuilder { } private: - static constexpr size_t BUFFER_SIZE = 1024 * 1024; // 1M - const int32_t batch_size_; std::shared_ptr pool_; const std::map options_; diff --git a/src/paimon/format/avro/avro_record_converter.cpp b/src/paimon/format/avro/avro_record_converter.cpp deleted file mode 100644 index b74aaf1b..00000000 --- a/src/paimon/format/avro/avro_record_converter.cpp +++ /dev/null @@ -1,90 +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_record_converter.h" - -#include -#include -#include -#include - -#include "arrow/api.h" -#include "arrow/array/builder_base.h" -#include "arrow/array/builder_nested.h" -#include "arrow/c/abi.h" -#include "arrow/util/checked_cast.h" -#include "avro/GenericDatum.hh" -#include "paimon/common/utils/arrow/mem_utils.h" -#include "paimon/common/utils/arrow/status_utils.h" -#include "paimon/format/avro/avro_record_data_getter.h" -#include "paimon/status.h" - -namespace paimon { -class MemoryPool; -} // namespace paimon - -namespace paimon::avro { - -Result> AvroRecordConverter::Create( - const std::shared_ptr<::arrow::DataType>& type, const std::shared_ptr& pool) { - auto arrow_pool = GetArrowPool(pool); - std::unique_ptr array_builder; - PAIMON_RETURN_NOT_OK_FROM_ARROW(arrow::MakeBuilder(arrow_pool.get(), type, &array_builder)); - - auto struct_builder = - arrow::internal::checked_pointer_cast(std::move(array_builder)); - assert(struct_builder); - std::vector appenders; - // first is the root struct array - int32_t reserve_count = 1; - for (size_t i = 0; i < type->fields().size(); i++) { - PAIMON_ASSIGN_OR_RAISE( - RowToArrowArrayConverter::AppendValueFunc func, - AppendField(/*use_view=*/true, struct_builder->field_builder(i), &reserve_count)); - appenders.emplace_back(func); - } - return std::unique_ptr( - new AvroRecordConverter(reserve_count, std::move(appenders), std::move(struct_builder), - std::move(arrow_pool), type, pool)); -} - -Result AvroRecordConverter::NextBatch( - const std::vector<::avro::GenericDatum>& avro_datums) { - PAIMON_RETURN_NOT_OK(ResetAndReserve()); - PAIMON_RETURN_NOT_OK_FROM_ARROW( - array_builder_->AppendValues(avro_datums.size(), /*valid_bytes=*/nullptr)); - const auto& fields = type_->fields(); - for (size_t i = 0; i < fields.size(); i++) { - for (const auto& avro_datum : avro_datums) { - PAIMON_RETURN_NOT_OK_FROM_ARROW(appenders_[i]( - AvroRecordDataGetter(avro_datum.value<::avro::GenericRecord>(), pool_), i)); - } - } - return FinishAndAccumulate(); -} - -AvroRecordConverter::AvroRecordConverter(int32_t reserve_count, - std::vector&& appenders, - std::unique_ptr&& array_builder, - std::unique_ptr&& arrow_pool, - const std::shared_ptr<::arrow::DataType>& type, - const std::shared_ptr& pool) - : RowToArrowArrayConverter(reserve_count, std::move(appenders), std::move(array_builder), - std::move(arrow_pool)), - type_(type), - pool_(pool) {} - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_record_converter.h b/src/paimon/format/avro/avro_record_converter.h deleted file mode 100644 index 33765199..00000000 --- a/src/paimon/format/avro/avro_record_converter.h +++ /dev/null @@ -1,62 +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 "avro/GenericDatum.hh" -#include "paimon/core/io/row_to_arrow_array_converter.h" -#include "paimon/reader/batch_reader.h" -#include "paimon/result.h" - -namespace arrow { -class DataType; -class MemoryPool; -class StructBuilder; -} // namespace arrow -namespace avro { -class GenericDatum; -} // namespace avro -namespace paimon { -class MemoryPool; -} // namespace paimon - -namespace paimon::avro { - -class AvroRecordConverter - : public RowToArrowArrayConverter<::avro::GenericDatum, BatchReader::ReadBatch> { - public: - static Result> Create( - const std::shared_ptr<::arrow::DataType>& type, const std::shared_ptr& pool); - - Result NextBatch( - const std::vector<::avro::GenericDatum>& avro_datums) override; - - private: - AvroRecordConverter(int32_t reserve_count, std::vector&& appenders, - std::unique_ptr&& array_builder, - std::unique_ptr&& arrow_pool, - const std::shared_ptr<::arrow::DataType>& type, - const std::shared_ptr& pool); - - std::shared_ptr type_; - std::shared_ptr pool_; -}; - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_record_converter_test.cpp b/src/paimon/format/avro/avro_record_converter_test.cpp deleted file mode 100644 index 558f2769..00000000 --- a/src/paimon/format/avro/avro_record_converter_test.cpp +++ /dev/null @@ -1,99 +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_record_converter.h" - -#include -#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/Compiler.hh" -#include "avro/GenericDatum.hh" -#include "avro/ValidSchema.hh" -#include "gtest/gtest.h" -#include "paimon/memory/memory_pool.h" -#include "paimon/status.h" -#include "paimon/testing/utils/testharness.h" - -namespace paimon::avro::test { -class AvroRecordConverterTest : public testing::Test { - public: - std::shared_ptr CreateSimpleSchema() { - std::string schema_str = R"({ - "type": "record", - "name": "TestRecord", - "fields": [ - {"name": "id", "type": "int"}, - {"name": "name", "type": "string"}, - {"name": "is_active", "type": "boolean"} - ] - })"; - return std::make_shared( - ::avro::compileJsonSchemaFromString(schema_str)); - } - - std::vector<::avro::GenericDatum> CreateTestDatums(int count) { - auto valid_schema = CreateSimpleSchema(); - std::vector<::avro::GenericDatum> datums; - for (int i = 0; i < count; ++i) { - ::avro::GenericRecord record(valid_schema->root()); - record.setFieldAt(0, ::avro::GenericDatum(i + 1)); - record.setFieldAt(1, ::avro::GenericDatum("user_" + std::to_string(i + 1))); - record.setFieldAt(2, ::avro::GenericDatum(i % 2 == 0)); - - ::avro::GenericDatum datum(*valid_schema); - auto& value = datum.value<::avro::GenericRecord>(); - value = std::move(record); - datums.emplace_back(std::move(datum)); - } - return datums; - } - - std::shared_ptr CreateExpectedArrowSchema() { - auto id_field = arrow::field("id", arrow::int32()); - auto name_field = arrow::field("name", arrow::utf8()); - auto active_field = arrow::field("is_active", arrow::boolean()); - return arrow::schema({id_field, name_field, active_field}); - } -}; - -TEST_F(AvroRecordConverterTest, NextBatchConvertsSimpleDataCorrectly) { - auto arrow_schema = CreateExpectedArrowSchema(); - auto arrow_type = arrow::struct_({arrow_schema->fields()}); - ASSERT_OK_AND_ASSIGN(auto converter, AvroRecordConverter::Create(arrow_type, GetDefaultPool())); - - auto avro_datums = CreateTestDatums(3); - ASSERT_OK_AND_ASSIGN(auto batch, converter->NextBatch(avro_datums)); - auto [c_array, c_schema] = std::move(batch); - ASSERT_EQ(c_array->length, 3); - auto array = arrow::ImportArray(c_array.get(), c_schema.get()).ValueOr(nullptr); - ASSERT_TRUE(array); - std::string data_str = R"([ -[1, "user_1", true], -[2, "user_2", false], -[3, "user_3", true] -])"; - auto expected_array = - arrow::ipc::internal::json::ArrayFromJSON(arrow_type, data_str).ValueOr(nullptr); - ASSERT_TRUE(expected_array); - ASSERT_TRUE(expected_array->Equals(array)); -} - -} // namespace paimon::avro::test diff --git a/src/paimon/format/avro/avro_record_data_getter.cpp b/src/paimon/format/avro/avro_record_data_getter.cpp deleted file mode 100644 index 39f93db1..00000000 --- a/src/paimon/format/avro/avro_record_data_getter.cpp +++ /dev/null @@ -1,127 +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_record_data_getter.h" - -#include - -#include "avro/GenericDatum.hh" -#include "paimon/format/avro/avro_datum_data_getter.h" -#include "paimon/status.h" - -namespace paimon { -class InternalMap; -class MemoryPool; -} // namespace paimon - -namespace paimon::avro { - -AvroRecordDataGetter::AvroRecordDataGetter(const ::avro::GenericRecord& record, - const std::shared_ptr& pool) - : record_(record), pool_(pool) {} - -bool AvroRecordDataGetter::IsNullAt(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::IsNullAt(record_.fieldAt(pos)); -} -bool AvroRecordDataGetter::GetBoolean(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetBoolean(record_.fieldAt(pos)); -} -char AvroRecordDataGetter::GetByte(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetByte(record_.fieldAt(pos)); -} -int16_t AvroRecordDataGetter::GetShort(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetShort(record_.fieldAt(pos)); -} -int32_t AvroRecordDataGetter::GetInt(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetInt(record_.fieldAt(pos)); -} -int32_t AvroRecordDataGetter::GetDate(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetDate(record_.fieldAt(pos)); -} -int64_t AvroRecordDataGetter::GetLong(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetLong(record_.fieldAt(pos)); -} -float AvroRecordDataGetter::GetFloat(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetFloat(record_.fieldAt(pos)); -} -double AvroRecordDataGetter::GetDouble(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetDouble(record_.fieldAt(pos)); -} -BinaryString AvroRecordDataGetter::GetString(int32_t pos) const { - assert(false); - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetString(record_.fieldAt(pos), pool_); -} -std::string_view AvroRecordDataGetter::GetStringView(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetStringView(record_.fieldAt(pos)); -} -Decimal AvroRecordDataGetter::GetDecimal(int32_t pos, int32_t precision, int32_t scale) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetDecimal(record_.fieldAt(pos), precision, scale, pool_); -} -Timestamp AvroRecordDataGetter::GetTimestamp(int32_t pos, int32_t precision) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetTimestamp(record_.fieldAt(pos), precision); -} -std::shared_ptr AvroRecordDataGetter::GetBinary(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetBinary(record_.fieldAt(pos), pool_); -} - -std::shared_ptr AvroRecordDataGetter::GetArray(int32_t pos) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetArray(record_.fieldAt(pos), pool_); -} - -std::shared_ptr AvroRecordDataGetter::GetRow(int32_t pos, int32_t num_fields) const { - assert(pos < GetFieldCount()); - return AvroDatumDataGetter::GetRow(record_.fieldAt(pos), num_fields, pool_); -} - -int32_t AvroRecordDataGetter::GetFieldCount() const { - return record_.fieldCount(); -} - -std::shared_ptr AvroRecordDataGetter::GetMap(int32_t pos) const { - assert(false); - return nullptr; -} - -Result AvroRecordDataGetter::GetRowKind() const { - assert(false); - return Status::Invalid("avro record do not have row kind."); -} - -void AvroRecordDataGetter::SetRowKind(const RowKind* kind) { - assert(false); -} - -std::string AvroRecordDataGetter::ToString() const { - assert(false); - return ""; -} - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_record_data_getter.h b/src/paimon/format/avro/avro_record_data_getter.h deleted file mode 100644 index 6ce406fe..00000000 --- a/src/paimon/format/avro/avro_record_data_getter.h +++ /dev/null @@ -1,76 +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 "avro/GenericDatum.hh" -#include "paimon/common/data/binary_string.h" -#include "paimon/common/data/data_define.h" -#include "paimon/common/data/internal_array.h" -#include "paimon/common/data/internal_row.h" -#include "paimon/common/types/row_kind.h" -#include "paimon/data/decimal.h" -#include "paimon/data/timestamp.h" -#include "paimon/memory/bytes.h" -#include "paimon/result.h" - -namespace avro { -class GenericRecord; -} // namespace avro -namespace paimon { -class MemoryPool; -} // namespace paimon - -namespace paimon::avro { - -class AvroRecordDataGetter : public InternalRow { - public: - AvroRecordDataGetter(const ::avro::GenericRecord& record, - const std::shared_ptr& pool); - - bool IsNullAt(int32_t pos) const override; - bool GetBoolean(int32_t pos) const override; - char GetByte(int32_t pos) const override; - int16_t GetShort(int32_t pos) const override; - int32_t GetInt(int32_t pos) const override; - int32_t GetDate(int32_t pos) const override; - int64_t GetLong(int32_t pos) const override; - float GetFloat(int32_t pos) const override; - double GetDouble(int32_t pos) const override; - BinaryString GetString(int32_t pos) const override; - std::string_view GetStringView(int32_t pos) const override; - Decimal GetDecimal(int32_t pos, int32_t precision, int32_t scale) const override; - Timestamp GetTimestamp(int32_t pos, int32_t precision) const override; - std::shared_ptr GetBinary(int32_t pos) const override; - std::shared_ptr GetArray(int32_t pos) const override; - std::shared_ptr GetRow(int32_t pos, int32_t num_fields) const override; - int32_t GetFieldCount() const override; - std::shared_ptr GetMap(int32_t pos) const override; - Result GetRowKind() const override; - void SetRowKind(const RowKind* kind) override; - std::string ToString() const override; - - private: - const ::avro::GenericRecord& record_; - std::shared_ptr pool_; -}; - -} // namespace paimon::avro diff --git a/src/paimon/format/avro/avro_schema_converter.cpp b/src/paimon/format/avro/avro_schema_converter.cpp index 621e5151..e5daa4e2 100644 --- a/src/paimon/format/avro/avro_schema_converter.cpp +++ b/src/paimon/format/avro/avro_schema_converter.cpp @@ -30,9 +30,10 @@ #include "avro/ValidSchema.hh" #include "fmt/format.h" #include "paimon/common/utils/date_time_utils.h" +#include "paimon/format/avro/avro_file_format_factory.h" +#include "paimon/format/avro/avro_utils.h" #include "paimon/macros.h" #include "paimon/status.h" - namespace paimon::avro { /// Returns schema with nullable true. @@ -44,6 +45,18 @@ ::avro::Schema AvroSchemaConverter::NullableSchema(const ::avro::Schema& schema) return union_schema; } +void AvroSchemaConverter::AddRecordField(::avro::RecordSchema* record_schema, + const std::string& field_name, + const ::avro::Schema& field_schema) { + if (field_schema.type() == ::avro::Type::AVRO_UNION) { + ::avro::CustomAttributes attrs; + attrs.addAttribute("default", "null", /*addQuotes=*/false); + record_schema->addField(field_name, field_schema, attrs); + } else { + record_schema->addField(field_name, field_schema); + } +} + Result AvroSchemaConverter::CheckUnionType(const ::avro::NodePtr& avro_node) { auto type = avro_node->type(); if (type == ::avro::AVRO_UNION) { @@ -142,9 +155,39 @@ Result> AvroSchemaConverter::GetArrowType( auto timezone = DateTimeUtils::GetLocalTimezoneName(); return arrow::timestamp(arrow::TimeUnit::NANO, timezone); } + case ::avro::LogicalType::Type::CUSTOM: { + if (!AvroUtils::HasMapLogicalType(avro_node)) { + return Status::TypeError("invalid avro logical map type"); + } + if (type != ::avro::AVRO_ARRAY) { + return Status::TypeError("invalid avro logical map stored as ", toString(type)); + } + size_t subtype_count = avro_node->leaves(); + if (subtype_count != 1) { + return Status::TypeError("invalid avro logical map type"); + } + PAIMON_ASSIGN_OR_RAISE(std::shared_ptr logical_map_field, + GetArrowField("item", avro_node->leafAt(0))); + auto logical_map_type = logical_map_field->type(); + if (logical_map_type->id() != arrow::Type::STRUCT) { + return Status::TypeError("invalid avro logical map item type"); + } + auto struct_type = + arrow::internal::checked_pointer_cast(logical_map_type); + const auto& fields = struct_type->fields(); + if (fields.size() != 2) { + return Status::TypeError("invalid avro logical map struct fields size"); + } + auto key_field = fields[0]->WithNullable(false); + auto value_field = fields[1]; + if (key_field->name() != "key" || value_field->name() != "value") { + return Status::TypeError("invalid avro logical map struct field names"); + } + return std::make_shared(std::move(key_field), std::move(value_field)); + } default: - return Status::NotImplemented("not support logical type ", - std::to_string(logical_type.type())); + return Status::Invalid("invalid avro logical type: ", + AvroUtils::ToString(logical_type)); } size_t subtype_count = avro_node->leaves(); @@ -205,7 +248,7 @@ Result> AvroSchemaConverter::GetArrowType( } Result<::avro::Schema> AvroSchemaConverter::ArrowTypeToAvroSchema( - const std::shared_ptr& field) { + const std::shared_ptr& field, const std::string& row_name) { bool nullable = field->nullable(); auto arrow_type = field->type(); switch (arrow_type->id()) { @@ -232,20 +275,37 @@ Result<::avro::Schema> AvroSchemaConverter::ArrowTypeToAvroSchema( return nullable ? NullableSchema(date_schema) : date_schema; } case arrow::Type::type::TIMESTAMP: { - // TODO(jinli.zjw): support convert with multiple precision & timezone const auto& arrow_timestamp_type = arrow::internal::checked_pointer_cast(arrow_type); - if (!arrow_timestamp_type->timezone().empty()) { - return Status::Invalid("Unsupported TimestampType with timezone"); - } - if (arrow_timestamp_type->unit() != arrow::TimeUnit::type::NANO) { - return Status::Invalid("Only supported TimestampType with nano time unit"); - } - // NOTE: Java Avro only support TIMESTAMP_MILLIS && TIMESTAMP_MICROS - ::avro::LogicalType timestamp_type = - ::avro::LogicalType(::avro::LogicalType::TIMESTAMP_MICROS); + bool has_timezone = !arrow_timestamp_type->timezone().empty(); ::avro::LongSchema timestamp_schema; - timestamp_schema.root()->setLogicalType(timestamp_type); + switch (arrow_timestamp_type->unit()) { + // Avro doesn't support seconds, convert to milliseconds + case arrow::TimeUnit::type::SECOND: + case arrow::TimeUnit::type::MILLI: { + ::avro::LogicalType logical_type = ::avro::LogicalType( + has_timezone ? ::avro::LogicalType::LOCAL_TIMESTAMP_MILLIS + : ::avro::LogicalType::TIMESTAMP_MILLIS); + timestamp_schema.root()->setLogicalType(logical_type); + break; + } + case arrow::TimeUnit::type::MICRO: { + ::avro::LogicalType logical_type = ::avro::LogicalType( + has_timezone ? ::avro::LogicalType::LOCAL_TIMESTAMP_MICROS + : ::avro::LogicalType::TIMESTAMP_MICROS); + timestamp_schema.root()->setLogicalType(logical_type); + break; + } + case arrow::TimeUnit::type::NANO: { + ::avro::LogicalType logical_type = ::avro::LogicalType( + has_timezone ? ::avro::LogicalType::LOCAL_TIMESTAMP_NANOS + : ::avro::LogicalType::TIMESTAMP_NANOS); + timestamp_schema.root()->setLogicalType(logical_type); + break; + } + default: + return Status::Invalid("Unknown TimeUnit in TimestampType"); + } return nullable ? NullableSchema(timestamp_schema) : timestamp_schema; } case arrow::Type::type::DECIMAL128: { @@ -258,6 +318,56 @@ Result<::avro::Schema> AvroSchemaConverter::ArrowTypeToAvroSchema( decimal_schema.root()->setLogicalType(decimal_type); return nullable ? NullableSchema(decimal_schema) : decimal_schema; } + case arrow::Type::LIST: { + const auto& list_type = + arrow::internal::checked_pointer_cast(arrow_type); + const auto& value_field = list_type->value_field(); + PAIMON_ASSIGN_OR_RAISE(auto value_schema, ArrowTypeToAvroSchema(value_field, row_name)); + ::avro::ArraySchema array_schema(value_schema); + return nullable ? NullableSchema(array_schema) : array_schema; + } + case arrow::Type::STRUCT: { + const auto& struct_type = + arrow::internal::checked_pointer_cast(arrow_type); + const auto& fields = struct_type->fields(); + + ::avro::RecordSchema record_schema(row_name); + for (const auto& f : fields) { + PAIMON_ASSIGN_OR_RAISE(auto field_schema, + ArrowTypeToAvroSchema(f, row_name + "_" + f->name())); + AddRecordField(&record_schema, f->name(), field_schema); + } + return nullable ? NullableSchema(record_schema) : record_schema; + } + case arrow::Type::MAP: { + const auto& map_type = + arrow::internal::checked_pointer_cast(arrow_type); + const auto& key_field = map_type->key_field(); + const auto& item_field = map_type->item_field(); + if (key_field->nullable()) { + return Status::Invalid("Avro Map key cannot be nullable"); + } + if (key_field->type()->id() == arrow::Type::STRING) { + PAIMON_ASSIGN_OR_RAISE(auto item_schema, + ArrowTypeToAvroSchema(item_field, row_name)); + ::avro::MapSchema map_schema(item_schema); + return nullable ? NullableSchema(map_schema) : map_schema; + } else { + // convert to list> + PAIMON_ASSIGN_OR_RAISE(auto key_schema, + ArrowTypeToAvroSchema(key_field, row_name + "_key")); + PAIMON_ASSIGN_OR_RAISE(auto item_schema, + ArrowTypeToAvroSchema(item_field, row_name + "_value")); + ::avro::LogicalType logical_map_type = + ::avro::LogicalType(std::make_shared()); + ::avro::RecordSchema record_schema(row_name); + AddRecordField(&record_schema, "key", key_schema); + AddRecordField(&record_schema, "value", item_schema); + ::avro::ArraySchema logical_map_schema(record_schema); + logical_map_schema.root()->setLogicalType(logical_map_type); + return nullable ? NullableSchema(logical_map_schema) : logical_map_schema; + } + } default: return Status::Invalid(fmt::format("Not support arrow type '{}' convert to avro", field->type()->ToString())); @@ -266,16 +376,14 @@ Result<::avro::Schema> AvroSchemaConverter::ArrowTypeToAvroSchema( Result<::avro::ValidSchema> AvroSchemaConverter::ArrowSchemaToAvroSchema( const std::shared_ptr& arrow_schema) { - ::avro::RecordSchema record_schema("record"); + // top level row name of avro record, the same as java paimon + static const std::string kTopLevelRowName = "org.apache.paimon.avro.generated.record"; + ::avro::RecordSchema record_schema(kTopLevelRowName); for (const auto& field : arrow_schema->fields()) { - PAIMON_ASSIGN_OR_RAISE(::avro::Schema schema, ArrowTypeToAvroSchema(field)); - if (schema.type() == ::avro::Type::AVRO_UNION) { - ::avro::CustomAttributes attrs; - attrs.addAttribute("default", "null", /*addQuotes=*/false); - record_schema.addField(field->name(), schema, attrs); - } else { - record_schema.addField(field->name(), schema); - } + PAIMON_ASSIGN_OR_RAISE( + ::avro::Schema field_schema, + ArrowTypeToAvroSchema(field, kTopLevelRowName + "_" + field->name())); + AddRecordField(&record_schema, field->name(), field_schema); } return ::avro::ValidSchema(record_schema); } diff --git a/src/paimon/format/avro/avro_schema_converter.h b/src/paimon/format/avro/avro_schema_converter.h index f963ff43..92bd6185 100644 --- a/src/paimon/format/avro/avro_schema_converter.h +++ b/src/paimon/format/avro/avro_schema_converter.h @@ -32,20 +32,25 @@ class AvroSchemaConverter { AvroSchemaConverter() = delete; ~AvroSchemaConverter() = delete; + // TODO(menglingda.mld): add field id for avro static Result<::avro::ValidSchema> ArrowSchemaToAvroSchema( const std::shared_ptr& arrow_schema); static Result> AvroSchemaToArrowDataType( const ::avro::ValidSchema& avro_schema); + private: static Result> GetArrowType(const ::avro::NodePtr& avro_node, bool* nullable); - private: - static Result<::avro::Schema> ArrowTypeToAvroSchema(const std::shared_ptr& field); + static Result<::avro::Schema> ArrowTypeToAvroSchema(const std::shared_ptr& field, + const std::string& row_name); static ::avro::Schema NullableSchema(const ::avro::Schema& schema); + static void AddRecordField(::avro::RecordSchema* record_schema, const std::string& field_name, + const ::avro::Schema& field_schema); + static Result CheckUnionType(const ::avro::NodePtr& avro_node); static Result> GetArrowField(const std::string& name, diff --git a/src/paimon/format/avro/avro_schema_converter_test.cpp b/src/paimon/format/avro/avro_schema_converter_test.cpp index 08550aae..c18e4bcb 100644 --- a/src/paimon/format/avro/avro_schema_converter_test.cpp +++ b/src/paimon/format/avro/avro_schema_converter_test.cpp @@ -32,6 +32,7 @@ TEST(AvroSchemaConverterTest, TestSimple) { // Test a basic record with primitive types std::string schema_json = R"({ "type": "record", + "namespace": "org.apache.paimon.avro.generated", "name": "record", "fields": [ {"name": "f_bool", "type": "boolean"}, @@ -196,6 +197,7 @@ TEST(AvroSchemaConverterTest, TestAvroSchemaToArrowDataTypeWithNullableAndComple TEST(AvroSchemaConverterTest, TestAvroSchemaToArrowDataTypeWithTimestampType) { std::string schema_json = R"({ "type": "record", + "namespace": "org.apache.paimon.avro.generated", "name": "record", "fields": [ { diff --git a/src/paimon/format/avro/avro_utils.h b/src/paimon/format/avro/avro_utils.h new file mode 100644 index 00000000..6a79292b --- /dev/null +++ b/src/paimon/format/avro/avro_utils.h @@ -0,0 +1,50 @@ +/* + * 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 "avro/Node.hh" + +namespace paimon::avro { + +class AvroUtils { + public: + AvroUtils() = delete; + ~AvroUtils() = delete; + + static std::string ToString(const ::avro::NodePtr& node) { + std::stringstream ss; + ss << *node; + return ss.str(); + } + + static std::string ToString(const ::avro::LogicalType& type) { + std::stringstream ss; + type.printJson(ss); + return ss.str(); + } + + static bool HasMapLogicalType(const ::avro::NodePtr& node) { + return node->logicalType().type() == ::avro::LogicalType::CUSTOM && + node->logicalType().customLogicalType() != nullptr && + node->logicalType().customLogicalType()->name() == "map"; + } +}; + +} // namespace paimon::avro diff --git a/src/paimon/format/orc/orc_adapter.cpp b/src/paimon/format/orc/orc_adapter.cpp index e986c797..18672bab 100644 --- a/src/paimon/format/orc/orc_adapter.cpp +++ b/src/paimon/format/orc/orc_adapter.cpp @@ -481,7 +481,7 @@ class UnPooledStringDictionaryBuilder : public EmptyBuilder { dict_builder->IncreaseLength(dict_offset.size() - 1); std::shared_ptr dictionary; ARROW_RETURN_NOT_OK(dict_builder->Finish(&dictionary)); - dictionary_.reset(new OrcStringDictionary(dictionary->data(), orc_dictionary)); + dictionary_ = std::make_shared(dictionary->data(), orc_dictionary); return arrow::Status::OK(); } @@ -856,7 +856,7 @@ Result> MakeArrowBuilder( arrow::MemoryPool* pool) { if (column_vector_batch->numElements == 0) { PAIMON_ASSIGN_OR_RAISE_FROM_ARROW(std::shared_ptr builder, - arrow::MakeBuilder(type)); + arrow::MakeBuilder(type, pool)); return builder; } arrow::Type::type kind = type->id(); diff --git a/test/inte/scan_and_read_inte_test.cpp b/test/inte/scan_and_read_inte_test.cpp index 98a5747d..8c38c28b 100644 --- a/test/inte/scan_and_read_inte_test.cpp +++ b/test/inte/scan_and_read_inte_test.cpp @@ -2566,6 +2566,141 @@ TEST_P(ScanAndReadInteTest, TestCastTimestampType) { ASSERT_TRUE(expected->Equals(read_result)) << read_result->ToString(); } +#ifdef PAIMON_ENABLE_AVRO +// TODO(zjw): remove DISABLED_ when avro write is ready +TEST_F(ScanAndReadInteTest, DISABLED_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 + ScanContextBuilder scan_context_builder(table_path); + scan_context_builder.AddOption(Options::SCAN_SNAPSHOT_ID, std::to_string(snapshot_id)); + ASSERT_OK_AND_ASSIGN(auto scan_context, scan_context_builder.Finish()); + ASSERT_OK_AND_ASSIGN(auto table_scan, TableScan::Create(std::move(scan_context))); + ASSERT_OK_AND_ASSIGN(auto result_plan, table_scan->CreatePlan()); + ASSERT_EQ(result_plan->SnapshotId().value(), snapshot_id); + auto splits = result_plan->Splits(); + ASSERT_EQ(3, splits.size()); + + // read + ReadContextBuilder read_context_builder(table_path); + read_context_builder.AddOption("test.enable-adaptive-prefetch-strategy", "false"); + read_context_builder.EnablePrefetch(true).SetPrefetchBatchCount(3); + ASSERT_OK_AND_ASSIGN(std::unique_ptr read_context, + read_context_builder.Finish()); + ASSERT_OK_AND_ASSIGN(auto table_read, TableRead::Create(std::move(read_context))); + ASSERT_OK_AND_ASSIGN(auto batch_reader, table_read->CreateReader(splits)); + ASSERT_OK_AND_ASSIGN(auto read_result, + ReadResultCollector::CollectResult(batch_reader.get())); + + // check result + auto timezone = DateTimeUtils::GetLocalTimezoneName(); + arrow::FieldVector fields = { + arrow::field("_VALUE_KIND", arrow::int8()), + arrow::field("f0", arrow::int8()), + arrow::field("f1", arrow::int16()), + arrow::field("f2", arrow::int32()), + arrow::field("f3", arrow::int64()), + arrow::field("f4", arrow::float32()), + arrow::field("f5", arrow::float64()), + arrow::field("f6", arrow::utf8()), + arrow::field("f7", arrow::binary()), + arrow::field("f8", arrow::date32()), + arrow::field("f9", arrow::decimal128(5, 2)), + arrow::field("f10", arrow::timestamp(arrow::TimeUnit::SECOND)), + arrow::field("f11", arrow::timestamp(arrow::TimeUnit::MILLI)), + arrow::field("f12", arrow::timestamp(arrow::TimeUnit::MICRO)), + arrow::field("f13", arrow::timestamp(arrow::TimeUnit::SECOND, timezone)), + arrow::field("f14", arrow::timestamp(arrow::TimeUnit::MILLI, timezone)), + arrow::field("f15", arrow::timestamp(arrow::TimeUnit::MICRO, timezone)), + arrow::field("f16", arrow::struct_( + {arrow::field("f0", arrow::map(arrow::utf8(), arrow::int32())), + arrow::field("f1", arrow::list(arrow::int32()))})), + }; + auto expected = std::make_shared( + arrow::ipc::internal::json::ArrayFromJSON(arrow::struct_(fields), result_json) + .ValueOrDie()); + ASSERT_TRUE(expected); + ASSERT_TRUE(expected->Equals(read_result)) << read_result->ToString(); + }; + + read_data(1, R"([ +[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, 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, 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]]] +])"); + + 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]]] +])"); +} + +TEST_F(ScanAndReadInteTest, DISABLED_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"; + // scan + ScanContextBuilder scan_context_builder(table_path); + scan_context_builder.AddOption(Options::SCAN_SNAPSHOT_ID, std::to_string(snapshot_id)); + ASSERT_OK_AND_ASSIGN(auto scan_context, scan_context_builder.Finish()); + ASSERT_OK_AND_ASSIGN(auto table_scan, TableScan::Create(std::move(scan_context))); + ASSERT_OK_AND_ASSIGN(auto result_plan, table_scan->CreatePlan()); + ASSERT_EQ(result_plan->SnapshotId().value(), snapshot_id); + + auto splits = result_plan->Splits(); + ASSERT_EQ(1, splits.size()); + + // read + ReadContextBuilder read_context_builder(table_path); + ASSERT_OK_AND_ASSIGN(std::unique_ptr read_context, + read_context_builder.Finish()); + ASSERT_OK_AND_ASSIGN(auto table_read, TableRead::Create(std::move(read_context))); + ASSERT_OK_AND_ASSIGN(auto batch_reader, table_read->CreateReader(splits)); + ASSERT_OK_AND_ASSIGN(auto read_result, + ReadResultCollector::CollectResult(batch_reader.get())); + + // check result + arrow::FieldVector fields = { + arrow::field("_VALUE_KIND", arrow::int8()), + 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::date32()), + arrow::field("f10", arrow::decimal128(5, 2)), + arrow::field("f11", arrow::struct_( + {arrow::field("f0", arrow::map(arrow::utf8(), arrow::int32())), + arrow::field("f1", arrow::list(arrow::int32()))})), + }; + auto expected = std::make_shared( + arrow::ipc::internal::json::ArrayFromJSON(struct_(fields), result_json).ValueOrDie()); + ASSERT_TRUE(expected); + ASSERT_TRUE(expected->Equals(read_result)) << read_result->ToString(); + }; + + read_data(1, R"([ +[0, false, 10, 1, 1, 1000, 1.5, 2.5, "Alice", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]]], +[0, false, 10, 1, 1, 1000, 1.5, 2.5, "Bob", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]]], +[0, true, 10, 1, 1, 1000, 1.5, 2.5, "Emily", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]]], +[0, true, 10, 1, 1, 1000, 1.5, 2.5, "Tony", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]]] +])"); + + read_data(2, R"([ +[0, false, 10, 1, 1, 1000, 1.5, 2.5, "Alice", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]]], +[0, false, 10, 1, 1, 1000, 1.5, 2.5, "Bob", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]]], +[0, true, 10, 1, 1, 1000, 1.5, 2.5, "Lucy", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]]], +[0, true, 10, 1, 1, 1000, 1.5, 2.5, "Tony", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]]] +])"); +} +#endif + std::vector> GetTestValuesForScanAndReadInteTest() { std::vector> values = {{"parquet", false}, {"parquet", true}}; #ifdef PAIMON_ENABLE_ORC diff --git a/test/test_data/avro/append_multiple.db/append_multiple/README b/test/test_data/avro/append_multiple.db/append_multiple/README new file mode 100644 index 00000000..58d086c9 --- /dev/null +++ b/test/test_data/avro/append_multiple.db/append_multiple/README @@ -0,0 +1,14 @@ +f0:tinyint f1:smallint f2:int f3:bigint f4:float f5:double f6:string f7:bytes f8:date f9:decimal(5,2) f10:TIMESTAMP(0) f11:TIMESTAMP(3) f12:TIMESTAMP(6) f13:TIMESTAMP_WITH_LOCAL_TIME_ZONE(0) f14:TIMESTAMP_WITH_LOCAL_TIME_ZONE(3) f15:TIMESTAMP_WITH_LOCAL_TIME_ZONE(6) f16:struct,array> +bucket count: 2 + +Msgs: +snapshot-1 (3 data files) +Add: 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]] +Add: 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]] +Add: 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]] +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 diff --git a/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-0/data-3c670e9a-cc3a-4c29-9ba2-775908cb5650-0.avro b/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-0/data-3c670e9a-cc3a-4c29-9ba2-775908cb5650-0.avro new file mode 100644 index 00000000..c1f94311 Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-0/data-3c670e9a-cc3a-4c29-9ba2-775908cb5650-0.avro differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-0/data-3c670e9a-cc3a-4c29-9ba2-775908cb5650-1.avro b/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-0/data-3c670e9a-cc3a-4c29-9ba2-775908cb5650-1.avro new file mode 100644 index 00000000..244423ec Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-0/data-3c670e9a-cc3a-4c29-9ba2-775908cb5650-1.avro differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-1/data-155f9742-6651-4b8e-9ba6-95e8b74b0549-0.avro b/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-1/data-155f9742-6651-4b8e-9ba6-95e8b74b0549-0.avro new file mode 100644 index 00000000..80fb4017 Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-1/data-155f9742-6651-4b8e-9ba6-95e8b74b0549-0.avro differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-1/data-155f9742-6651-4b8e-9ba6-95e8b74b0549-1.avro b/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-1/data-155f9742-6651-4b8e-9ba6-95e8b74b0549-1.avro new file mode 100644 index 00000000..73a9a159 Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/f1=10/bucket-1/data-155f9742-6651-4b8e-9ba6-95e8b74b0549-1.avro differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/f1=11/bucket-1/data-3fbfe712-94fa-4b8a-870c-c500dec838a5-0.avro b/test/test_data/avro/append_multiple.db/append_multiple/f1=11/bucket-1/data-3fbfe712-94fa-4b8a-870c-c500dec838a5-0.avro new file mode 100644 index 00000000..9315ec52 Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/f1=11/bucket-1/data-3fbfe712-94fa-4b8a-870c-c500dec838a5-0.avro differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/f1=11/bucket-1/data-3fbfe712-94fa-4b8a-870c-c500dec838a5-1.avro b/test/test_data/avro/append_multiple.db/append_multiple/f1=11/bucket-1/data-3fbfe712-94fa-4b8a-870c-c500dec838a5-1.avro new file mode 100644 index 00000000..556c10d1 Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/f1=11/bucket-1/data-3fbfe712-94fa-4b8a-870c-c500dec838a5-1.avro differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-39146ee7-fda8-42a9-894d-5395bc44e8a2-0 b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-39146ee7-fda8-42a9-894d-5395bc44e8a2-0 new file mode 100644 index 00000000..34a12eb1 Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-39146ee7-fda8-42a9-894d-5395bc44e8a2-0 differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-b91f4949-0601-4535-8b29-84b79666b306-0 b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-b91f4949-0601-4535-8b29-84b79666b306-0 new file mode 100644 index 00000000..9e82634b Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-b91f4949-0601-4535-8b29-84b79666b306-0 differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-21081c8d-4a00-4e90-a265-84f4487d1ec0-0 b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-21081c8d-4a00-4e90-a265-84f4487d1ec0-0 new file mode 100644 index 00000000..70ec039d Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-21081c8d-4a00-4e90-a265-84f4487d1ec0-0 differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-21081c8d-4a00-4e90-a265-84f4487d1ec0-1 b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-21081c8d-4a00-4e90-a265-84f4487d1ec0-1 new file mode 100644 index 00000000..3988f11b Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-21081c8d-4a00-4e90-a265-84f4487d1ec0-1 differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-6f4689de-a277-459f-955a-e91a9b52b4ad-0 b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-6f4689de-a277-459f-955a-e91a9b52b4ad-0 new file mode 100644 index 00000000..a697d5f9 Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-6f4689de-a277-459f-955a-e91a9b52b4ad-0 differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-6f4689de-a277-459f-955a-e91a9b52b4ad-1 b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-6f4689de-a277-459f-955a-e91a9b52b4ad-1 new file mode 100644 index 00000000..c1f8ca83 Binary files /dev/null and b/test/test_data/avro/append_multiple.db/append_multiple/manifest/manifest-list-6f4689de-a277-459f-955a-e91a9b52b4ad-1 differ diff --git a/test/test_data/avro/append_multiple.db/append_multiple/schema/schema-0 b/test/test_data/avro/append_multiple.db/append_multiple/schema/schema-0 new file mode 100644 index 00000000..c0d6d755 --- /dev/null +++ b/test/test_data/avro/append_multiple.db/append_multiple/schema/schema-0 @@ -0,0 +1,124 @@ +{ + "version": 3, + "id": 0, + "fields": [ + { + "id": 0, + "name": "f0", + "type": "TINYINT" + }, + { + "id": 1, + "name": "f1", + "type": "SMALLINT" + }, + { + "id": 2, + "name": "f2", + "type": "INT" + }, + { + "id": 3, + "name": "f3", + "type": "BIGINT" + }, + { + "id": 4, + "name": "f4", + "type": "FLOAT" + }, + { + "id": 5, + "name": "f5", + "type": "DOUBLE" + }, + { + "id": 6, + "name": "f6", + "type": "STRING" + }, + { + "id": 7, + "name": "f7", + "type": "BYTES" + }, + { + "id": 8, + "name": "f8", + "type": "DATE" + }, + { + "id": 9, + "name": "f9", + "type": "DECIMAL(5, 2)" + }, + { + "id": 10, + "name": "f10", + "type": "TIMESTAMP(0)" + }, + { + "id": 11, + "name": "f11", + "type": "TIMESTAMP(3)" + }, + { + "id": 12, + "name": "f12", + "type": "TIMESTAMP(6)" + }, + { + "id": 13, + "name": "f13", + "type": "TIMESTAMP(0) WITH LOCAL TIME ZONE" + }, + { + "id": 14, + "name": "f14", + "type": "TIMESTAMP(3) WITH LOCAL TIME ZONE" + }, + { + "id": 15, + "name": "f15", + "type": "TIMESTAMP(6) WITH LOCAL TIME ZONE" + }, + { + "id": 16, + "name": "f16", + "type": { + "type": "ROW", + "fields": [ + { + "id": 17, + "name": "f0", + "type": { + "type": "MAP", + "key": "STRING", + "value": "INT" + } + }, + { + "id": 18, + "name": "f1", + "type": { + "type": "ARRAY", + "element": "INT" + } + } + ] + } + } + ], + "highestFieldId": 18, + "partitionKeys": [ + "f1" + ], + "primaryKeys": [], + "options": { + "bucket": "2", + "bucket-key": "f2", + "manifest.format": "avro", + "file.format": "avro" + }, + "timeMillis": 1767864228000 +} \ No newline at end of file diff --git a/test/test_data/avro/append_multiple.db/append_multiple/snapshot/EARLIEST b/test/test_data/avro/append_multiple.db/append_multiple/snapshot/EARLIEST new file mode 100644 index 00000000..56a6051c --- /dev/null +++ b/test/test_data/avro/append_multiple.db/append_multiple/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/test/test_data/avro/append_multiple.db/append_multiple/snapshot/LATEST b/test/test_data/avro/append_multiple.db/append_multiple/snapshot/LATEST new file mode 100644 index 00000000..d8263ee9 --- /dev/null +++ b/test/test_data/avro/append_multiple.db/append_multiple/snapshot/LATEST @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/test/test_data/avro/append_multiple.db/append_multiple/snapshot/snapshot-1 b/test/test_data/avro/append_multiple.db/append_multiple/snapshot/snapshot-1 new file mode 100644 index 00000000..e31d38e0 --- /dev/null +++ b/test/test_data/avro/append_multiple.db/append_multiple/snapshot/snapshot-1 @@ -0,0 +1,19 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-21081c8d-4a00-4e90-a265-84f4487d1ec0-0", + "baseManifestListSize" : 1006, + "deltaManifestList" : "manifest-list-21081c8d-4a00-4e90-a265-84f4487d1ec0-1", + "deltaManifestListSize" : 1125, + "changelogManifestList" : null, + "commitUser" : "c45f3c63-761e-4938-9bdb-edffde5211a1", + "commitIdentifier" : 1, + "commitKind" : "APPEND", + "timeMillis" : 1767864229282, + "logOffsets" : { }, + "totalRecordCount" : 4, + "deltaRecordCount" : 4, + "changelogRecordCount" : 0, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/append_multiple.db/append_multiple/snapshot/snapshot-2 b/test/test_data/avro/append_multiple.db/append_multiple/snapshot/snapshot-2 new file mode 100644 index 00000000..1c5316cd --- /dev/null +++ b/test/test_data/avro/append_multiple.db/append_multiple/snapshot/snapshot-2 @@ -0,0 +1,19 @@ +{ + "version" : 3, + "id" : 2, + "schemaId" : 0, + "baseManifestList" : "manifest-list-6f4689de-a277-459f-955a-e91a9b52b4ad-0", + "baseManifestListSize" : 1125, + "deltaManifestList" : "manifest-list-6f4689de-a277-459f-955a-e91a9b52b4ad-1", + "deltaManifestListSize" : 1122, + "changelogManifestList" : null, + "commitUser" : "c45f3c63-761e-4938-9bdb-edffde5211a1", + "commitIdentifier" : 2, + "commitKind" : "APPEND", + "timeMillis" : 1767864229369, + "logOffsets" : { }, + "totalRecordCount" : 7, + "deltaRecordCount" : 3, + "changelogRecordCount" : 0, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/append_simple.db/append_simple/README b/test/test_data/avro/append_simple.db/append_simple/README new file mode 100644 index 00000000..ce5aa90f --- /dev/null +++ b/test/test_data/avro/append_simple.db/append_simple/README @@ -0,0 +1,10 @@ +f0:int f1:double f2:smallint f3:string f4:struct,array> +bucket count: -1 + +Msgs: +snapshot-1 (1 data files) +Add: 1, 2.0, "test", [[["key",123]],[1,2,3]] +Add: 1, 2.0, "test", [[["key",123]],[1,2,3]] +Add: 1, 2.0, "test", [[["key",123]],[1,2,3]] +Add: 1, 2.0, "test", [[["key",123]],[1,2,3]] +NoCompact \ No newline at end of file diff --git a/test/test_data/avro/append_simple.db/append_simple/bucket-0/data-d7d1c416-6e34-4834-af87-341d09418f0c-0.avro b/test/test_data/avro/append_simple.db/append_simple/bucket-0/data-d7d1c416-6e34-4834-af87-341d09418f0c-0.avro new file mode 100644 index 00000000..b811fb47 Binary files /dev/null and b/test/test_data/avro/append_simple.db/append_simple/bucket-0/data-d7d1c416-6e34-4834-af87-341d09418f0c-0.avro differ diff --git a/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-f77f2a65-c6be-4e7c-accd-c8dc75578c9a-0 b/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-f77f2a65-c6be-4e7c-accd-c8dc75578c9a-0 new file mode 100644 index 00000000..2be83567 Binary files /dev/null and b/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-f77f2a65-c6be-4e7c-accd-c8dc75578c9a-0 differ diff --git a/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-list-3602dcaf-7dc8-4879-9dce-9215f674e866-0 b/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-list-3602dcaf-7dc8-4879-9dce-9215f674e866-0 new file mode 100644 index 00000000..da13848b Binary files /dev/null and b/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-list-3602dcaf-7dc8-4879-9dce-9215f674e866-0 differ diff --git a/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-list-3602dcaf-7dc8-4879-9dce-9215f674e866-1 b/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-list-3602dcaf-7dc8-4879-9dce-9215f674e866-1 new file mode 100644 index 00000000..fd00865c Binary files /dev/null and b/test/test_data/avro/append_simple.db/append_simple/manifest/manifest-list-3602dcaf-7dc8-4879-9dce-9215f674e866-1 differ diff --git a/test/test_data/avro/append_simple.db/append_simple/schema/schema-0 b/test/test_data/avro/append_simple.db/append_simple/schema/schema-0 new file mode 100644 index 00000000..be12e26c --- /dev/null +++ b/test/test_data/avro/append_simple.db/append_simple/schema/schema-0 @@ -0,0 +1,48 @@ +{ + "version" : 3, + "id" : 0, + "fields" : [ { + "id" : 0, + "name" : "f0", + "type" : "INT" + }, { + "id" : 1, + "name" : "f1", + "type" : "DOUBLE" + }, { + "id" : 2, + "name" : "f2", + "type" : "STRING" + }, { + "id" : 3, + "name" : "f3", + "type" : { + "type" : "ROW", + "fields" : [ { + "id" : 4, + "name" : "f0", + "type" : { + "type" : "MAP", + "key" : "STRING", + "value" : "INT" + } + }, { + "id" : 5, + "name" : "f1", + "type" : { + "type" : "ARRAY", + "element" : "INT" + } + } ] + } + } ], + "highestFieldId" : 5, + "partitionKeys" : [ ], + "primaryKeys" : [ ], + "options" : { + "bucket" : "-1", + "manifest.format" : "avro", + "file.format" : "avro" + }, + "timeMillis" : 1767779394352 +} \ No newline at end of file diff --git a/test/test_data/avro/append_simple.db/append_simple/snapshot/EARLIEST b/test/test_data/avro/append_simple.db/append_simple/snapshot/EARLIEST new file mode 100644 index 00000000..56a6051c --- /dev/null +++ b/test/test_data/avro/append_simple.db/append_simple/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/test/test_data/avro/append_simple.db/append_simple/snapshot/LATEST b/test/test_data/avro/append_simple.db/append_simple/snapshot/LATEST new file mode 100644 index 00000000..56a6051c --- /dev/null +++ b/test/test_data/avro/append_simple.db/append_simple/snapshot/LATEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/test/test_data/avro/append_simple.db/append_simple/snapshot/snapshot-1 b/test/test_data/avro/append_simple.db/append_simple/snapshot/snapshot-1 new file mode 100644 index 00000000..ec396696 --- /dev/null +++ b/test/test_data/avro/append_simple.db/append_simple/snapshot/snapshot-1 @@ -0,0 +1,19 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-3602dcaf-7dc8-4879-9dce-9215f674e866-0", + "baseManifestListSize" : 1006, + "deltaManifestList" : "manifest-list-3602dcaf-7dc8-4879-9dce-9215f674e866-1", + "deltaManifestListSize" : 1108, + "changelogManifestList" : null, + "commitUser" : "4c4a17b4-c139-4fd0-91f6-332509cc3eb1", + "commitIdentifier" : 9223372036854775807, + "commitKind" : "APPEND", + "timeMillis" : 1767779395600, + "logOffsets" : { }, + "totalRecordCount" : 4, + "deltaRecordCount" : 4, + "changelogRecordCount" : 0, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/README b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/README new file mode 100644 index 00000000..6b21711e --- /dev/null +++ b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/README @@ -0,0 +1,7 @@ +f0:map f1:map f2:map f3:map f4:map f5:map> f6:map> f7:map> +bucket count: -1 + +Msgs: +snapshot-1 (1 data files) +Add: [[1,10],[2,20]],[[1.1,10.1],[2.2,20.2]],[["key1","val1"],["key2","val2"]],[["123456","abcdef"]],[["2023-01-01 12:00:00.123000","2023-01-01 12:00:00.123000"],["2023-01-02 13:30:00.456000","2023-01-02 13:30:00.456000"]], + [["arr_key",[1.5, 2.5, 3.5]]],[["outer_key",[[99.9,"nested_val"]]]],[[1000, [42, "row_str", "123.45"]]] diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/bucket-0/data-72442742-e49e-48a4-a736-a2475aac2d2c-0.avro b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/bucket-0/data-72442742-e49e-48a4-a736-a2475aac2d2c-0.avro new file mode 100644 index 00000000..1c423b98 Binary files /dev/null and b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/bucket-0/data-72442742-e49e-48a4-a736-a2475aac2d2c-0.avro differ diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-3c01ce68-8b9e-4bc8-8a56-b481d26faab6-0 b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-3c01ce68-8b9e-4bc8-8a56-b481d26faab6-0 new file mode 100644 index 00000000..830f9b7c Binary files /dev/null and b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-3c01ce68-8b9e-4bc8-8a56-b481d26faab6-0 differ diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-list-6a56727e-83dc-4b80-a420-86a036632e80-0 b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-list-6a56727e-83dc-4b80-a420-86a036632e80-0 new file mode 100644 index 00000000..69433868 Binary files /dev/null and b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-list-6a56727e-83dc-4b80-a420-86a036632e80-0 differ diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-list-6a56727e-83dc-4b80-a420-86a036632e80-1 b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-list-6a56727e-83dc-4b80-a420-86a036632e80-1 new file mode 100644 index 00000000..2e39fcf8 Binary files /dev/null and b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/manifest/manifest-list-6a56727e-83dc-4b80-a420-86a036632e80-1 differ diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/schema/schema-0 b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/schema/schema-0 new file mode 100644 index 00000000..b118e224 --- /dev/null +++ b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/schema/schema-0 @@ -0,0 +1,100 @@ +{ + "version" : 3, + "id" : 0, + "fields" : [ { + "id" : 0, + "name" : "f0", + "type" : { + "type" : "MAP", + "key" : "INT", + "value" : "INT" + } + }, { + "id" : 1, + "name" : "f1", + "type" : { + "type" : "MAP", + "key" : "DOUBLE", + "value" : "DOUBLE" + } + }, { + "id" : 2, + "name" : "f2", + "type" : { + "type" : "MAP", + "key" : "STRING", + "value" : "STRING" + } + }, { + "id" : 3, + "name" : "f3", + "type" : { + "type" : "MAP", + "key" : "STRING", + "value" : "BINARY(6)" + } + }, { + "id" : 4, + "name" : "f4", + "type" : { + "type" : "MAP", + "key" : "TIMESTAMP(6)", + "value" : "TIMESTAMP(6)" + } + }, { + "id" : 5, + "name" : "f5", + "type" : { + "type" : "MAP", + "key" : "STRING", + "value" : { + "type" : "ARRAY", + "element" : "DOUBLE" + } + } + }, { + "id" : 6, + "name" : "f6", + "type" : { + "type" : "MAP", + "key" : "STRING", + "value" : { + "type" : "MAP", + "key" : "DOUBLE", + "value" : "STRING" + } + } + }, { + "id" : 7, + "name" : "f7", + "type" : { + "type" : "MAP", + "key" : "BIGINT", + "value" : { + "type" : "ROW", + "fields" : [ { + "id" : 8, + "name" : "f0", + "type" : "INT" + }, { + "id" : 9, + "name" : "f1", + "type" : "STRING" + }, { + "id" : 10, + "name" : "f2", + "type" : "DECIMAL(5, 2)" + } ] + } + } + } ], + "highestFieldId" : 10, + "partitionKeys" : [ ], + "primaryKeys" : [ ], + "options" : { + "bucket" : "-1", + "manifest.format" : "avro", + "file.format" : "avro" + }, + "timeMillis" : 1767697915909 +} \ No newline at end of file diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/EARLIEST b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/EARLIEST new file mode 100644 index 00000000..56a6051c --- /dev/null +++ b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/LATEST b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/LATEST new file mode 100644 index 00000000..56a6051c --- /dev/null +++ b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/LATEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/snapshot-1 b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/snapshot-1 new file mode 100644 index 00000000..8f7c272d --- /dev/null +++ b/test/test_data/avro/append_with_multiple_map.db/append_with_multiple_map/snapshot/snapshot-1 @@ -0,0 +1,19 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-6a56727e-83dc-4b80-a420-86a036632e80-0", + "baseManifestListSize" : 1006, + "deltaManifestList" : "manifest-list-6a56727e-83dc-4b80-a420-86a036632e80-1", + "deltaManifestListSize" : 1110, + "changelogManifestList" : null, + "commitUser" : "028d90a6-ff1d-4b63-a87d-ea0455075786", + "commitIdentifier" : 9223372036854775807, + "commitKind" : "APPEND", + "timeMillis" : 1767697916612, + "logOffsets" : { }, + "totalRecordCount" : 1, + "deltaRecordCount" : 1, + "changelogRecordCount" : 0, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/README b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/README new file mode 100644 index 00000000..2465ff55 --- /dev/null +++ b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/README @@ -0,0 +1,16 @@ +f0:bool f1:tinyint f2:smallint f3:int f4:bigint f5:float f6:double f7:string f8:bytes f9:data f10:decimal(5,2) f11:struct,array> +primary key: f2,f7 +partition key: f1 +bucket key: f2 +bucket count: 1 + +Msgs: +snapshot-1 (1 data files) +Add: 0, false, 10, 1, 1, 1000, 1.5, 2.5, "Alice", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]] +Add: 0, false, 10, 1, 1, 1000, 1.5, 2.5, "Bob", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]] +Add: 0, true, 10, 1, 1, 1000, 1.5, 2.5, "Emily", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]] +Add: 0, true, 10, 1, 1, 1000, 1.5, 2.5, "Tony", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]] + +snapshot-2 (1 data files) +Delete: 0, true, 10, 1, 1, 1000, 1.5, 2.5, "Emily", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]] +Add: 0, true, 10, 1, 1, 1000, 1.5, 2.5, "Lucy", "abcdef", 100, "123.45", [[["key",123]],[1,2,3]] diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/f1=10/bucket-0/data-0dbda3f3-c4d4-4e2e-b771-c48fc89f30a7-0.avro b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/f1=10/bucket-0/data-0dbda3f3-c4d4-4e2e-b771-c48fc89f30a7-0.avro new file mode 100644 index 00000000..f17c067d Binary files /dev/null and b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/f1=10/bucket-0/data-0dbda3f3-c4d4-4e2e-b771-c48fc89f30a7-0.avro differ diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/f1=10/bucket-0/data-0dbda3f3-c4d4-4e2e-b771-c48fc89f30a7-1.avro b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/f1=10/bucket-0/data-0dbda3f3-c4d4-4e2e-b771-c48fc89f30a7-1.avro new file mode 100644 index 00000000..82ef1bfe Binary files /dev/null and b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/f1=10/bucket-0/data-0dbda3f3-c4d4-4e2e-b771-c48fc89f30a7-1.avro differ diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-4cd8b157-3a02-4a9b-8b65-4976410cb4a9-0 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-4cd8b157-3a02-4a9b-8b65-4976410cb4a9-0 new file mode 100644 index 00000000..3b15c396 Binary files /dev/null and b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-4cd8b157-3a02-4a9b-8b65-4976410cb4a9-0 differ diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-4cd8b157-3a02-4a9b-8b65-4976410cb4a9-1 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-4cd8b157-3a02-4a9b-8b65-4976410cb4a9-1 new file mode 100644 index 00000000..a2b645c1 Binary files /dev/null and b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-4cd8b157-3a02-4a9b-8b65-4976410cb4a9-1 differ diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-0 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-0 new file mode 100644 index 00000000..378bf936 Binary files /dev/null and b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-0 differ diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-1 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-1 new file mode 100644 index 00000000..f3341991 Binary files /dev/null and b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-1 differ diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-2 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-2 new file mode 100644 index 00000000..782b3625 Binary files /dev/null and b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-2 differ diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-3 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-3 new file mode 100644 index 00000000..84bd8cce Binary files /dev/null and b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/manifest/manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-3 differ diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/schema/schema-0 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/schema/schema-0 new file mode 100644 index 00000000..608d0df8 --- /dev/null +++ b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/schema/schema-0 @@ -0,0 +1,81 @@ +{ + "version" : 3, + "id" : 0, + "fields" : [ { + "id" : 0, + "name" : "f0", + "type" : "BOOLEAN" + }, { + "id" : 1, + "name" : "f1", + "type" : "TINYINT" + }, { + "id" : 2, + "name" : "f2", + "type" : "SMALLINT NOT NULL" + }, { + "id" : 3, + "name" : "f3", + "type" : "INT" + }, { + "id" : 4, + "name" : "f4", + "type" : "BIGINT" + }, { + "id" : 5, + "name" : "f5", + "type" : "FLOAT" + }, { + "id" : 6, + "name" : "f6", + "type" : "DOUBLE" + }, { + "id" : 7, + "name" : "f7", + "type" : "STRING NOT NULL" + }, { + "id" : 8, + "name" : "f8", + "type" : "BYTES" + }, { + "id" : 9, + "name" : "f9", + "type" : "DATE" + }, { + "id" : 10, + "name" : "f10", + "type" : "DECIMAL(5, 2)" + }, { + "id" : 11, + "name" : "f11", + "type" : { + "type" : "ROW", + "fields" : [ { + "id" : 12, + "name" : "f0", + "type" : { + "type" : "MAP", + "key" : "STRING", + "value" : "INT" + } + }, { + "id" : 13, + "name" : "f1", + "type" : { + "type" : "ARRAY", + "element" : "INT" + } + } ] + } + } ], + "highestFieldId" : 13, + "partitionKeys" : [ "f1" ], + "primaryKeys" : [ "f2", "f7" ], + "options" : { + "bucket" : "1", + "bucket-key" : "f2", + "manifest.format" : "avro", + "file.format" : "avro" + }, + "timeMillis" : 1768206714821 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/EARLIEST b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/EARLIEST new file mode 100644 index 00000000..56a6051c --- /dev/null +++ b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/LATEST b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/LATEST new file mode 100644 index 00000000..d8263ee9 --- /dev/null +++ b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/LATEST @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/snapshot-1 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/snapshot-1 new file mode 100644 index 00000000..b3afcbc0 --- /dev/null +++ b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/snapshot-1 @@ -0,0 +1,19 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-0", + "baseManifestListSize" : 1006, + "deltaManifestList" : "manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-1", + "deltaManifestListSize" : 1119, + "changelogManifestList" : null, + "commitUser" : "65d24992-0149-4bd8-92bf-715e6e4804ca", + "commitIdentifier" : 1, + "commitKind" : "APPEND", + "timeMillis" : 1768211240603, + "logOffsets" : { }, + "totalRecordCount" : 4, + "deltaRecordCount" : 4, + "changelogRecordCount" : 0, + "nextRowId" : 0 +} \ No newline at end of file diff --git a/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/snapshot-2 b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/snapshot-2 new file mode 100644 index 00000000..f24c3dbb --- /dev/null +++ b/test/test_data/avro/pk_with_multiple_type.db/pk_with_multiple_type/snapshot/snapshot-2 @@ -0,0 +1,19 @@ +{ + "version" : 3, + "id" : 2, + "schemaId" : 0, + "baseManifestList" : "manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-2", + "baseManifestListSize" : 1119, + "deltaManifestList" : "manifest-list-bc16121e-12e3-4ba0-a499-bf054e158852-3", + "deltaManifestListSize" : 1119, + "changelogManifestList" : null, + "commitUser" : "65d24992-0149-4bd8-92bf-715e6e4804ca", + "commitIdentifier" : 2, + "commitKind" : "APPEND", + "timeMillis" : 1768211240667, + "logOffsets" : { }, + "totalRecordCount" : 6, + "deltaRecordCount" : 2, + "changelogRecordCount" : 0, + "nextRowId" : 0 +} \ No newline at end of file