From dbc3ceb85c74b71e2ca3112add470f0dea927a8a Mon Sep 17 00:00:00 2001 From: Li Feiyang Date: Thu, 13 Nov 2025 17:23:24 +0800 Subject: [PATCH 1/4] refactor: make PartitionSpec constructor private and add factory methods with validation --- src/iceberg/json_internal.cc | 15 +- src/iceberg/partition_field.cc | 1 - src/iceberg/partition_spec.cc | 72 +++++- src/iceberg/partition_spec.h | 52 +++- src/iceberg/sort_order.cc | 2 +- src/iceberg/sort_order.h | 1 - src/iceberg/test/json_internal_test.cc | 23 +- .../test/manifest_reader_writer_test.cc | 5 +- src/iceberg/test/metadata_serde_test.cc | 26 +- src/iceberg/test/partition_spec_test.cc | 226 +++++++++++++++--- src/iceberg/test/type_test.cc | 88 +++++++ src/iceberg/util/meson.build | 1 + src/iceberg/util/type_util.h | 72 ++++++ 13 files changed, 510 insertions(+), 74 deletions(-) create mode 100644 src/iceberg/util/type_util.h diff --git a/src/iceberg/json_internal.cc b/src/iceberg/json_internal.cc index ebbc5de65..2117e6b72 100644 --- a/src/iceberg/json_internal.cc +++ b/src/iceberg/json_internal.cc @@ -542,9 +542,10 @@ Result> PartitionSpecFromJson( ICEBERG_ASSIGN_OR_RAISE(auto partition_field, PartitionFieldFromJson(field_json)); partition_fields.push_back(std::move(*partition_field)); } - // TODO(Li Feiyang):use a new PartitionSpec::Make to find the source field of each - // partition field from schema and then verify it - return std::make_unique(spec_id, std::move(partition_fields)); + ICEBERG_ASSIGN_OR_RAISE( + auto spec, PartitionSpec::Make(*schema, spec_id, std::move(partition_fields), + true /*allow_missing_fields*/)); + return spec; } Result> SnapshotRefFromJson(const nlohmann::json& json) { @@ -917,10 +918,10 @@ Status ParsePartitionSpecs(const nlohmann::json& json, int8_t format_version, std::move(field->transform())); } - // TODO(Li Feiyang):use a new PartitionSpec::Make to find the source field of each - // partition field from schema and then verify it - auto spec = - std::make_unique(PartitionSpec::kInitialSpecId, std::move(fields)); + // Create partition spec with schema validation + ICEBERG_ASSIGN_OR_RAISE( + auto spec, PartitionSpec::Make(*current_schema, PartitionSpec::kInitialSpecId, + std::move(fields), true /*allow_missing_fields*/)); default_spec_id = spec->spec_id(); partition_specs.push_back(std::move(spec)); } diff --git a/src/iceberg/partition_field.cc b/src/iceberg/partition_field.cc index 741fcafbf..4ef19ab67 100644 --- a/src/iceberg/partition_field.cc +++ b/src/iceberg/partition_field.cc @@ -22,7 +22,6 @@ #include #include "iceberg/transform.h" -#include "iceberg/type.h" #include "iceberg/util/formatter.h" // IWYU pragma: keep namespace iceberg { diff --git a/src/iceberg/partition_spec.cc b/src/iceberg/partition_spec.cc index f0a211443..f65f483e7 100644 --- a/src/iceberg/partition_spec.cc +++ b/src/iceberg/partition_spec.cc @@ -20,15 +20,19 @@ #include "iceberg/partition_spec.h" #include +#include #include #include #include +#include +#include "iceberg/result.h" #include "iceberg/schema.h" #include "iceberg/schema_field.h" #include "iceberg/transform.h" #include "iceberg/util/formatter.h" // IWYU pragma: keep #include "iceberg/util/macros.h" +#include "iceberg/util/type_util.h" namespace iceberg { @@ -47,9 +51,8 @@ PartitionSpec::PartitionSpec(int32_t spec_id, std::vector fields } const std::shared_ptr& PartitionSpec::Unpartitioned() { - static const std::shared_ptr unpartitioned = - std::make_shared(kInitialSpecId, std::vector{}, - kLegacyPartitionDataIdStart - 1); + static const std::shared_ptr unpartitioned(new PartitionSpec( + kInitialSpecId, std::vector{}, kLegacyPartitionDataIdStart - 1)); return unpartitioned; } @@ -104,4 +107,67 @@ bool PartitionSpec::Equals(const PartitionSpec& other) const { return spec_id_ == other.spec_id_ && fields_ == other.fields_; } +Status PartitionSpec::Validate(const Schema& schema, bool allow_missing_fields) const { + std::unordered_map parents = indexParents(schema); + for (const auto& partition_field : fields_) { + ICEBERG_ASSIGN_OR_RAISE(auto source_field, + schema.FindFieldById(partition_field.source_id())); + // In the case the underlying field is dropped, we cannot check if they are compatible + if (allow_missing_fields && !source_field.has_value()) { + continue; + } + const auto& field_transform = partition_field.transform(); + + // In the case of a Version 1 partition-spec field gets deleted, it is replaced with a + // void transform, see: https://iceberg.apache.org/spec/#partition-transforms. We + // don't care about the source type since a VoidTransform is always compatible and + // skip the checks + if (field_transform->transform_type() != TransformType::kVoid) { + if (!source_field.has_value()) { + return InvalidArgument("Cannot find source column for partition field: {}", + partition_field); + } + const auto& source_type = source_field.value().get().type(); + if (!field_transform->CanTransform(*source_type)) { + return InvalidArgument("Invalid source type {} for transform {}", + source_type->ToString(), field_transform->ToString()); + } + + // The only valid parent types for a PartitionField are StructTypes. This must be + // checked recursively. + auto parent_id_iter = parents.find(partition_field.source_id()); + while (parent_id_iter != parents.end()) { + int32_t parent_id = parent_id_iter->second; + ICEBERG_ASSIGN_OR_RAISE(auto parent_field, schema.FindFieldById(parent_id)); + if (!parent_field.has_value()) { + return InvalidArgument("Cannot find parent field with ID: {}", parent_id); + } + const auto& parent_type = parent_field.value().get().type(); + if (parent_type->type_id() != TypeId::kStruct) { + return InvalidArgument("Invalid partition field parent type: {}", + parent_type->ToString()); + } + parent_id_iter = parents.find(parent_id); + } + } + } + return {}; +} + +Result> PartitionSpec::Make( + const Schema& schema, int32_t spec_id, std::vector fields, + bool allow_missing_fields, std::optional last_assigned_field_id) { + auto partition_spec = std::unique_ptr( + new PartitionSpec(spec_id, std::move(fields), last_assigned_field_id)); + ICEBERG_RETURN_UNEXPECTED(partition_spec->Validate(schema, allow_missing_fields)); + return partition_spec; +} + +Result> PartitionSpec::Make( + int32_t spec_id, std::vector fields, + std::optional last_assigned_field_id) { + return std::unique_ptr( + new PartitionSpec(spec_id, std::move(fields), last_assigned_field_id)); +} + } // namespace iceberg diff --git a/src/iceberg/partition_spec.h b/src/iceberg/partition_spec.h index d1dd3a0fb..12beb9c97 100644 --- a/src/iceberg/partition_spec.h +++ b/src/iceberg/partition_spec.h @@ -50,16 +50,6 @@ class ICEBERG_EXPORT PartitionSpec : public util::Formattable { static constexpr int32_t kLegacyPartitionDataIdStart = 1000; static constexpr int32_t kInvalidPartitionFieldId = -1; - /// \brief Create a new partition spec. - /// - /// \param schema The table schema. - /// \param spec_id The spec ID. - /// \param fields The partition fields. - /// \param last_assigned_field_id The last assigned field ID. If not provided, it will - /// be calculated from the fields. - PartitionSpec(int32_t spec_id, std::vector fields, - std::optional last_assigned_field_id = std::nullopt); - /// \brief Get an unsorted partition spec singleton. static const std::shared_ptr& Unpartitioned(); @@ -80,7 +70,49 @@ class ICEBERG_EXPORT PartitionSpec : public util::Formattable { return lhs.Equals(rhs); } + /// \brief Validates the partition spec against a schema. + /// \param schema The schema to validate against. + /// \param allowMissingFields Whether to skip validation for partition fields whose + /// source columns have been dropped from the schema. + /// \return Error status if the partition spec is invalid. + Status Validate(const Schema& schema, bool allow_missing_fields) const; + + /// \brief Create a PartitionSpec binding to a schema. + /// \param schema The schema to bind the partition spec to. + /// \param spec_id The spec ID. + /// \param fields The partition fields. + /// \param allowMissingFields Whether to skip validation for partition fields whose + /// source columns have been dropped from the schema. + /// \param last_assigned_field_id The last assigned field ID assigned to ensure new + /// fields get unique IDs. + /// \return A Result containing the partition spec or an error. + static Result> Make( + const Schema& schema, int32_t spec_id, std::vector fields, + bool allow_missing_fields, + std::optional last_assigned_field_id = std::nullopt); + + /// \brief Create a PartitionSpec without binding to a schema. + /// \param spec_id The spec ID. + /// \param fields The partition fields. + /// \param last_assigned_field_id The last assigned field ID assigned to ensure new + /// fields get unique IDs. + /// \return A Result containing the partition spec or an error. + /// \note This method does not check whether the sort fields are valid for any schema. + static Result> Make( + int32_t spec_id, std::vector fields, + std::optional last_assigned_field_id = std::nullopt); + private: + /// \brief Create a new partition spec. + /// + /// \param schema The table schema. + /// \param spec_id The spec ID. + /// \param fields The partition fields. + /// \param last_assigned_field_id The last assigned field ID. If not provided, it will + /// be calculated from the fields. + PartitionSpec(int32_t spec_id, std::vector fields, + std::optional last_assigned_field_id = std::nullopt); + /// \brief Compare two partition specs for equality. bool Equals(const PartitionSpec& other) const; diff --git a/src/iceberg/sort_order.cc b/src/iceberg/sort_order.cc index f8978a80e..71e5e18df 100644 --- a/src/iceberg/sort_order.cc +++ b/src/iceberg/sort_order.cc @@ -90,7 +90,7 @@ bool SortOrder::Equals(const SortOrder& other) const { Status SortOrder::Validate(const Schema& schema) const { for (const auto& field : fields_) { ICEBERG_ASSIGN_OR_RAISE(auto schema_field, schema.FindFieldById(field.source_id())); - if (!schema_field.has_value() || schema_field == std::nullopt) { + if (!schema_field.has_value()) { return InvalidArgument("Cannot find source column for sort field: {}", field); } diff --git a/src/iceberg/sort_order.h b/src/iceberg/sort_order.h index 85ed0c3de..1e7285d32 100644 --- a/src/iceberg/sort_order.h +++ b/src/iceberg/sort_order.h @@ -88,7 +88,6 @@ class ICEBERG_EXPORT SortOrder : public util::Formattable { /// \param fields The sort fields. /// \return A Result containing the SortOrder or an error. /// \note This method does not check whether the sort fields are valid for any schema. - /// Use IsBoundToSchema to check if the sort order is valid for a given schema. static Result> Make(int32_t sort_id, std::vector fields); diff --git a/src/iceberg/test/json_internal_test.cc b/src/iceberg/test/json_internal_test.cc index 64f9c1125..5446900c1 100644 --- a/src/iceberg/test/json_internal_test.cc +++ b/src/iceberg/test/json_internal_test.cc @@ -103,14 +103,13 @@ TEST(JsonInternalTest, SortField) { TEST(JsonInternalTest, SortOrder) { auto schema = std::make_shared( - std::vector{SchemaField(5, "region", iceberg::string(), false), - SchemaField(7, "ts", iceberg::int64(), false)}, + std::vector{SchemaField(5, "region", string(), false), + SchemaField(7, "ts", int64(), false)}, /*schema_id=*/100); auto identity_transform = Transform::Identity(); SortField st_ts(5, identity_transform, SortDirection::kAscending, NullOrder::kFirst); SortField st_bar(7, identity_transform, SortDirection::kDescending, NullOrder::kLast); - ICEBERG_UNWRAP_OR_FAIL(auto sort_order, SortOrder::Make(100, {st_ts, st_bar})); - EXPECT_TRUE(sort_order->Validate(*schema)); + ICEBERG_UNWRAP_OR_FAIL(auto sort_order, SortOrder::Make(*schema, 100, {st_ts, st_bar})); nlohmann::json expected_sort_order = R"({"order-id":100,"fields":[ {"transform":"identity","source-id":5,"direction":"asc","null-order":"nulls-first"}, @@ -132,7 +131,7 @@ TEST(JsonInternalTest, PartitionField) { TestJsonConversion(field, expected_json); } -TEST(JsonPartitionTest, PartitionFieldFromJsonMissingField) { +TEST(JsonInternalTest, PartitionFieldFromJsonMissingField) { nlohmann::json invalid_json = R"({"field-id":101,"transform":"identity","name":"region"})"_json; // missing source-id @@ -143,15 +142,19 @@ TEST(JsonPartitionTest, PartitionFieldFromJsonMissingField) { EXPECT_THAT(result, HasErrorMessage("Missing 'source-id'")); } -TEST(JsonPartitionTest, PartitionSpec) { +TEST(JsonInternalTest, PartitionSpec) { auto schema = std::make_shared( std::vector{SchemaField(3, "region", string(), false), SchemaField(5, "ts", int64(), false)}, /*schema_id=*/100); auto identity_transform = Transform::Identity(); - PartitionSpec spec(1, {PartitionField(3, 101, "region", identity_transform), - PartitionField(5, 102, "ts", identity_transform)}); - auto json = ToJson(spec); + ICEBERG_UNWRAP_OR_FAIL( + auto spec, + PartitionSpec::Make(*schema, 1, + {PartitionField(3, 101, "region", identity_transform), + PartitionField(5, 102, "ts", identity_transform)}, + false)); + auto json = ToJson(*spec); nlohmann::json expected_json = R"({"spec-id": 1, "fields": [ {"source-id": 3, @@ -167,7 +170,7 @@ TEST(JsonPartitionTest, PartitionSpec) { auto parsed_spec_result = PartitionSpecFromJson(schema, json); ASSERT_TRUE(parsed_spec_result.has_value()) << parsed_spec_result.error().message; - EXPECT_EQ(spec, *parsed_spec_result.value()); + EXPECT_EQ(*spec, *parsed_spec_result.value()); } TEST(JsonInternalTest, SnapshotRefBranch) { diff --git a/src/iceberg/test/manifest_reader_writer_test.cc b/src/iceberg/test/manifest_reader_writer_test.cc index e5c291201..f400f2f9d 100644 --- a/src/iceberg/test/manifest_reader_writer_test.cc +++ b/src/iceberg/test/manifest_reader_writer_test.cc @@ -186,7 +186,10 @@ TEST_F(ManifestV1Test, WritePartitionedTest) { auto identity_transform = Transform::Identity(); std::vector fields{ PartitionField(1, 1000, "order_ts_hour", identity_transform)}; - auto partition_spec = std::make_shared(1, fields); + auto partition_spec_result = PartitionSpec::Make(1, fields); + ASSERT_TRUE(partition_spec_result.has_value()); + auto partition_spec = + std::shared_ptr(std::move(partition_spec_result.value())); auto expected_entries = PreparePartitionedTestData(); auto write_manifest_path = CreateNewTempFilePath(); diff --git a/src/iceberg/test/metadata_serde_test.cc b/src/iceberg/test/metadata_serde_test.cc index cb1cd8005..a5ccce298 100644 --- a/src/iceberg/test/metadata_serde_test.cc +++ b/src/iceberg/test/metadata_serde_test.cc @@ -99,10 +99,13 @@ TEST(MetadataSerdeTest, DeserializeV1Valid) { SchemaField::MakeRequired(3, "z", int64())}, /*schema_id=*/std::nullopt); - auto expected_spec = std::make_shared( + auto expected_spec_result = PartitionSpec::Make( /*spec_id=*/0, std::vector{PartitionField(/*source_id=*/1, /*field_id=*/1000, "x", Transform::Identity())}); + ASSERT_TRUE(expected_spec_result.has_value()); + auto expected_spec = + std::shared_ptr(std::move(expected_spec_result.value())); TableMetadata expected{ .format_version = 1, @@ -133,7 +136,7 @@ TEST(MetadataSerdeTest, DeserializeV2Valid) { ASSERT_NO_FATAL_FAILURE(ReadTableMetadata("TableMetadataV2Valid.json", &metadata)); auto expected_schema_1 = std::make_shared( - std::vector{SchemaField(/*field_id=*/1, "x", iceberg::int64(), + std::vector{SchemaField(/*field_id=*/1, "x", int64(), /*optional=*/false)}, /*schema_id=*/0); @@ -143,10 +146,13 @@ TEST(MetadataSerdeTest, DeserializeV2Valid) { SchemaField::MakeRequired(3, "z", int64())}, /*schema_id=*/1); - auto expected_spec = std::make_shared( + auto expected_spec_result = PartitionSpec::Make( /*spec_id=*/0, std::vector{PartitionField(/*source_id=*/1, /*field_id=*/1000, "x", Transform::Identity())}); + ASSERT_TRUE(expected_spec_result.has_value()); + auto expected_spec = + std::shared_ptr(std::move(expected_spec_result.value())); ICEBERG_UNWRAP_OR_FAIL( auto sort_order, @@ -228,10 +234,13 @@ TEST(MetadataSerdeTest, DeserializeV2ValidMinimal) { SchemaField::MakeRequired(3, "z", int64())}, /*schema_id=*/0); - auto expected_spec = std::make_shared( + auto expected_spec_result = PartitionSpec::Make( /*spec_id=*/0, std::vector{PartitionField(/*source_id=*/1, /*field_id=*/1000, "x", Transform::Identity())}); + ASSERT_TRUE(expected_spec_result.has_value()); + auto expected_spec = + std::shared_ptr(std::move(expected_spec_result.value())); ICEBERG_UNWRAP_OR_FAIL( auto sort_order, @@ -277,12 +286,15 @@ TEST(MetadataSerdeTest, DeserializeStatisticsFiles) { ReadTableMetadata("TableMetadataStatisticsFiles.json", &metadata)); auto expected_schema = std::make_shared( - std::vector{SchemaField(/*field_id=*/1, "x", iceberg::int64(), + std::vector{SchemaField(/*field_id=*/1, "x", int64(), /*optional=*/false)}, /*schema_id=*/0); + auto expected_spec_result = + PartitionSpec::Make(/*spec_id=*/0, std::vector{}); + ASSERT_TRUE(expected_spec_result.has_value()); auto expected_spec = - std::make_shared(/*spec_id=*/0, std::vector{}); + std::shared_ptr(std::move(expected_spec_result.value())); auto expected_snapshot = std::make_shared(Snapshot{ .snapshot_id = 3055729675574597004, @@ -353,7 +365,7 @@ TEST(MetadataSerdeTest, DeserializePartitionStatisticsFiles) { .last_updated_ms = TimePointMsFromUnixMs(1602638573590).value(), .last_column_id = 3, .schemas = {std::make_shared( - std::vector{SchemaField(/*field_id=*/1, "x", iceberg::int64(), + std::vector{SchemaField(/*field_id=*/1, "x", int64(), /*optional=*/false)}, /*schema_id=*/0)}, .current_schema_id = 0, diff --git a/src/iceberg/test/partition_spec_test.cc b/src/iceberg/test/partition_spec_test.cc index 95e15b1e5..eafcd1016 100644 --- a/src/iceberg/test/partition_spec_test.cc +++ b/src/iceberg/test/partition_spec_test.cc @@ -30,6 +30,7 @@ #include "iceberg/partition_field.h" #include "iceberg/schema.h" #include "iceberg/schema_field.h" +#include "iceberg/test/matchers.h" #include "iceberg/transform.h" #include "iceberg/util/formatter.h" // IWYU pragma: keep @@ -37,13 +38,15 @@ namespace iceberg { TEST(PartitionSpecTest, Basics) { { - SchemaField field1(5, "ts", iceberg::timestamp(), true); - SchemaField field2(7, "bar", iceberg::string(), true); + SchemaField field1(5, "ts", timestamp(), true); + SchemaField field2(7, "bar", string(), true); auto identity_transform = Transform::Identity(); PartitionField pt_field1(5, 1000, "day", identity_transform); PartitionField pt_field2(5, 1001, "hour", identity_transform); - PartitionSpec spec(100, {pt_field1, pt_field2}); + auto spec_result = PartitionSpec::Make(100, {pt_field1, pt_field2}); + ASSERT_TRUE(spec_result.has_value()); + auto& spec = *spec_result.value(); ASSERT_EQ(spec, spec); ASSERT_EQ(100, spec.spec_id()); std::span fields = spec.fields(); @@ -59,42 +62,42 @@ TEST(PartitionSpecTest, Basics) { } TEST(PartitionSpecTest, Equality) { - SchemaField field1(5, "ts", iceberg::timestamp(), true); - SchemaField field2(7, "bar", iceberg::string(), true); + SchemaField field1(5, "ts", timestamp(), true); + SchemaField field2(7, "bar", string(), true); auto identity_transform = Transform::Identity(); PartitionField pt_field1(5, 1000, "day", identity_transform); PartitionField pt_field2(7, 1001, "hour", identity_transform); PartitionField pt_field3(7, 1001, "hour", identity_transform); - PartitionSpec schema1(100, {pt_field1, pt_field2}); - PartitionSpec schema2(101, {pt_field1, pt_field2}); - PartitionSpec schema3(101, {pt_field1}); - PartitionSpec schema4(101, {pt_field3, pt_field1}); - PartitionSpec schema5(100, {pt_field1, pt_field2}); - PartitionSpec schema6(100, {pt_field2, pt_field1}); - - ASSERT_EQ(schema1, schema1); - ASSERT_NE(schema1, schema2); - ASSERT_NE(schema2, schema1); - ASSERT_NE(schema1, schema3); - ASSERT_NE(schema3, schema1); - ASSERT_NE(schema1, schema4); - ASSERT_NE(schema4, schema1); - ASSERT_EQ(schema1, schema5); - ASSERT_EQ(schema5, schema1); - ASSERT_NE(schema1, schema6); - ASSERT_NE(schema6, schema1); + auto schema1 = PartitionSpec::Make(100, {pt_field1, pt_field2}).value(); + auto schema2 = PartitionSpec::Make(101, {pt_field1, pt_field2}).value(); + auto schema3 = PartitionSpec::Make(101, {pt_field1}).value(); + auto schema4 = PartitionSpec::Make(101, {pt_field3, pt_field1}).value(); + auto schema5 = PartitionSpec::Make(100, {pt_field1, pt_field2}).value(); + auto schema6 = PartitionSpec::Make(100, {pt_field2, pt_field1}).value(); + + ASSERT_EQ(*schema1, *schema1); + ASSERT_NE(*schema1, *schema2); + ASSERT_NE(*schema2, *schema1); + ASSERT_NE(*schema1, *schema3); + ASSERT_NE(*schema3, *schema1); + ASSERT_NE(*schema1, *schema4); + ASSERT_NE(*schema4, *schema1); + ASSERT_EQ(*schema1, *schema5); + ASSERT_EQ(*schema5, *schema1); + ASSERT_NE(*schema1, *schema6); + ASSERT_NE(*schema6, *schema1); } TEST(PartitionSpecTest, PartitionSchemaTest) { - SchemaField field1(5, "ts", iceberg::timestamp(), true); - SchemaField field2(7, "bar", iceberg::string(), true); + SchemaField field1(5, "ts", timestamp(), true); + SchemaField field2(7, "bar", string(), true); Schema schema({field1, field2}, 100); auto identity_transform = Transform::Identity(); PartitionField pt_field1(5, 1000, "day", identity_transform); PartitionField pt_field2(7, 1001, "hour", identity_transform); - PartitionSpec spec(100, {pt_field1, pt_field2}); + auto spec = PartitionSpec::Make(100, {pt_field1, pt_field2}).value(); - auto partition_type = spec.PartitionType(schema); + auto partition_type = spec->PartitionType(schema); ASSERT_TRUE(partition_type.has_value()); ASSERT_EQ(2, partition_type.value()->fields().size()); EXPECT_EQ(pt_field1.name(), partition_type.value()->fields()[0].name()); @@ -125,12 +128,12 @@ TEST(PartitionSpecTest, PartitionTypeTest) { } ] })"_json; - SchemaField field1(1, "id", int32(), false); - SchemaField field2(2, "name", string(), false); - SchemaField field3(3, "ts", timestamp(), false); - SchemaField field4(4, "ts_day", timestamp(), false); - SchemaField field5(5, "id_bucket", int32(), false); - SchemaField field6(6, "id_truncate", int32(), false); + auto field1 = SchemaField::MakeRequired(1, "id", int32()); + auto field2 = SchemaField::MakeRequired(2, "name", string()); + auto field3 = SchemaField::MakeRequired(3, "ts", timestamp()); + auto field4 = SchemaField::MakeRequired(4, "ts_day", timestamp()); + auto field5 = SchemaField::MakeRequired(5, "id_bucket", int32()); + auto field6 = SchemaField::MakeRequired(6, "id_truncate", int32()); auto const schema = std::make_shared( std::vector{field1, field2, field3, field4, field5, field6}, Schema::kInitialSchemaId); @@ -152,4 +155,161 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, InvalidTransformForType) { + // Test Day transform on string type (should fail) + auto field_string = SchemaField::MakeRequired(6, "s", string()); + Schema schema_string({field_string}, Schema::kInitialSchemaId); + + PartitionField pt_field_invalid(6, 1005, "s_day", Transform::Day()); + auto result = PartitionSpec::Make(schema_string, 1, {pt_field_invalid}, false); + EXPECT_THAT(result, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result, HasErrorMessage("Invalid source type")); + + // Test that Void transform does not error out even with incompatible types + // Void transform is used for V1 partition-spec when a field gets deleted + PartitionField pt_field_void(6, 1006, "s_void", Transform::Void()); + auto result_void = PartitionSpec::Make(schema_string, 1, {pt_field_void}, false); + EXPECT_THAT(result_void, IsOk()); +} + +TEST(PartitionSpecTest, SourceIdNotFound) { + auto field1 = SchemaField::MakeRequired(1, "id", int64()); + auto field2 = SchemaField::MakeRequired(2, "ts", timestamp()); + Schema schema({field1, field2}, Schema::kInitialSchemaId); + + // Try to create partition field with source ID 99 which doesn't exist + PartitionField pt_field_invalid(99, 1000, "Test", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field_invalid}, false); + EXPECT_THAT(result, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result, HasErrorMessage("Cannot find source column for partition field")); +} + +TEST(PartitionSpecTest, AllowMissingFields) { + auto field1 = SchemaField::MakeRequired(1, "id", int64()); + auto field2 = SchemaField::MakeRequired(2, "ts", timestamp()); + Schema schema({field1, field2}, Schema::kInitialSchemaId); + + // Create partition field with source ID 99 which doesn't exist + PartitionField pt_field_missing(99, 1000, "Test", Transform::Identity()); + + // Without allow_missing_fields, this should fail + auto result_no_allow = + PartitionSpec::Make(schema, 1, {pt_field_missing}, false, std::nullopt); + EXPECT_THAT(result_no_allow, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result_no_allow, + HasErrorMessage("Cannot find source column for partition field")); + + // With allow_missing_fields, this should succeed (e.g., for evolved schemas where + // source field was dropped) + auto result_allow = + PartitionSpec::Make(schema, 1, {pt_field_missing}, true, std::nullopt); + EXPECT_THAT(result_allow, IsOk()); +} + +TEST(PartitionSpecTest, PartitionFieldInStruct) { + auto field1 = SchemaField::MakeRequired(1, "id", int64()); + auto field2 = SchemaField::MakeRequired(2, "ts", timestamp()); + Schema base_schema({field1, field2}, Schema::kInitialSchemaId); + + auto struct_type = + std::make_shared(std::vector{field1, field2}); + auto outer_struct = SchemaField::MakeRequired(11, "MyStruct", struct_type); + + Schema schema({outer_struct}, Schema::kInitialSchemaId); + PartitionField pt_field(1, 1000, "id_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field}, false); + EXPECT_THAT(result, IsOk()); +} + +TEST(PartitionSpecTest, PartitionFieldInStructInStruct) { + auto field1 = SchemaField::MakeRequired(1, "id", int64()); + auto field2 = SchemaField::MakeRequired(2, "ts", timestamp()); + + auto inner_struct = + std::make_shared(std::vector{field1, field2}); + auto inner_field = SchemaField::MakeRequired(11, "Inner", inner_struct); + auto outer_struct = std::make_shared(std::vector{inner_field}); + SchemaField outer_field(12, "Outer", outer_struct, true); + + Schema schema({outer_field}, Schema::kInitialSchemaId); + PartitionField pt_field(1, 1000, "id_partition", Transform::Identity()); + auto result = PartitionSpec::Make(schema, 1, {pt_field}, false); + EXPECT_THAT(result, IsOk()); +} + +TEST(PartitionSpecTest, PartitionFieldInList) { + auto list_type = std::make_shared(1, int32(), /*element_required=*/false); + auto list_field = SchemaField::MakeRequired(2, "MyList", list_type); + Schema schema({list_field}, Schema::kInitialSchemaId); + + // Try to partition on the list element field (field ID 1 is the element) + PartitionField pt_field(1, 1000, "element_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field}, false); + EXPECT_THAT(result, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result, HasErrorMessage("Invalid partition field parent")); +} + +TEST(PartitionSpecTest, PartitionFieldInStructInList) { + auto struct_in_list = std::make_shared( + std::vector{SchemaField(1, "Foo", int32(), true)}); + auto list_type = std::make_shared(2, struct_in_list, + /*element_required=*/false); + auto list_field = SchemaField::MakeRequired(3, "MyList", list_type); + + Schema schema({list_field}, Schema::kInitialSchemaId); + PartitionField pt_field(1, 1000, "foo_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field}, false); + EXPECT_THAT(result, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result, HasErrorMessage("Invalid partition field parent")); +} + +TEST(PartitionSpecTest, PartitionFieldInMap) { + auto key_field = SchemaField::MakeRequired(1, "key", int32()); + auto value_field = SchemaField::MakeRequired(2, "value", int32()); + auto map_type = std::make_shared(key_field, value_field); + auto map_field = SchemaField::MakeRequired(3, "MyMap", map_type); + + Schema schema({map_field}, Schema::kInitialSchemaId); + PartitionField pt_field_key(1, 1000, "key_partition", Transform::Identity()); + + auto result_key = PartitionSpec::Make(schema, 1, {pt_field_key}, false); + EXPECT_THAT(result_key, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result_key, HasErrorMessage("Invalid partition field parent")); + + PartitionField pt_field_value(2, 1001, "value_partition", Transform::Identity()); + + auto result_value = PartitionSpec::Make(schema, 1, {pt_field_value}, false); + EXPECT_THAT(result_value, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result_value, HasErrorMessage("Invalid partition field parent")); +} + +TEST(PartitionSpecTest, PartitionFieldInStructInMap) { + auto struct_key = std::make_shared( + std::vector{SchemaField(1, "Foo", int32(), true)}); + auto struct_value = std::make_shared( + std::vector{SchemaField(2, "Bar", int32(), true)}); + + auto key_field = SchemaField::MakeRequired(3, "key", struct_key); + auto value_field = SchemaField::MakeRequired(4, "value", struct_value); + auto map_type = std::make_shared(key_field, value_field); + auto map_field = SchemaField::MakeRequired(5, "MyMap", map_type); + + Schema schema({map_field}, Schema::kInitialSchemaId); + PartitionField pt_field_key(1, 1000, "foo_partition", Transform::Identity()); + + auto result_key = PartitionSpec::Make(schema, 1, {pt_field_key}, false); + EXPECT_THAT(result_key, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result_key, HasErrorMessage("Invalid partition field parent")); + + PartitionField pt_field_value(2, 1001, "bar_partition", Transform::Identity()); + + auto result_value = PartitionSpec::Make(schema, 1, {pt_field_value}, false); + EXPECT_THAT(result_value, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(result_value, HasErrorMessage("Invalid partition field parent")); +} + } // namespace iceberg diff --git a/src/iceberg/test/type_test.cc b/src/iceberg/test/type_test.cc index 82379540b..4568f9517 100644 --- a/src/iceberg/test/type_test.cc +++ b/src/iceberg/test/type_test.cc @@ -30,6 +30,7 @@ #include "iceberg/exception.h" #include "iceberg/test/matchers.h" #include "iceberg/util/formatter.h" // IWYU pragma: keep +#include "iceberg/util/type_util.h" struct TypeTestCase { /// Test case name, must be safe for Googletest (alphanumeric + underscore) @@ -604,3 +605,90 @@ TEST_F(StructTypeThreadSafetyTest, MixedConcurrentOperations) { thread.join(); } } + +TEST(TypeTest, IndexParents) { + auto inner_preferences = + std::make_shared(std::vector{ + iceberg::SchemaField::MakeRequired(12, "feature3", iceberg::boolean()), + iceberg::SchemaField::MakeOptional(13, "feature4", iceberg::boolean()), + }); + + auto preferences = + std::make_shared(std::vector{ + iceberg::SchemaField::MakeRequired(6, "feature1", iceberg::boolean()), + iceberg::SchemaField::MakeOptional(7, "feature2", iceberg::boolean()), + iceberg::SchemaField::MakeOptional(8, "inner_preferences", inner_preferences), + }); + + auto locations_key_struct = + std::make_shared(std::vector{ + iceberg::SchemaField::MakeRequired(20, "address", iceberg::string()), + iceberg::SchemaField::MakeRequired(21, "city", iceberg::string()), + iceberg::SchemaField::MakeRequired(22, "state", iceberg::string()), + iceberg::SchemaField::MakeRequired(23, "zip", iceberg::int32()), + }); + + auto locations_value_struct = + std::make_shared(std::vector{ + iceberg::SchemaField::MakeRequired(14, "lat", iceberg::float32()), + iceberg::SchemaField::MakeRequired(15, "long", iceberg::float32()), + }); + + auto locations = iceberg::SchemaField::MakeRequired( + 4, "locations", + std::make_shared( + iceberg::SchemaField::MakeRequired(9, "key", locations_key_struct), + iceberg::SchemaField::MakeRequired(10, "value", locations_value_struct))); + + auto points_struct = + std::make_shared(std::vector{ + iceberg::SchemaField::MakeRequired(16, "x", iceberg::int64()), + iceberg::SchemaField::MakeRequired(17, "y", iceberg::int64()), + }); + + auto points = iceberg::SchemaField::MakeOptional( + 5, "points", + std::make_shared( + iceberg::SchemaField::MakeOptional(11, "element", points_struct))); + + auto root_struct = iceberg::StructType(std::vector{ + iceberg::SchemaField::MakeRequired(1, "id", iceberg::int32()), + iceberg::SchemaField::MakeOptional(2, "data", iceberg::string()), + iceberg::SchemaField::MakeOptional(3, "preferences", preferences), + locations, + points, + }); + + std::unordered_map parent_index = iceberg::indexParents(root_struct); + + // Verify top-level fields have no parent + ASSERT_EQ(parent_index.find(1), parent_index.end()); + ASSERT_EQ(parent_index.find(2), parent_index.end()); + ASSERT_EQ(parent_index.find(3), parent_index.end()); + ASSERT_EQ(parent_index.find(4), parent_index.end()); + ASSERT_EQ(parent_index.find(5), parent_index.end()); + + // Verify struct field parents + ASSERT_EQ(parent_index[6], 3); + ASSERT_EQ(parent_index[7], 3); + ASSERT_EQ(parent_index[8], 3); + ASSERT_EQ(parent_index[12], 8); + ASSERT_EQ(parent_index[13], 8); + + // Verify map field parents + ASSERT_EQ(parent_index[9], 4); + ASSERT_EQ(parent_index[10], 4); + ASSERT_EQ(parent_index[20], 9); + ASSERT_EQ(parent_index[21], 9); + ASSERT_EQ(parent_index[22], 9); + ASSERT_EQ(parent_index[23], 9); + ASSERT_EQ(parent_index[14], 10); + ASSERT_EQ(parent_index[15], 10); + + // Verify list field parents + ASSERT_EQ(parent_index[11], 5); + ASSERT_EQ(parent_index[16], 11); + ASSERT_EQ(parent_index[17], 11); + + ASSERT_EQ(parent_index.size(), 16); +} diff --git a/src/iceberg/util/meson.build b/src/iceberg/util/meson.build index 06984a479..1f755be62 100644 --- a/src/iceberg/util/meson.build +++ b/src/iceberg/util/meson.build @@ -28,6 +28,7 @@ install_headers( 'string_util.h', 'timepoint.h', 'truncate_util.h', + 'type_util.h', 'visitor_generate.h', 'visit_type.h', ], diff --git a/src/iceberg/util/type_util.h b/src/iceberg/util/type_util.h new file mode 100644 index 000000000..d0c106b26 --- /dev/null +++ b/src/iceberg/util/type_util.h @@ -0,0 +1,72 @@ +/* + * 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. + */ + +#pragma once + +#include +#include + +#include "iceberg/type.h" + +/// \file iceberg/util/type_util.h +/// Utility functions for Iceberg types. + +namespace iceberg { + +/// \brief Index parent field IDs for all fields in a struct hierarchy. +/// \param root_struct The root struct type to analyze +/// \return A map from field ID to its parent struct field ID +/// \note This function assumes the input StructType has already been validated: +/// - All field IDs must be non-negative +/// - All field IDs must be unique across the entire schema hierarchy +/// If the struct is part of a Schema, these invariants are enforced by +/// StructType::InitFieldById which checks for duplicate field IDs. +ICEBERG_EXPORT static std::unordered_map indexParents( + const StructType& root_struct) { + std::unordered_map id_to_parent; + std::stack parent_id_stack; + + // Recursive function to visit and build parent relationships + std::function visit = [&](const Type& type) -> void { + switch (type.type_id()) { + case TypeId::kStruct: + case TypeId::kList: + case TypeId::kMap: { + const auto& nested_type = static_cast(type); + for (const auto& field : nested_type.fields()) { + if (!parent_id_stack.empty()) { + id_to_parent[field.field_id()] = parent_id_stack.top(); + } + parent_id_stack.push(field.field_id()); + visit(*field.type()); + parent_id_stack.pop(); + } + break; + } + + default: + break; + } + }; + + visit(root_struct); + return id_to_parent; +} + +} // namespace iceberg From 4804ccd322858170af8b72a09831546b562f5d59 Mon Sep 17 00:00:00 2001 From: Li Feiyang Date: Thu, 13 Nov 2025 17:35:54 +0800 Subject: [PATCH 2/4] fix ci --- src/iceberg/util/type_util.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/iceberg/util/type_util.h b/src/iceberg/util/type_util.h index d0c106b26..cad9da54a 100644 --- a/src/iceberg/util/type_util.h +++ b/src/iceberg/util/type_util.h @@ -37,8 +37,7 @@ namespace iceberg { /// - All field IDs must be unique across the entire schema hierarchy /// If the struct is part of a Schema, these invariants are enforced by /// StructType::InitFieldById which checks for duplicate field IDs. -ICEBERG_EXPORT static std::unordered_map indexParents( - const StructType& root_struct) { +static std::unordered_map indexParents(const StructType& root_struct) { std::unordered_map id_to_parent; std::stack parent_id_stack; From 3a9eb0118296593006f323d9c014f05efb373bb0 Mon Sep 17 00:00:00 2001 From: Li Feiyang Date: Thu, 13 Nov 2025 18:01:44 +0800 Subject: [PATCH 3/4] fix --- src/iceberg/json_internal.cc | 20 +++++++++++++------ src/iceberg/json_internal.h | 4 +++- src/iceberg/test/json_internal_test.cc | 2 +- .../test/manifest_reader_writer_test.cc | 6 ++---- src/iceberg/test/partition_spec_test.cc | 2 +- 5 files changed, 21 insertions(+), 13 deletions(-) diff --git a/src/iceberg/json_internal.cc b/src/iceberg/json_internal.cc index 2117e6b72..2a0042327 100644 --- a/src/iceberg/json_internal.cc +++ b/src/iceberg/json_internal.cc @@ -533,7 +533,8 @@ Result> PartitionFieldFromJson( } Result> PartitionSpecFromJson( - const std::shared_ptr& schema, const nlohmann::json& json) { + const std::shared_ptr& schema, const nlohmann::json& json, + int32_t default_spec_id) { ICEBERG_ASSIGN_OR_RAISE(auto spec_id, GetJsonValue(json, kSpecId)); ICEBERG_ASSIGN_OR_RAISE(auto fields, GetJsonValue(json, kFields)); @@ -542,9 +543,16 @@ Result> PartitionSpecFromJson( ICEBERG_ASSIGN_OR_RAISE(auto partition_field, PartitionFieldFromJson(field_json)); partition_fields.push_back(std::move(*partition_field)); } - ICEBERG_ASSIGN_OR_RAISE( - auto spec, PartitionSpec::Make(*schema, spec_id, std::move(partition_fields), - true /*allow_missing_fields*/)); + + std::unique_ptr spec; + if (default_spec_id == spec_id) { + ICEBERG_ASSIGN_OR_RAISE( + spec, PartitionSpec::Make(*schema, spec_id, std::move(partition_fields), + true /*allow_missing_fields*/)); + } else { + ICEBERG_ASSIGN_OR_RAISE(spec, + PartitionSpec::Make(spec_id, std::move(partition_fields))); + } return spec; } @@ -886,8 +894,8 @@ Status ParsePartitionSpecs(const nlohmann::json& json, int8_t format_version, ICEBERG_ASSIGN_OR_RAISE(default_spec_id, GetJsonValue(json, kDefaultSpecId)); for (const auto& spec_json : spec_array) { - ICEBERG_ASSIGN_OR_RAISE(auto spec, - PartitionSpecFromJson(current_schema, spec_json)); + ICEBERG_ASSIGN_OR_RAISE( + auto spec, PartitionSpecFromJson(current_schema, spec_json, default_spec_id)); partition_specs.push_back(std::move(spec)); } } else { diff --git a/src/iceberg/json_internal.h b/src/iceberg/json_internal.h index 4f9cef8e8..8ca0a676e 100644 --- a/src/iceberg/json_internal.h +++ b/src/iceberg/json_internal.h @@ -176,10 +176,12 @@ ICEBERG_EXPORT Result ToJsonString(const PartitionSpec& partition_s /// /// \param schema The current schema. /// \param json The JSON object representing a `PartitionSpec`. +/// \param default_spec_id The default spec ID from the table metadata. /// \return An `expected` value containing either a `PartitionSpec` object or an error. If /// the JSON is malformed or missing expected fields, an error will be returned. ICEBERG_EXPORT Result> PartitionSpecFromJson( - const std::shared_ptr& schema, const nlohmann::json& json); + const std::shared_ptr& schema, const nlohmann::json& json, + int32_t default_spec_id); /// \brief Serializes a `SnapshotRef` object to JSON. /// diff --git a/src/iceberg/test/json_internal_test.cc b/src/iceberg/test/json_internal_test.cc index 5446900c1..a23cc6802 100644 --- a/src/iceberg/test/json_internal_test.cc +++ b/src/iceberg/test/json_internal_test.cc @@ -168,7 +168,7 @@ TEST(JsonInternalTest, PartitionSpec) { EXPECT_EQ(json, expected_json); - auto parsed_spec_result = PartitionSpecFromJson(schema, json); + auto parsed_spec_result = PartitionSpecFromJson(schema, json, 1); ASSERT_TRUE(parsed_spec_result.has_value()) << parsed_spec_result.error().message; EXPECT_EQ(*spec, *parsed_spec_result.value()); } diff --git a/src/iceberg/test/manifest_reader_writer_test.cc b/src/iceberg/test/manifest_reader_writer_test.cc index f400f2f9d..62a997ef9 100644 --- a/src/iceberg/test/manifest_reader_writer_test.cc +++ b/src/iceberg/test/manifest_reader_writer_test.cc @@ -186,10 +186,8 @@ TEST_F(ManifestV1Test, WritePartitionedTest) { auto identity_transform = Transform::Identity(); std::vector fields{ PartitionField(1, 1000, "order_ts_hour", identity_transform)}; - auto partition_spec_result = PartitionSpec::Make(1, fields); - ASSERT_TRUE(partition_spec_result.has_value()); - auto partition_spec = - std::shared_ptr(std::move(partition_spec_result.value())); + ICEBERG_UNWRAP_OR_FAIL(std::shared_ptr partition_spec, + PartitionSpec::Make(*table_schema, 1, fields, false)); auto expected_entries = PreparePartitionedTestData(); auto write_manifest_path = CreateNewTempFilePath(); diff --git a/src/iceberg/test/partition_spec_test.cc b/src/iceberg/test/partition_spec_test.cc index eafcd1016..3ae264f7a 100644 --- a/src/iceberg/test/partition_spec_test.cc +++ b/src/iceberg/test/partition_spec_test.cc @@ -138,7 +138,7 @@ TEST(PartitionSpecTest, PartitionTypeTest) { std::vector{field1, field2, field3, field4, field5, field6}, Schema::kInitialSchemaId); - auto parsed_spec_result = PartitionSpecFromJson(schema, json); + auto parsed_spec_result = PartitionSpecFromJson(schema, json, 1); ASSERT_TRUE(parsed_spec_result.has_value()) << parsed_spec_result.error().message; auto partition_type = parsed_spec_result.value()->PartitionType(*schema); From 2d326918a266d6b10b5aa54d19f5bd7d14904c5a Mon Sep 17 00:00:00 2001 From: Li Feiyang Date: Fri, 14 Nov 2025 14:23:43 +0800 Subject: [PATCH 4/4] fix --- src/iceberg/json_internal.cc | 12 +++++++----- src/iceberg/test/manifest_reader_writer_test.cc | 1 + 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/iceberg/json_internal.cc b/src/iceberg/json_internal.cc index 2a0042327..caa17de02 100644 --- a/src/iceberg/json_internal.cc +++ b/src/iceberg/json_internal.cc @@ -548,10 +548,11 @@ Result> PartitionSpecFromJson( if (default_spec_id == spec_id) { ICEBERG_ASSIGN_OR_RAISE( spec, PartitionSpec::Make(*schema, spec_id, std::move(partition_fields), - true /*allow_missing_fields*/)); + /*allow_missing_fields=*/false)); } else { - ICEBERG_ASSIGN_OR_RAISE(spec, - PartitionSpec::Make(spec_id, std::move(partition_fields))); + ICEBERG_ASSIGN_OR_RAISE( + spec, PartitionSpec::Make(*schema, spec_id, std::move(partition_fields), + /*allow_missing_fields=*/true)); } return spec; } @@ -928,8 +929,9 @@ Status ParsePartitionSpecs(const nlohmann::json& json, int8_t format_version, // Create partition spec with schema validation ICEBERG_ASSIGN_OR_RAISE( - auto spec, PartitionSpec::Make(*current_schema, PartitionSpec::kInitialSpecId, - std::move(fields), true /*allow_missing_fields*/)); + auto spec, + PartitionSpec::Make(*current_schema, PartitionSpec::kInitialSpecId, + std::move(fields), /*allow_missing_fields=*/false)); default_spec_id = spec->spec_id(); partition_specs.push_back(std::move(spec)); } diff --git a/src/iceberg/test/manifest_reader_writer_test.cc b/src/iceberg/test/manifest_reader_writer_test.cc index 62a997ef9..54dcc754f 100644 --- a/src/iceberg/test/manifest_reader_writer_test.cc +++ b/src/iceberg/test/manifest_reader_writer_test.cc @@ -34,6 +34,7 @@ #include "iceberg/test/temp_file_test_base.h" #include "iceberg/test/test_common.h" #include "iceberg/transform.h" +#include "iceberg/type.h" namespace iceberg {