wgtmac commented on code in PR #316: URL: https://github.com/apache/iceberg-cpp/pull/316#discussion_r2521384236
########## src/iceberg/util/type_util.h: ########## @@ -0,0 +1,101 @@ +/* + * 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 <functional> +#include <stack> + +#include "iceberg/type.h" + +/// \file iceberg/util/type_util.h Review Comment: We need to install this file in the `meson.build`. ########## src/iceberg/partition_spec.cc: ########## @@ -104,4 +108,68 @@ bool PartitionSpec::Equals(const PartitionSpec& other) const { return spec_id_ == other.spec_id_ && fields_ == other.fields_; } +Status PartitionSpec::Validate(const Schema& schema, bool allowMissingFields) const { + std::unordered_map<int32_t, int32_t> 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 (allowMissingFields && !source_field.has_value()) { + continue; + } + 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 (!partition_field.transform()->CanTransform(*source_type)) { + return InvalidArgument("Invalid source type {} for transform {}", + source_type->ToString(), + partition_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<std::unique_ptr<PartitionSpec>> PartitionSpec::Make( + const Schema& schema, int32_t spec_id, std::vector<PartitionField> fields, + std::optional<int32_t> last_assigned_field_id, bool allowMissingFields) { + auto partition_spec = + std::make_unique<PartitionSpec>(spec_id, std::move(fields), last_assigned_field_id); Review Comment: Why not making ctors private and change downstreams all at once? ########## src/iceberg/partition_spec.cc: ########## @@ -104,4 +108,68 @@ bool PartitionSpec::Equals(const PartitionSpec& other) const { return spec_id_ == other.spec_id_ && fields_ == other.fields_; } +Status PartitionSpec::Validate(const Schema& schema, bool allowMissingFields) const { + std::unordered_map<int32_t, int32_t> 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 (allowMissingFields && !source_field.has_value()) { + continue; + } + auto field_transform = partition_field.transform(); Review Comment: ```suggestion const auto& field_transform = partition_field.transform(); ``` ########## src/iceberg/test/partition_spec_test.cc: ########## @@ -152,4 +152,174 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, MissingSourceColumn) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + SchemaField field3(6, "s", string(), false); + Schema schema({field1, field2, field3}, Schema::kInitialSchemaId); + + // Try to create partition field with non-existent source column ID 999 + PartitionField pt_field_invalid(999, 1000, "missing_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field_invalid}); Review Comment: Please also check `check_missing_fields` work ########## src/iceberg/partition_spec.cc: ########## @@ -104,4 +108,68 @@ bool PartitionSpec::Equals(const PartitionSpec& other) const { return spec_id_ == other.spec_id_ && fields_ == other.fields_; } +Status PartitionSpec::Validate(const Schema& schema, bool allowMissingFields) const { + std::unordered_map<int32_t, int32_t> 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 (allowMissingFields && !source_field.has_value()) { + continue; + } + 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 (!partition_field.transform()->CanTransform(*source_type)) { Review Comment: ```suggestion if (!field_transform->CanTransform(*source_type)) { ``` ########## src/iceberg/test/partition_spec_test.cc: ########## @@ -152,4 +152,174 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, MissingSourceColumn) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + SchemaField field3(6, "s", string(), false); + Schema schema({field1, field2, field3}, Schema::kInitialSchemaId); + + // Try to create partition field with non-existent source column ID 999 + PartitionField pt_field_invalid(999, 1000, "missing_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field_invalid}); + EXPECT_FALSE(result.has_value()); + EXPECT_THAT(result.error().message, + ::testing::HasSubstr("Cannot find source column for partition field")); +} + +TEST(PartitionSpecTest, InvalidTransformForType) { + // Test Day transform on string type (should fail) + SchemaField field_string(6, "s", string(), false); + Schema schema_string({field_string}, Schema::kInitialSchemaId); + + PartitionField pt_field_invalid(6, 1005, "s_day", Transform::Day()); + + auto result_string = PartitionSpec::Make(schema_string, 1, {pt_field_invalid}); + EXPECT_FALSE(result_string.has_value()); + EXPECT_THAT(result_string.error().message, ::testing::HasSubstr("Invalid source type")); +} + +TEST(PartitionSpecTest, SourceIdNotFound) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + 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}); + EXPECT_FALSE(result.has_value()); + EXPECT_THAT(result.error().message, + ::testing::HasSubstr("Cannot find source column for partition field")); +} + +TEST(PartitionSpecTest, PartitionFieldInStruct) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + Schema base_schema({field1, field2}, Schema::kInitialSchemaId); + + // Create a struct that contains the base schema fields + auto struct_type = + std::make_shared<StructType>(std::vector<SchemaField>{field1, field2}); + SchemaField outer_struct(11, "MyStruct", struct_type, false); + + Schema schema({outer_struct}, Schema::kInitialSchemaId); + + // Partition on a field within the struct (id field with ID 1) + PartitionField pt_field(1, 1000, "id_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field}); + EXPECT_TRUE(result.has_value()) << result.error().message; Review Comment: Same for below. ########## src/iceberg/util/type_util.h: ########## @@ -0,0 +1,101 @@ +/* + * 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 <functional> +#include <stack> + +#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 +std::unordered_map<int32_t, int32_t> indexParents(const StructType& root_struct) { Review Comment: Please add a test for this. Perhaps in the `type_test.cc` is fine. ########## src/iceberg/util/type_util.h: ########## @@ -0,0 +1,101 @@ +/* + * 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 <functional> +#include <stack> + +#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 +std::unordered_map<int32_t, int32_t> indexParents(const StructType& root_struct) { + std::unordered_map<int32_t, int32_t> id_to_parent; + std::stack<int32_t> parent_id_stack; + + // Recursive function to visit and build parent relationships + std::function<void(const Type&)> visit; + visit = [&](const Type& type) { + switch (type.type_id()) { + case TypeId::kStruct: { + const auto& struct_type = static_cast<const StructType&>(type); + for (const auto& field : struct_type.fields()) { + if (!parent_id_stack.empty()) { + id_to_parent[field.field_id()] = parent_id_stack.top(); Review Comment: `field.field_id()` might be invalid or duplicates with other fields if we don't validate the input `root_struct`. If you don't want to change the return type to be `Result<...>`, perhaps add a comment to let readers know what is the expected input. Otherwise we need to validate `field.field_id()` is non-negative and `id_to_parent.emplace(field.field_id(), parent_id_stack.top())` is successful (i.e. not exists already). ########## src/iceberg/util/type_util.h: ########## @@ -0,0 +1,101 @@ +/* + * 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 <functional> +#include <stack> + +#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 +std::unordered_map<int32_t, int32_t> indexParents(const StructType& root_struct) { Review Comment: ```suggestion ICEBERG_EXPORT static std::unordered_map<int32_t, int32_t> IndexParents(const StructType& root_struct) { ``` ########## src/iceberg/util/type_util.h: ########## @@ -0,0 +1,101 @@ +/* + * 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 <functional> +#include <stack> + +#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 +std::unordered_map<int32_t, int32_t> indexParents(const StructType& root_struct) { + std::unordered_map<int32_t, int32_t> id_to_parent; + std::stack<int32_t> parent_id_stack; + + // Recursive function to visit and build parent relationships + std::function<void(const Type&)> visit; + visit = [&](const Type& type) { Review Comment: ```suggestion auto visit = [&](const Type& type) -> void { ``` ########## src/iceberg/partition_spec.cc: ########## @@ -104,4 +108,68 @@ bool PartitionSpec::Equals(const PartitionSpec& other) const { return spec_id_ == other.spec_id_ && fields_ == other.fields_; } +Status PartitionSpec::Validate(const Schema& schema, bool allowMissingFields) const { + std::unordered_map<int32_t, int32_t> 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 (allowMissingFields && !source_field.has_value()) { + continue; + } + 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 (!partition_field.transform()->CanTransform(*source_type)) { + return InvalidArgument("Invalid source type {} for transform {}", + source_type->ToString(), + partition_field.transform()->ToString()); Review Comment: ```suggestion field_transform->ToString()); ``` ########## src/iceberg/partition_spec.h: ########## @@ -80,6 +80,38 @@ 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 allowMissingFields) 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 last_assigned_field_id The last assigned field ID assigned to ensure new + /// fields get unique IDs. + /// \param allowMissingFields Whether to skip validation for partition fields whose + /// source columns have been dropped from the schema. + /// \return A Result containing the partition spec or an error. + static Result<std::unique_ptr<PartitionSpec>> Make( + const Schema& schema, int32_t spec_id, std::vector<PartitionField> fields, + std::optional<int32_t> last_assigned_field_id = std::nullopt, + bool allowMissingFields = false); Review Comment: ```suggestion bool allow_missing_fields = false); ``` ########## src/iceberg/test/partition_spec_test.cc: ########## @@ -152,4 +152,174 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, MissingSourceColumn) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + SchemaField field3(6, "s", string(), false); + Schema schema({field1, field2, field3}, Schema::kInitialSchemaId); + + // Try to create partition field with non-existent source column ID 999 + PartitionField pt_field_invalid(999, 1000, "missing_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field_invalid}); + EXPECT_FALSE(result.has_value()); + EXPECT_THAT(result.error().message, + ::testing::HasSubstr("Cannot find source column for partition field")); +} + +TEST(PartitionSpecTest, InvalidTransformForType) { + // Test Day transform on string type (should fail) + SchemaField field_string(6, "s", string(), false); + Schema schema_string({field_string}, Schema::kInitialSchemaId); + + PartitionField pt_field_invalid(6, 1005, "s_day", Transform::Day()); + + auto result_string = PartitionSpec::Make(schema_string, 1, {pt_field_invalid}); + EXPECT_FALSE(result_string.has_value()); + EXPECT_THAT(result_string.error().message, ::testing::HasSubstr("Invalid source type")); +} + +TEST(PartitionSpecTest, SourceIdNotFound) { Review Comment: Isn't it duplicate with `TEST(PartitionSpecTest, MissingSourceColumn)`? ########## src/iceberg/util/type_util.h: ########## @@ -0,0 +1,101 @@ +/* + * 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 <functional> +#include <stack> + +#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 +std::unordered_map<int32_t, int32_t> indexParents(const StructType& root_struct) { Review Comment: We must add `static` or `inline` to avoid duplicate definition. It can happen when two files include this. ########## src/iceberg/test/partition_spec_test.cc: ########## @@ -152,4 +152,174 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, MissingSourceColumn) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + SchemaField field3(6, "s", string(), false); + Schema schema({field1, field2, field3}, Schema::kInitialSchemaId); + + // Try to create partition field with non-existent source column ID 999 + PartitionField pt_field_invalid(999, 1000, "missing_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field_invalid}); + EXPECT_FALSE(result.has_value()); + EXPECT_THAT(result.error().message, + ::testing::HasSubstr("Cannot find source column for partition field")); +} + +TEST(PartitionSpecTest, InvalidTransformForType) { + // Test Day transform on string type (should fail) + SchemaField field_string(6, "s", string(), false); + Schema schema_string({field_string}, Schema::kInitialSchemaId); + + PartitionField pt_field_invalid(6, 1005, "s_day", Transform::Day()); + + auto result_string = PartitionSpec::Make(schema_string, 1, {pt_field_invalid}); + EXPECT_FALSE(result_string.has_value()); + EXPECT_THAT(result_string.error().message, ::testing::HasSubstr("Invalid source type")); +} + +TEST(PartitionSpecTest, SourceIdNotFound) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + 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}); + EXPECT_FALSE(result.has_value()); + EXPECT_THAT(result.error().message, + ::testing::HasSubstr("Cannot find source column for partition field")); +} + +TEST(PartitionSpecTest, PartitionFieldInStruct) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + Schema base_schema({field1, field2}, Schema::kInitialSchemaId); + + // Create a struct that contains the base schema fields + auto struct_type = + std::make_shared<StructType>(std::vector<SchemaField>{field1, field2}); + SchemaField outer_struct(11, "MyStruct", struct_type, false); + + Schema schema({outer_struct}, Schema::kInitialSchemaId); + + // Partition on a field within the struct (id field with ID 1) + PartitionField pt_field(1, 1000, "id_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field}); + EXPECT_TRUE(result.has_value()) << result.error().message; Review Comment: ```suggestion EXPECT_THAT(result, IsOK()); ``` ########## src/iceberg/test/partition_spec_test.cc: ########## @@ -152,4 +152,174 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, MissingSourceColumn) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + SchemaField field3(6, "s", string(), false); + Schema schema({field1, field2, field3}, Schema::kInitialSchemaId); + + // Try to create partition field with non-existent source column ID 999 + PartitionField pt_field_invalid(999, 1000, "missing_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field_invalid}); + EXPECT_FALSE(result.has_value()); + EXPECT_THAT(result.error().message, + ::testing::HasSubstr("Cannot find source column for partition field")); +} + +TEST(PartitionSpecTest, InvalidTransformForType) { + // Test Day transform on string type (should fail) + SchemaField field_string(6, "s", string(), false); + Schema schema_string({field_string}, Schema::kInitialSchemaId); + + PartitionField pt_field_invalid(6, 1005, "s_day", Transform::Day()); Review Comment: Can we also check that void transform does not error out? ########## src/iceberg/util/type_util.h: ########## @@ -0,0 +1,101 @@ +/* + * 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 <functional> +#include <stack> + +#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 +std::unordered_map<int32_t, int32_t> indexParents(const StructType& root_struct) { + std::unordered_map<int32_t, int32_t> id_to_parent; + std::stack<int32_t> parent_id_stack; + + // Recursive function to visit and build parent relationships + std::function<void(const Type&)> visit; + visit = [&](const Type& type) { + switch (type.type_id()) { + case TypeId::kStruct: { + const auto& struct_type = static_cast<const StructType&>(type); + for (const auto& field : struct_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; + } + + case TypeId::kList: { + const auto& list_type = static_cast<const ListType&>(type); + const auto& element_field = list_type.fields()[0]; + + if (!parent_id_stack.empty()) { + id_to_parent[element_field.field_id()] = parent_id_stack.top(); + } + + parent_id_stack.push(element_field.field_id()); + visit(*element_field.type()); + parent_id_stack.pop(); + break; + } + + case TypeId::kMap: { + const auto& map_type = static_cast<const MapType&>(type); + const auto& key_field = map_type.key(); + const auto& value_field = map_type.value(); + + if (!parent_id_stack.empty()) { + id_to_parent[key_field.field_id()] = parent_id_stack.top(); + id_to_parent[value_field.field_id()] = parent_id_stack.top(); + } + + parent_id_stack.push(key_field.field_id()); + visit(*key_field.type()); + parent_id_stack.pop(); + + parent_id_stack.push(value_field.field_id()); + visit(*value_field.type()); + parent_id_stack.pop(); + break; + } Review Comment: ```suggestion case TypeId::kStruct: case TypeId::kList: case TypeId::kMap: { const auto& nested_type = static_cast<const NestedType&>(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; } ``` ########## src/iceberg/test/partition_spec_test.cc: ########## @@ -152,4 +152,174 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, MissingSourceColumn) { + SchemaField field1(1, "id", int64(), false); + SchemaField field2(2, "ts", timestamp(), false); + SchemaField field3(6, "s", string(), false); + Schema schema({field1, field2, field3}, Schema::kInitialSchemaId); + + // Try to create partition field with non-existent source column ID 999 + PartitionField pt_field_invalid(999, 1000, "missing_partition", Transform::Identity()); + + auto result = PartitionSpec::Make(schema, 1, {pt_field_invalid}); + EXPECT_FALSE(result.has_value()); Review Comment: Let's verify the error kind. Same for all below. ########## src/iceberg/test/partition_spec_test.cc: ########## @@ -152,4 +152,174 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, MissingSourceColumn) { + SchemaField field1(1, "id", int64(), false); Review Comment: Same for below. The point is to make test shorter. ########## src/iceberg/test/partition_spec_test.cc: ########## @@ -152,4 +152,174 @@ TEST(PartitionSpecTest, PartitionTypeTest) { EXPECT_EQ(pt_field3, partition_type.value()->fields()[2]); } +TEST(PartitionSpecTest, MissingSourceColumn) { + SchemaField field1(1, "id", int64(), false); Review Comment: nit: use SchemaField::MakeRequired and use a one-liner to create the Schema. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
