From d6449774020b2ae0bc7d98b3aad90af099a17d2f Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Sat, 11 Oct 2025 21:59:07 +0800 Subject: [PATCH 01/14] feat: implement metadata updates and update requirements This commit introduces the metadata update and update requirements system for table metadata modifications. --- src/iceberg/CMakeLists.txt | 3 + src/iceberg/metadata_update.cc | 285 +++++++++++++++++++ src/iceberg/metadata_update.h | 431 +++++++++++++++++++++++++++++ src/iceberg/result.h | 2 + src/iceberg/table_metadata.cc | 226 +++++++++++++++ src/iceberg/table_metadata.h | 198 +++++++++++++ src/iceberg/type_fwd.h | 5 +- src/iceberg/update_requirement.cc | 148 ++++++++++ src/iceberg/update_requirement.h | 220 +++++++++++++++ src/iceberg/update_requirements.cc | 77 ++++++ src/iceberg/update_requirements.h | 122 ++++++++ src/iceberg/util/string_util.h | 7 + 12 files changed, 1722 insertions(+), 2 deletions(-) create mode 100644 src/iceberg/metadata_update.cc create mode 100644 src/iceberg/metadata_update.h create mode 100644 src/iceberg/update_requirement.cc create mode 100644 src/iceberg/update_requirement.h create mode 100644 src/iceberg/update_requirements.cc create mode 100644 src/iceberg/update_requirements.h diff --git a/src/iceberg/CMakeLists.txt b/src/iceberg/CMakeLists.txt index 75ac9c847..fc00e8aad 100644 --- a/src/iceberg/CMakeLists.txt +++ b/src/iceberg/CMakeLists.txt @@ -36,6 +36,7 @@ set(ICEBERG_SOURCES manifest_reader_internal.cc manifest_writer.cc metadata_columns.cc + metadata_update.cc name_mapping.cc partition_field.cc partition_spec.cc @@ -56,6 +57,8 @@ set(ICEBERG_SOURCES transform.cc transform_function.cc type.cc + update_requirement.cc + update_requirements.cc util/conversions.cc util/decimal.cc util/gzip_internal.cc diff --git a/src/iceberg/metadata_update.cc b/src/iceberg/metadata_update.cc new file mode 100644 index 000000000..cd171c4de --- /dev/null +++ b/src/iceberg/metadata_update.cc @@ -0,0 +1,285 @@ +/* + * 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. + */ + +#include "iceberg/metadata_update.h" + +#include "iceberg/table_metadata.h" +#include "iceberg/update_requirements.h" + +namespace iceberg { + +// AssignUUID + +void AssignUUID::ApplyTo(TableMetadataBuilder& builder) const { + builder.AssignUUID(uuid_); +} + +void AssignUUID::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // AssignUUID doesn't generate any requirements +} + +// UpgradeFormatVersion + +void UpgradeFormatVersion::ApplyTo(TableMetadataBuilder& builder) const { + builder.UpgradeFormatVersion(format_version_); +} + +void UpgradeFormatVersion::GenerateRequirements( + UpdateRequirementsContext& /*context*/) const { + // UpgradeFormatVersion doesn't generate any requirements +} + +// AddSchema + +void AddSchema::ApplyTo(TableMetadataBuilder& builder) const { + builder.AddSchema(schema_); +} + +void AddSchema::GenerateRequirements(UpdateRequirementsContext& context) const { + if (context.base() != nullptr) { + context.AddRequirement( + std::make_unique(context.base()->last_column_id)); + } +} + +// SetCurrentSchema + +void SetCurrentSchema::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetCurrentSchema(schema_id_); +} + +void SetCurrentSchema::GenerateRequirements(UpdateRequirementsContext& context) const { + // Require current schema not changed + if (context.base() != nullptr && !context.is_replace()) { + if (context.base()->current_schema_id.has_value()) { + context.AddRequirement(std::make_unique( + context.base()->current_schema_id.value())); + } + } +} + +// AddPartitionSpec + +void AddPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { + builder.AddPartitionSpec(spec_); +} + +void AddPartitionSpec::GenerateRequirements(UpdateRequirementsContext& context) const { + if (context.base() != nullptr) { + context.AddRequirement(std::make_unique( + context.base()->last_partition_id)); + } +} + +// SetDefaultPartitionSpec + +void SetDefaultPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetDefaultPartitionSpec(spec_id_); +} + +void SetDefaultPartitionSpec::GenerateRequirements( + UpdateRequirementsContext& context) const { + // Require default partition spec not changed + if (context.base() != nullptr && !context.is_replace()) { + context.AddRequirement( + std::make_unique(context.base()->default_spec_id)); + } +} + +// RemovePartitionSpecs + +void RemovePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemovePartitionSpecs(spec_ids_); +} + +void RemovePartitionSpecs::GenerateRequirements( + UpdateRequirementsContext& context) const { + // Require default partition spec not changed + if (context.base() != nullptr && !context.is_replace()) { + context.AddRequirement( + std::make_unique(context.base()->default_spec_id)); + } + + // Require that no branches have changed + if (context.base() != nullptr && !context.is_replace()) { + for (const auto& [name, ref] : context.base()->refs) { + if (ref->type() == SnapshotRefType::kBranch && name != "main") { + context.AddRequirement( + std::make_unique(name, ref->snapshot_id)); + } + } + } +} + +// RemoveSchemas + +void RemoveSchemas::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemoveSchemas(schema_ids_); +} + +void RemoveSchemas::GenerateRequirements(UpdateRequirementsContext& context) const { + // Require current schema not changed + if (context.base() != nullptr && !context.is_replace()) { + if (context.base()->current_schema_id.has_value()) { + context.AddRequirement(std::make_unique( + context.base()->current_schema_id.value())); + } + } + + // Require that no branches have changed + if (context.base() != nullptr && !context.is_replace()) { + for (const auto& [name, ref] : context.base()->refs) { + if (ref->type() == SnapshotRefType::kBranch && name != "main") { + context.AddRequirement( + std::make_unique(name, ref->snapshot_id)); + } + } + } +} + +// AddSortOrder + +void AddSortOrder::ApplyTo(TableMetadataBuilder& builder) const { + builder.AddSortOrder(sort_order_); +} + +void AddSortOrder::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // AddSortOrder doesn't generate any requirements +} + +// SetDefaultSortOrder + +void SetDefaultSortOrder::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetDefaultSortOrder(sort_order_id_); +} + +void SetDefaultSortOrder::GenerateRequirements(UpdateRequirementsContext& context) const { + if (context.base() != nullptr && !context.is_replace()) { + context.AddRequirement(std::make_unique( + context.base()->default_sort_order_id)); + } +} + +// AddSnapshot + +void AddSnapshot::ApplyTo(TableMetadataBuilder& builder) const { + builder.AddSnapshot(snapshot_); +} + +void AddSnapshot::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // AddSnapshot doesn't generate any requirements +} + +// RemoveSnapshots + +void RemoveSnapshots::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemoveSnapshots(snapshot_ids_); +} + +void RemoveSnapshots::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // RemoveSnapshots doesn't generate any requirements +} + +// RemoveSnapshotRef + +void RemoveSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemoveRef(ref_name_); +} + +void RemoveSnapshotRef::GenerateRequirements( + UpdateRequirementsContext& /*context*/) const { + // RemoveSnapshotRef doesn't generate any requirements +} + +// SetSnapshotRef + +void SetSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { + // Create a SnapshotRef based on the type + std::shared_ptr ref; + + if (type_ == SnapshotRefType::kBranch) { + SnapshotRef::Branch branch; + branch.min_snapshots_to_keep = min_snapshots_to_keep_; + branch.max_snapshot_age_ms = max_snapshot_age_ms_; + branch.max_ref_age_ms = max_ref_age_ms_; + + ref = std::make_shared(); + ref->snapshot_id = snapshot_id_; + ref->retention = branch; + } else { + SnapshotRef::Tag tag; + tag.max_ref_age_ms = max_ref_age_ms_; + + ref = std::make_shared(); + ref->snapshot_id = snapshot_id_; + ref->retention = tag; + } + + builder.SetRef(ref_name_, ref); +} + +void SetSnapshotRef::GenerateRequirements(UpdateRequirementsContext& context) const { + // Require that the ref is unchanged from the base + if (context.base() != nullptr && !context.is_replace()) { + // Find the reference in the base metadata + auto it = context.base()->refs.find(ref_name_); + std::optional base_snapshot_id; + + if (it != context.base()->refs.end()) { + base_snapshot_id = it->second->snapshot_id; + } + + // Require that the ref does not exist (nullopt) or is the same as the base snapshot + context.AddRequirement( + std::make_unique(ref_name_, base_snapshot_id)); + } +} + +// SetProperties + +void SetProperties::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetProperties(updated_); +} + +void SetProperties::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // SetProperties doesn't generate any requirements +} + +// RemoveProperties + +void RemoveProperties::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemoveProperties(removed_); +} + +void RemoveProperties::GenerateRequirements( + UpdateRequirementsContext& /*context*/) const { + // RemoveProperties doesn't generate any requirements +} + +// SetLocation + +void SetLocation::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetLocation(location_); +} + +void SetLocation::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // SetLocation doesn't generate any requirements +} + +} // namespace iceberg diff --git a/src/iceberg/metadata_update.h b/src/iceberg/metadata_update.h new file mode 100644 index 000000000..8aeaea3a6 --- /dev/null +++ b/src/iceberg/metadata_update.h @@ -0,0 +1,431 @@ +/* + * 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 + +/// \file iceberg/metadata_update.h +/// Metadata update operations for Iceberg tables. + +#include +#include +#include +#include +#include + +#include "iceberg/iceberg_export.h" +#include "iceberg/snapshot.h" +#include "iceberg/type_fwd.h" + +namespace iceberg { + +class TableMetadataBuilder; +class UpdateRequirementsContext; + +/// \brief Base class for metadata update operations +/// +/// Represents a change to table metadata. Each concrete subclass +/// represents a specific type of update operation. +class ICEBERG_EXPORT MetadataUpdate { + public: + virtual ~MetadataUpdate() = default; + + /// \brief Clone this metadata update + virtual std::unique_ptr Clone() const = 0; + + /// \brief Apply this update to a TableMetadataBuilder + /// + /// This method modifies the builder by applying the update operation + /// it represents. Each subclass implements this to apply its specific + /// type of update. + /// + /// \param builder The builder to apply this update to + virtual void ApplyTo(TableMetadataBuilder& builder) const = 0; + + /// \brief Generate update requirements for this metadata update + /// + /// This method generates the appropriate UpdateRequirement instances + /// that must be validated before this update can be applied. The context + /// provides information about the base metadata and operation mode. + /// + /// \param context The context containing base metadata and operation state + virtual void GenerateRequirements(UpdateRequirementsContext& context) const = 0; +}; + +/// \brief Represents an assignment of a UUID to the table +class ICEBERG_EXPORT AssignUUID : public MetadataUpdate { + public: + explicit AssignUUID(std::string uuid) : uuid_(std::move(uuid)) {} + + const std::string& uuid() const { return uuid_; } + + std::unique_ptr Clone() const override { + return std::make_unique(uuid_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::string uuid_; +}; + +/// \brief Represents an upgrade of the table format version +class ICEBERG_EXPORT UpgradeFormatVersion : public MetadataUpdate { + public: + explicit UpgradeFormatVersion(int8_t format_version) + : format_version_(format_version) {} + + int8_t format_version() const { return format_version_; } + + std::unique_ptr Clone() const override { + return std::make_unique(format_version_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + int8_t format_version_; +}; + +/// \brief Represents adding a new schema to the table +class ICEBERG_EXPORT AddSchema : public MetadataUpdate { + public: + explicit AddSchema(std::shared_ptr schema, int32_t last_column_id) + : schema_(std::move(schema)), last_column_id_(last_column_id) {} + + const std::shared_ptr& schema() const { return schema_; } + + int32_t last_column_id() const { return last_column_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(schema_, last_column_id_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::shared_ptr schema_; + int32_t last_column_id_; +}; + +/// \brief Represents setting the current schema +class ICEBERG_EXPORT SetCurrentSchema : public MetadataUpdate { + public: + explicit SetCurrentSchema(int32_t schema_id) : schema_id_(schema_id) {} + + int32_t schema_id() const { return schema_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(schema_id_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + int32_t schema_id_; +}; + +/// \brief Represents adding a new partition spec to the table +class ICEBERG_EXPORT AddPartitionSpec : public MetadataUpdate { + public: + explicit AddPartitionSpec(std::shared_ptr spec) + : spec_(std::move(spec)) {} + + const std::shared_ptr& spec() const { return spec_; } + + std::unique_ptr Clone() const override { + return std::make_unique(spec_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::shared_ptr spec_; +}; + +/// \brief Represents setting the default partition spec +class ICEBERG_EXPORT SetDefaultPartitionSpec : public MetadataUpdate { + public: + explicit SetDefaultPartitionSpec(int32_t spec_id) : spec_id_(spec_id) {} + + int32_t spec_id() const { return spec_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(spec_id_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + int32_t spec_id_; +}; + +/// \brief Represents removing partition specs from the table +class ICEBERG_EXPORT RemovePartitionSpecs : public MetadataUpdate { + public: + explicit RemovePartitionSpecs(std::vector spec_ids) + : spec_ids_(std::move(spec_ids)) {} + + const std::vector& spec_ids() const { return spec_ids_; } + + std::unique_ptr Clone() const override { + return std::make_unique(spec_ids_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::vector spec_ids_; +}; + +/// \brief Represents removing schemas from the table +class ICEBERG_EXPORT RemoveSchemas : public MetadataUpdate { + public: + explicit RemoveSchemas(std::vector schema_ids) + : schema_ids_(std::move(schema_ids)) {} + + const std::vector& schema_ids() const { return schema_ids_; } + + std::unique_ptr Clone() const override { + return std::make_unique(schema_ids_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::vector schema_ids_; +}; + +/// \brief Represents adding a new sort order to the table +class ICEBERG_EXPORT AddSortOrder : public MetadataUpdate { + public: + explicit AddSortOrder(std::shared_ptr sort_order) + : sort_order_(std::move(sort_order)) {} + + const std::shared_ptr& sort_order() const { return sort_order_; } + + std::unique_ptr Clone() const override { + return std::make_unique(sort_order_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::shared_ptr sort_order_; +}; + +/// \brief Represents setting the default sort order +class ICEBERG_EXPORT SetDefaultSortOrder : public MetadataUpdate { + public: + explicit SetDefaultSortOrder(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} + + int32_t sort_order_id() const { return sort_order_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(sort_order_id_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + int32_t sort_order_id_; +}; + +/// \brief Represents adding a snapshot to the table +class ICEBERG_EXPORT AddSnapshot : public MetadataUpdate { + public: + explicit AddSnapshot(std::shared_ptr snapshot) + : snapshot_(std::move(snapshot)) {} + + const std::shared_ptr& snapshot() const { return snapshot_; } + + std::unique_ptr Clone() const override { + return std::make_unique(snapshot_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::shared_ptr snapshot_; +}; + +/// \brief Represents removing snapshots from the table +class ICEBERG_EXPORT RemoveSnapshots : public MetadataUpdate { + public: + explicit RemoveSnapshots(std::vector snapshot_ids) + : snapshot_ids_(std::move(snapshot_ids)) {} + + const std::vector& snapshot_ids() const { return snapshot_ids_; } + + std::unique_ptr Clone() const override { + return std::make_unique(snapshot_ids_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::vector snapshot_ids_; +}; + +/// \brief Represents removing a snapshot reference +class ICEBERG_EXPORT RemoveSnapshotRef : public MetadataUpdate { + public: + explicit RemoveSnapshotRef(std::string ref_name) : ref_name_(std::move(ref_name)) {} + + const std::string& ref_name() const { return ref_name_; } + + std::unique_ptr Clone() const override { + return std::make_unique(ref_name_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::string ref_name_; +}; + +/// \brief Represents setting a snapshot reference +class ICEBERG_EXPORT SetSnapshotRef : public MetadataUpdate { + public: + SetSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, + std::optional min_snapshots_to_keep = std::nullopt, + std::optional max_snapshot_age_ms = std::nullopt, + std::optional max_ref_age_ms = std::nullopt) + : ref_name_(std::move(ref_name)), + snapshot_id_(snapshot_id), + type_(type), + min_snapshots_to_keep_(min_snapshots_to_keep), + max_snapshot_age_ms_(max_snapshot_age_ms), + max_ref_age_ms_(max_ref_age_ms) {} + + const std::string& ref_name() const { return ref_name_; } + int64_t snapshot_id() const { return snapshot_id_; } + SnapshotRefType type() const { return type_; } + const std::optional& min_snapshots_to_keep() const { + return min_snapshots_to_keep_; + } + const std::optional& max_snapshot_age_ms() const { + return max_snapshot_age_ms_; + } + const std::optional& max_ref_age_ms() const { return max_ref_age_ms_; } + + std::unique_ptr Clone() const override { + return std::make_unique(ref_name_, snapshot_id_, type_, + min_snapshots_to_keep_, max_snapshot_age_ms_, + max_ref_age_ms_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::string ref_name_; + int64_t snapshot_id_; + SnapshotRefType type_; + std::optional min_snapshots_to_keep_; + std::optional max_snapshot_age_ms_; + std::optional max_ref_age_ms_; +}; + +/// \brief Represents setting table properties +class ICEBERG_EXPORT SetProperties : public MetadataUpdate { + public: + explicit SetProperties(std::unordered_map updated) + : updated_(std::move(updated)) {} + + const std::unordered_map& updated() const { return updated_; } + + std::unique_ptr Clone() const override { + return std::make_unique(updated_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::unordered_map updated_; +}; + +/// \brief Represents removing table properties +class ICEBERG_EXPORT RemoveProperties : public MetadataUpdate { + public: + explicit RemoveProperties(std::vector removed) + : removed_(std::move(removed)) {} + + const std::vector& removed() const { return removed_; } + + std::unique_ptr Clone() const override { + return std::make_unique(removed_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::vector removed_; +}; + +/// \brief Represents setting the table location +class ICEBERG_EXPORT SetLocation : public MetadataUpdate { + public: + explicit SetLocation(std::string location) : location_(std::move(location)) {} + + const std::string& location() const { return location_; } + + std::unique_ptr Clone() const override { + return std::make_unique(location_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::string location_; +}; + +} // namespace iceberg diff --git a/src/iceberg/result.h b/src/iceberg/result.h index 79dd52b93..99df37247 100644 --- a/src/iceberg/result.h +++ b/src/iceberg/result.h @@ -30,6 +30,7 @@ namespace iceberg { /// \brief Error types for iceberg. enum class ErrorKind { kAlreadyExists, + kCommitFailed, kCommitStateUnknown, kDecompressError, kInvalid, // For general invalid errors @@ -78,6 +79,7 @@ using Status = Result; } DEFINE_ERROR_FUNCTION(AlreadyExists) +DEFINE_ERROR_FUNCTION(CommitFailed) DEFINE_ERROR_FUNCTION(CommitStateUnknown) DEFINE_ERROR_FUNCTION(DecompressError) DEFINE_ERROR_FUNCTION(Invalid) diff --git a/src/iceberg/table_metadata.cc b/src/iceberg/table_metadata.cc index e58d06aeb..5247c82ba 100644 --- a/src/iceberg/table_metadata.cc +++ b/src/iceberg/table_metadata.cc @@ -19,6 +19,7 @@ #include "iceberg/table_metadata.h" +#include #include #include @@ -196,4 +197,229 @@ Status TableMetadataUtil::Write(FileIO& io, const std::string& location, return io.WriteFile(location, json_string); } +// TableMetadataBuilder implementation + +struct TableMetadataBuilder::Impl { + // Base metadata (if building from existing metadata) + std::shared_ptr base; + + // Mutable fields that will be used to build the final TableMetadata + int8_t format_version; + std::string table_uuid; + std::string location; + int64_t last_sequence_number; + TimePointMs last_updated_ms; + int32_t last_column_id; + std::vector> schemas; + std::optional current_schema_id; + std::vector> partition_specs; + int32_t default_spec_id; + int32_t last_partition_id; + std::unordered_map properties; + int64_t current_snapshot_id; + std::vector> snapshots; + std::vector snapshot_log; + std::vector metadata_log; + std::vector> sort_orders; + int32_t default_sort_order_id; + std::unordered_map> refs; + std::vector> statistics; + std::vector> partition_statistics; + int64_t next_row_id; + + // List of changes (MetadataUpdate objects) + std::vector> changes; + + explicit Impl(int8_t fmt_version) + : format_version(fmt_version), + last_sequence_number(TableMetadata::kInitialSequenceNumber), + last_updated_ms(std::chrono::milliseconds(0)), + last_column_id(0), + default_spec_id(0), + last_partition_id(0), + current_snapshot_id(-1), + default_sort_order_id(0), + next_row_id(TableMetadata::kInitialRowId) {} + + explicit Impl(const std::shared_ptr& base_metadata) + : base(base_metadata), + format_version(base_metadata->format_version), + table_uuid(base_metadata->table_uuid), + location(base_metadata->location), + last_sequence_number(base_metadata->last_sequence_number), + last_updated_ms(base_metadata->last_updated_ms), + last_column_id(base_metadata->last_column_id), + schemas(base_metadata->schemas), + current_schema_id(base_metadata->current_schema_id), + partition_specs(base_metadata->partition_specs), + default_spec_id(base_metadata->default_spec_id), + last_partition_id(base_metadata->last_partition_id), + properties(base_metadata->properties), + current_snapshot_id(base_metadata->current_snapshot_id), + snapshots(base_metadata->snapshots), + snapshot_log(base_metadata->snapshot_log), + metadata_log(base_metadata->metadata_log), + sort_orders(base_metadata->sort_orders), + default_sort_order_id(base_metadata->default_sort_order_id), + refs(base_metadata->refs), + statistics(base_metadata->statistics), + partition_statistics(base_metadata->partition_statistics), + next_row_id(base_metadata->next_row_id) {} +}; + +TableMetadataBuilder::TableMetadataBuilder(int8_t format_version) + : impl_(std::make_unique(format_version)) {} + +TableMetadataBuilder::TableMetadataBuilder(std::shared_ptr base) + : impl_(std::make_unique(base)) {} + +TableMetadataBuilder::~TableMetadataBuilder() = default; + +TableMetadataBuilder::TableMetadataBuilder(TableMetadataBuilder&&) noexcept = default; + +TableMetadataBuilder& TableMetadataBuilder::operator=(TableMetadataBuilder&&) noexcept = + default; + +TableMetadataBuilder TableMetadataBuilder::BuildFromEmpty(int8_t format_version) { + return TableMetadataBuilder(format_version); +} + +TableMetadataBuilder TableMetadataBuilder::BuildFrom( + const std::shared_ptr& base) { + return TableMetadataBuilder(base); +} + +TableMetadataBuilder& TableMetadataBuilder::AssignUUID() { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AssignUUID(const std::string& uuid) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::UpgradeFormatVersion( + int8_t new_format_version) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetCurrentSchema( + std::shared_ptr schema, int32_t new_last_column_id) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetCurrentSchema(int32_t schema_id) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddSchema(std::shared_ptr schema) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetDefaultPartitionSpec( + std::shared_ptr spec) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetDefaultPartitionSpec(int32_t spec_id) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddPartitionSpec( + std::shared_ptr spec) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemovePartitionSpecs( + const std::vector& spec_ids) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveSchemas( + const std::vector& schema_ids) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetDefaultSortOrder( + std::shared_ptr order) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetDefaultSortOrder(int32_t order_id) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddSortOrder( + std::shared_ptr order) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddSnapshot( + std::shared_ptr snapshot) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetBranchSnapshot(int64_t snapshot_id, + const std::string& branch) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetRef(const std::string& name, + std::shared_ptr ref) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveRef(const std::string& name) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveSnapshots( + const std::vector& snapshot_ids) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetProperties( + const std::unordered_map& updated) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveProperties( + const std::vector& removed) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetLocation(const std::string& location) { + // TODO(gty404): Implement + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::DiscardChanges() { + // TODO(gty404): Implement + return *this; +} + +Result> TableMetadataBuilder::Build() { + return NotImplemented("TableMetadataBuilder::Build is not implemented"); +} + } // namespace iceberg diff --git a/src/iceberg/table_metadata.h b/src/iceberg/table_metadata.h index 427447a10..e80a351d9 100644 --- a/src/iceberg/table_metadata.h +++ b/src/iceberg/table_metadata.h @@ -29,6 +29,7 @@ #include #include "iceberg/iceberg_export.h" +#include "iceberg/metadata_update.h" #include "iceberg/type_fwd.h" #include "iceberg/util/timepoint.h" @@ -144,6 +145,203 @@ ICEBERG_EXPORT std::string ToString(const SnapshotLogEntry& entry); /// \brief Returns a string representation of a MetadataLogEntry ICEBERG_EXPORT std::string ToString(const MetadataLogEntry& entry); +/// \brief Builder class for constructing TableMetadata objects +/// +/// This builder provides a fluent interface for creating and modifying table metadata. +/// It supports both creating new tables and building from existing metadata. +/// +/// Each modification method generates a corresponding MetadataUpdate that is tracked +/// in a changes list. This allows the builder to maintain a complete history of all +/// modifications made to the table metadata, which is important for tracking table +/// evolution and for serialization purposes. +class ICEBERG_EXPORT TableMetadataBuilder { + public: + /// \brief Create a builder for a new table + /// + /// \param format_version The format version for the table + /// \return A new TableMetadataBuilder instance + static TableMetadataBuilder BuildFromEmpty( + int8_t format_version = TableMetadata::kDefaultTableFormatVersion); + + /// \brief Create a builder from existing table metadata + /// + /// \param base The base table metadata to build from + /// \return A new TableMetadataBuilder instance initialized with base metadata + static TableMetadataBuilder BuildFrom(const std::shared_ptr& base); + + /// \brief Assign a UUID to the table + /// + /// If no UUID is provided, a random UUID will be generated. + /// \return Reference to this builder for method chaining + TableMetadataBuilder& AssignUUID(); + + /// \brief Assign a specific UUID to the table + /// + /// \param uuid The UUID string to assign + /// \return Reference to this builder for method chaining + TableMetadataBuilder& AssignUUID(const std::string& uuid); + + /// \brief Upgrade the format version of the table + /// + /// \param new_format_version The new format version (must be >= current version) + /// \return Reference to this builder for method chaining + TableMetadataBuilder& UpgradeFormatVersion(int8_t new_format_version); + + /// \brief Set the current schema for the table + /// + /// \param schema The schema to set as current + /// \param new_last_column_id The highest column ID in the schema + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetCurrentSchema(std::shared_ptr schema, + int32_t new_last_column_id); + + /// \brief Set the current schema by schema ID + /// + /// \param schema_id The ID of the schema to set as current + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetCurrentSchema(int32_t schema_id); + + /// \brief Add a schema to the table + /// + /// \param schema The schema to add + /// \return Reference to this builder for method chaining + TableMetadataBuilder& AddSchema(std::shared_ptr schema); + + /// \brief Set the default partition spec for the table + /// + /// \param spec The partition spec to set as default + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetDefaultPartitionSpec( + std::shared_ptr spec); + + /// \brief Set the default partition spec by spec ID + /// + /// \param spec_id The ID of the partition spec to set as default + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetDefaultPartitionSpec(int32_t spec_id); + + /// \brief Add a partition spec to the table + /// + /// \param spec The partition spec to add + /// \return Reference to this builder for method chaining + TableMetadataBuilder& AddPartitionSpec(std::shared_ptr spec); + + /// \brief Remove partition specs from the table + /// + /// \param spec_ids The IDs of partition specs to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemovePartitionSpecs(const std::vector& spec_ids); + + /// \brief Remove schemas from the table + /// + /// \param schema_ids The IDs of schemas to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemoveSchemas(const std::vector& schema_ids); + + /// \brief Set the default sort order for the table + /// + /// \param order The sort order to set as default + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetDefaultSortOrder(std::shared_ptr order); + + /// \brief Set the default sort order by order ID + /// + /// \param order_id The ID of the sort order to set as default + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetDefaultSortOrder(int32_t order_id); + + /// \brief Add a sort order to the table + /// + /// \param order The sort order to add + /// \return Reference to this builder for method chaining + TableMetadataBuilder& AddSortOrder(std::shared_ptr order); + + /// \brief Add a snapshot to the table + /// + /// \param snapshot The snapshot to add + /// \return Reference to this builder for method chaining + TableMetadataBuilder& AddSnapshot(std::shared_ptr snapshot); + + /// \brief Set a branch to point to a specific snapshot + /// + /// \param snapshot_id The snapshot ID the branch should reference + /// \param branch The name of the branch + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetBranchSnapshot(int64_t snapshot_id, const std::string& branch); + + /// \brief Set a snapshot reference + /// + /// \param name The name of the reference + /// \param ref The snapshot reference to set + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetRef(const std::string& name, std::shared_ptr ref); + + /// \brief Remove a snapshot reference + /// + /// \param name The name of the reference to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemoveRef(const std::string& name); + + /// \brief Remove snapshots from the table + /// + /// \param snapshot_ids The IDs of snapshots to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemoveSnapshots(const std::vector& snapshot_ids); + + /// \brief Set table properties + /// + /// \param updated Map of properties to set or update + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetProperties( + const std::unordered_map& updated); + + /// \brief Remove table properties + /// + /// \param removed Set of property keys to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemoveProperties(const std::vector& removed); + + /// \brief Set the table location + /// + /// \param location The table base location + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetLocation(const std::string& location); + + /// \brief Discard all accumulated changes + /// + /// This is useful when you want to reset the builder state without + /// creating a new builder instance. + /// \return Reference to this builder for method chaining + TableMetadataBuilder& DiscardChanges(); + + /// \brief Build the TableMetadata object + /// + /// \return A Result containing the constructed TableMetadata or an error + Result> Build(); + + /// \brief Destructor + ~TableMetadataBuilder(); + + // Delete copy operations (use BuildFrom to create a new builder) + TableMetadataBuilder(const TableMetadataBuilder&) = delete; + TableMetadataBuilder& operator=(const TableMetadataBuilder&) = delete; + + // Enable move operations + TableMetadataBuilder(TableMetadataBuilder&&) noexcept; + TableMetadataBuilder& operator=(TableMetadataBuilder&&) noexcept; + + private: + /// \brief Private constructor for building from empty state + explicit TableMetadataBuilder(int8_t format_version); + + /// \brief Private constructor for building from existing metadata + explicit TableMetadataBuilder(std::shared_ptr base); + + /// Internal state members + struct Impl; + std::unique_ptr impl_; +}; + /// \brief The codec type of the table metadata file. enum class ICEBERG_EXPORT MetadataFileCodecType { kNone, diff --git a/src/iceberg/type_fwd.h b/src/iceberg/type_fwd.h index bdc5c1e35..24eaef535 100644 --- a/src/iceberg/type_fwd.h +++ b/src/iceberg/type_fwd.h @@ -142,12 +142,13 @@ class StructLike; class ArrayLike; class MapLike; +class MetadataUpdate; +class UpdateRequirement; + /// ---------------------------------------------------------------------------- /// TODO: Forward declarations below are not added yet. /// ---------------------------------------------------------------------------- -class MetadataUpdate; -class UpdateRequirement; class AppendFiles; } // namespace iceberg diff --git a/src/iceberg/update_requirement.cc b/src/iceberg/update_requirement.cc new file mode 100644 index 000000000..25451f312 --- /dev/null +++ b/src/iceberg/update_requirement.cc @@ -0,0 +1,148 @@ +/* + * 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. + */ + +#include "iceberg/update_requirement.h" + +#include "iceberg/table_metadata.h" +#include "iceberg/util/string_util.h" + +namespace iceberg { + +Status AssertTableDoesNotExist::Validate(const TableMetadata* base) const { + if (base != nullptr) { + return CommitFailed("Requirement failed: table already exists"); + } + return {}; +} + +Status AssertTableUUID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + if (uuid_.empty()) { + return CommitFailed("Requirement failed: expected non-empty UUID"); + } + + if (!StringUtils::EqualsIgnoreCase(uuid_, base->table_uuid)) { + return CommitFailed("Requirement failed: UUID does not match: expected {} != {}", + base->table_uuid, uuid_); + } + + return {}; +} + +Status AssertRefSnapshotID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + // Find the reference in the table metadata + auto it = base->refs.find(ref_name_); + + if (it != base->refs.end()) { + // Reference exists + const auto& ref = it->second; + std::string type = (ref->type() == SnapshotRefType::kBranch) ? "branch" : "tag"; + + if (!snapshot_id_.has_value()) { + // A null snapshot ID means the ref should not exist already + return CommitFailed("Requirement failed: {} {} was created concurrently", type, + ref_name_); + } + if (snapshot_id_.value() != ref->snapshot_id) { + return CommitFailed("Requirement failed: {} {} has changed: expected id {} != {}", + type, ref_name_, snapshot_id_.value(), ref->snapshot_id); + } + } else { + // Reference does not exist + if (snapshot_id_.has_value()) { + return CommitFailed("Requirement failed: branch or tag {} is missing, expected {}", + ref_name_, snapshot_id_.value()); + } + } + + return {}; +} + +Status AssertLastAssignedFieldId::Validate(const TableMetadata* base) const { + if (base != nullptr && base->last_column_id != last_assigned_field_id_) { + return CommitFailed( + "Requirement failed: last assigned field id changed: expected id {} != {}", + last_assigned_field_id_, base->last_column_id); + } + return {}; +} + +Status AssertCurrentSchemaID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + if (!base->current_schema_id.has_value()) { + return CommitFailed("Requirement failed: table has no current schema"); + } + + if (schema_id_ != base->current_schema_id.value()) { + return CommitFailed( + "Requirement failed: current schema changed: expected id {} != {}", schema_id_, + base->current_schema_id.value()); + } + + return {}; +} + +Status AssertLastAssignedPartitionId::Validate(const TableMetadata* base) const { + if (base != nullptr && base->last_partition_id != last_assigned_partition_id_) { + return CommitFailed( + "Requirement failed: last assigned partition id changed: expected id {} != {}", + last_assigned_partition_id_, base->last_partition_id); + } + return {}; +} + +Status AssertDefaultSpecID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + if (spec_id_ != base->default_spec_id) { + return CommitFailed( + "Requirement failed: default partition spec changed: expected id {} != {}", + spec_id_, base->default_spec_id); + } + + return {}; +} + +Status AssertDefaultSortOrderID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + if (sort_order_id_ != base->default_sort_order_id) { + return CommitFailed( + "Requirement failed: default sort order changed: expected id {} != {}", + sort_order_id_, base->default_sort_order_id); + } + + return {}; +} + +} // namespace iceberg diff --git a/src/iceberg/update_requirement.h b/src/iceberg/update_requirement.h new file mode 100644 index 000000000..50b570c30 --- /dev/null +++ b/src/iceberg/update_requirement.h @@ -0,0 +1,220 @@ +/* + * 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 + +/// \file iceberg/update_requirement.h +/// Update requirements for Iceberg table operations. +/// +/// Update requirements are conditions that must be satisfied before +/// applying metadata updates to a table. They are used for optimistic +/// concurrency control in table operations. + +#include +#include +#include + +#include "iceberg/iceberg_export.h" +#include "iceberg/result.h" +#include "iceberg/type_fwd.h" + +namespace iceberg { + +/// \brief Base class for update requirement operations +/// +/// Represents a requirement that must be validated before applying +/// metadata updates to a table. Each concrete subclass represents +/// a specific type of requirement check. +class ICEBERG_EXPORT UpdateRequirement { + public: + virtual ~UpdateRequirement() = default; + + /// \brief Clone this update requirement + virtual std::unique_ptr Clone() const = 0; + + /// \brief Validate this requirement against table metadata + /// + /// \param base The base table metadata to validate against (may be nullptr) + /// \return Status indicating success or failure with error details + virtual Status Validate(const TableMetadata* base) const = 0; +}; + +/// \brief Requirement that the table does not exist +/// +/// This requirement is used when creating a new table to ensure +/// it doesn't already exist. +class ICEBERG_EXPORT AssertTableDoesNotExist : public UpdateRequirement { + public: + AssertTableDoesNotExist() = default; + + std::unique_ptr Clone() const override { + return std::make_unique(); + } + + Status Validate(const TableMetadata* base) const override; +}; + +/// \brief Requirement that the table UUID matches the expected value +/// +/// This ensures the table hasn't been replaced or recreated between +/// reading the metadata and attempting to update it. +class ICEBERG_EXPORT AssertTableUUID : public UpdateRequirement { + public: + explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} + + const std::string& uuid() const { return uuid_; } + + std::unique_ptr Clone() const override { + return std::make_unique(uuid_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + std::string uuid_; +}; + +/// \brief Requirement that a reference (branch or tag) points to a specific snapshot +/// +/// This requirement validates that a named reference (branch or tag) either: +/// - Points to the expected snapshot ID +/// - Does not exist (if snapshot_id is nullopt) +class ICEBERG_EXPORT AssertRefSnapshotID : public UpdateRequirement { + public: + AssertRefSnapshotID(std::string ref_name, std::optional snapshot_id) + : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} + + const std::string& ref_name() const { return ref_name_; } + const std::optional& snapshot_id() const { return snapshot_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(ref_name_, snapshot_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + std::string ref_name_; + std::optional snapshot_id_; +}; + +/// \brief Requirement that the last assigned field ID matches +/// +/// This ensures the schema hasn't been modified (by adding fields) +/// since the metadata was read. +class ICEBERG_EXPORT AssertLastAssignedFieldId : public UpdateRequirement { + public: + explicit AssertLastAssignedFieldId(int32_t last_assigned_field_id) + : last_assigned_field_id_(last_assigned_field_id) {} + + int32_t last_assigned_field_id() const { return last_assigned_field_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(last_assigned_field_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t last_assigned_field_id_; +}; + +/// \brief Requirement that the current schema ID matches +/// +/// This ensures the active schema hasn't changed since the +/// metadata was read. +class ICEBERG_EXPORT AssertCurrentSchemaID : public UpdateRequirement { + public: + explicit AssertCurrentSchemaID(int32_t schema_id) : schema_id_(schema_id) {} + + int32_t schema_id() const { return schema_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(schema_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t schema_id_; +}; + +/// \brief Requirement that the last assigned partition ID matches +/// +/// This ensures partition specs haven't been modified since the +/// metadata was read. +class ICEBERG_EXPORT AssertLastAssignedPartitionId : public UpdateRequirement { + public: + explicit AssertLastAssignedPartitionId(int32_t last_assigned_partition_id) + : last_assigned_partition_id_(last_assigned_partition_id) {} + + int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(last_assigned_partition_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t last_assigned_partition_id_; +}; + +/// \brief Requirement that the default partition spec ID matches +/// +/// This ensures the default partition spec hasn't changed since +/// the metadata was read. +class ICEBERG_EXPORT AssertDefaultSpecID : public UpdateRequirement { + public: + explicit AssertDefaultSpecID(int32_t spec_id) : spec_id_(spec_id) {} + + int32_t spec_id() const { return spec_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(spec_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t spec_id_; +}; + +/// \brief Requirement that the default sort order ID matches +/// +/// This ensures the default sort order hasn't changed since +/// the metadata was read. +class ICEBERG_EXPORT AssertDefaultSortOrderID : public UpdateRequirement { + public: + explicit AssertDefaultSortOrderID(int32_t sort_order_id) + : sort_order_id_(sort_order_id) {} + + int32_t sort_order_id() const { return sort_order_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(sort_order_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t sort_order_id_; +}; + +} // namespace iceberg diff --git a/src/iceberg/update_requirements.cc b/src/iceberg/update_requirements.cc new file mode 100644 index 000000000..d3c2df959 --- /dev/null +++ b/src/iceberg/update_requirements.cc @@ -0,0 +1,77 @@ +/* + * 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. + */ + +#include "iceberg/update_requirements.h" + +#include "iceberg/metadata_update.h" +#include "iceberg/table_metadata.h" + +namespace iceberg { + +std::vector> UpdateRequirements::ForCreateTable( + const std::vector>& metadata_updates) { + // Create context for table creation (no base metadata) + UpdateRequirementsContext context(nullptr, false); + + // Add requirement that table does not exist + context.AddRequirement(std::make_unique()); + + // Let each metadata update generate its requirements + for (const auto& update : metadata_updates) { + update->GenerateRequirements(context); + } + + return context.Build(); +} + +std::vector> UpdateRequirements::ForReplaceTable( + const TableMetadata& base, + const std::vector>& metadata_updates) { + // Create context for table replacement (is_replace = true) + UpdateRequirementsContext context(&base, true); + + // Add requirement that UUID matches + context.AddRequirement(std::make_unique(base.table_uuid)); + + // Let each metadata update generate its requirements + for (const auto& update : metadata_updates) { + update->GenerateRequirements(context); + } + + return context.Build(); +} + +std::vector> UpdateRequirements::ForUpdateTable( + const TableMetadata& base, + const std::vector>& metadata_updates) { + // Create context for table update (is_replace = false) + UpdateRequirementsContext context(&base, false); + + // Add requirement that UUID matches + context.AddRequirement(std::make_unique(base.table_uuid)); + + // Let each metadata update generate its requirements + for (const auto& update : metadata_updates) { + update->GenerateRequirements(context); + } + + return context.Build(); +} + +} // namespace iceberg diff --git a/src/iceberg/update_requirements.h b/src/iceberg/update_requirements.h new file mode 100644 index 000000000..d7ad97463 --- /dev/null +++ b/src/iceberg/update_requirements.h @@ -0,0 +1,122 @@ +/* + * 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 + +/// \file iceberg/update_requirements.h +/// Factory for generating update requirements from metadata updates. +/// +/// This utility class generates the appropriate UpdateRequirement instances +/// based on a list of MetadataUpdate operations. The requirements are used +/// for optimistic concurrency control when committing table changes. + +#include +#include + +#include "iceberg/iceberg_export.h" +#include "iceberg/type_fwd.h" +#include "iceberg/update_requirement.h" + +namespace iceberg { + +/// \brief Context for generating update requirements +/// +/// This context is passed to each MetadataUpdate's GenerateRequirements method +/// and maintains state about what requirements have already been added to avoid +/// duplicates. +class ICEBERG_EXPORT UpdateRequirementsContext { + public: + /// \brief Construct a context for requirement generation + /// + /// \param base The base table metadata (may be nullptr for table creation) + /// \param is_replace Whether this is a replace operation (more permissive) + UpdateRequirementsContext(const TableMetadata* base, bool is_replace) + : base_(base), is_replace_(is_replace) {} + + // Delete copy operations (contains unique_ptr members) + UpdateRequirementsContext(const UpdateRequirementsContext&) = delete; + UpdateRequirementsContext& operator=(const UpdateRequirementsContext&) = delete; + + // Enable move construction only (assignment deleted due to const members) + UpdateRequirementsContext(UpdateRequirementsContext&&) noexcept = default; + + /// \brief Add a requirement to the list + void AddRequirement(std::unique_ptr requirement) { + requirements_.push_back(std::move(requirement)); + } + + /// \brief Get the base table metadata + const TableMetadata* base() const { return base_; } + + /// \brief Check if this is a replace operation + bool is_replace() const { return is_replace_; } + + /// \brief Build and return the list of requirements + std::vector> Build() { + return std::move(requirements_); + } + + private: + const TableMetadata* base_; + const bool is_replace_; + + std::vector> requirements_; +}; + +/// \brief Factory class for generating update requirements +/// +/// This class analyzes a sequence of metadata updates and generates the +/// appropriate update requirements to ensure safe concurrent modifications. +class ICEBERG_EXPORT UpdateRequirements { + public: + /// \brief Generate requirements for creating a new table + /// + /// For table creation, this requires that the table does not already exist. + /// + /// \param metadata_updates The list of metadata updates for table creation + /// \return A list of update requirements to validate before creation + static std::vector> ForCreateTable( + const std::vector>& metadata_updates); + + /// \brief Generate requirements for replacing an existing table + /// + /// For table replacement, this requires that the table UUID matches but + /// allows more aggressive changes than a regular update. + /// + /// \param base The base table metadata + /// \param metadata_updates The list of metadata updates for replacement + /// \return A list of update requirements to validate before replacement + static std::vector> ForReplaceTable( + const TableMetadata& base, + const std::vector>& metadata_updates); + + /// \brief Generate requirements for updating an existing table + /// + /// For table updates, this generates requirements to ensure that key + /// metadata properties haven't changed concurrently. + /// + /// \param base The base table metadata + /// \param metadata_updates The list of metadata updates + /// \return A list of update requirements to validate before update + static std::vector> ForUpdateTable( + const TableMetadata& base, + const std::vector>& metadata_updates); +}; + +} // namespace iceberg diff --git a/src/iceberg/util/string_util.h b/src/iceberg/util/string_util.h index a0fccfd35..e12a1d098 100644 --- a/src/iceberg/util/string_util.h +++ b/src/iceberg/util/string_util.h @@ -44,6 +44,13 @@ class ICEBERG_EXPORT StringUtils { [](char c) { return std::toupper(c); }); // NOLINT return input; } + + static bool EqualsIgnoreCase(const std::string& a, const std::string& b) { + return a.size() == b.size() && + std::equal(a.begin(), a.end(), b.begin(), [](char ca, char cb) { + return std::tolower(ca) == std::tolower(cb); + }); + } }; /// \brief Transparent hash function that supports std::string_view as lookup key From a1c800c7a2926c64394cad9289b8c2576748ff6a Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Wed, 15 Oct 2025 09:27:48 +0800 Subject: [PATCH 02/14] Fix the string comparison issue --- src/iceberg/util/string_util.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/iceberg/util/string_util.h b/src/iceberg/util/string_util.h index e12a1d098..66bb7b525 100644 --- a/src/iceberg/util/string_util.h +++ b/src/iceberg/util/string_util.h @@ -46,8 +46,7 @@ class ICEBERG_EXPORT StringUtils { } static bool EqualsIgnoreCase(const std::string& a, const std::string& b) { - return a.size() == b.size() && - std::equal(a.begin(), a.end(), b.begin(), [](char ca, char cb) { + return a.size() == b.size() && std::ranges::equal(a, b, [](char ca, char cb) { return std::tolower(ca) == std::tolower(cb); }); } From 994741c84619b7878f1226767c1a2d445287f971 Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Wed, 15 Oct 2025 10:06:14 +0800 Subject: [PATCH 03/14] fix comments --- src/iceberg/metadata_update.h | 24 +++++++-------- src/iceberg/table_metadata.cc | 50 ++++++++++++++----------------- src/iceberg/table_metadata.h | 24 +++++++-------- src/iceberg/update_requirement.cc | 1 + 4 files changed, 46 insertions(+), 53 deletions(-) diff --git a/src/iceberg/metadata_update.h b/src/iceberg/metadata_update.h index 8aeaea3a6..bf5dfc8e2 100644 --- a/src/iceberg/metadata_update.h +++ b/src/iceberg/metadata_update.h @@ -109,10 +109,10 @@ class ICEBERG_EXPORT UpgradeFormatVersion : public MetadataUpdate { /// \brief Represents adding a new schema to the table class ICEBERG_EXPORT AddSchema : public MetadataUpdate { public: - explicit AddSchema(std::shared_ptr schema, int32_t last_column_id) + explicit AddSchema(std::shared_ptr schema, int32_t last_column_id) : schema_(std::move(schema)), last_column_id_(last_column_id) {} - const std::shared_ptr& schema() const { return schema_; } + const std::shared_ptr& schema() const { return schema_; } int32_t last_column_id() const { return last_column_id_; } @@ -125,7 +125,7 @@ class ICEBERG_EXPORT AddSchema : public MetadataUpdate { void GenerateRequirements(UpdateRequirementsContext& context) const override; private: - std::shared_ptr schema_; + std::shared_ptr schema_; int32_t last_column_id_; }; @@ -151,10 +151,10 @@ class ICEBERG_EXPORT SetCurrentSchema : public MetadataUpdate { /// \brief Represents adding a new partition spec to the table class ICEBERG_EXPORT AddPartitionSpec : public MetadataUpdate { public: - explicit AddPartitionSpec(std::shared_ptr spec) + explicit AddPartitionSpec(std::shared_ptr spec) : spec_(std::move(spec)) {} - const std::shared_ptr& spec() const { return spec_; } + const std::shared_ptr& spec() const { return spec_; } std::unique_ptr Clone() const override { return std::make_unique(spec_); @@ -165,7 +165,7 @@ class ICEBERG_EXPORT AddPartitionSpec : public MetadataUpdate { void GenerateRequirements(UpdateRequirementsContext& context) const override; private: - std::shared_ptr spec_; + std::shared_ptr spec_; }; /// \brief Represents setting the default partition spec @@ -230,10 +230,10 @@ class ICEBERG_EXPORT RemoveSchemas : public MetadataUpdate { /// \brief Represents adding a new sort order to the table class ICEBERG_EXPORT AddSortOrder : public MetadataUpdate { public: - explicit AddSortOrder(std::shared_ptr sort_order) + explicit AddSortOrder(std::shared_ptr sort_order) : sort_order_(std::move(sort_order)) {} - const std::shared_ptr& sort_order() const { return sort_order_; } + const std::shared_ptr& sort_order() const { return sort_order_; } std::unique_ptr Clone() const override { return std::make_unique(sort_order_); @@ -244,7 +244,7 @@ class ICEBERG_EXPORT AddSortOrder : public MetadataUpdate { void GenerateRequirements(UpdateRequirementsContext& context) const override; private: - std::shared_ptr sort_order_; + std::shared_ptr sort_order_; }; /// \brief Represents setting the default sort order @@ -269,10 +269,10 @@ class ICEBERG_EXPORT SetDefaultSortOrder : public MetadataUpdate { /// \brief Represents adding a snapshot to the table class ICEBERG_EXPORT AddSnapshot : public MetadataUpdate { public: - explicit AddSnapshot(std::shared_ptr snapshot) + explicit AddSnapshot(std::shared_ptr snapshot) : snapshot_(std::move(snapshot)) {} - const std::shared_ptr& snapshot() const { return snapshot_; } + const std::shared_ptr& snapshot() const { return snapshot_; } std::unique_ptr Clone() const override { return std::make_unique(snapshot_); @@ -283,7 +283,7 @@ class ICEBERG_EXPORT AddSnapshot : public MetadataUpdate { void GenerateRequirements(UpdateRequirementsContext& context) const override; private: - std::shared_ptr snapshot_; + std::shared_ptr snapshot_; }; /// \brief Represents removing snapshots from the table diff --git a/src/iceberg/table_metadata.cc b/src/iceberg/table_metadata.cc index 5247c82ba..742e5cc0b 100644 --- a/src/iceberg/table_metadata.cc +++ b/src/iceberg/table_metadata.cc @@ -27,6 +27,7 @@ #include "iceberg/file_io.h" #include "iceberg/json_internal.h" +#include "iceberg/metadata_update.h" #include "iceberg/partition_spec.h" #include "iceberg/result.h" #include "iceberg/schema.h" @@ -201,47 +202,38 @@ Status TableMetadataUtil::Write(FileIO& io, const std::string& location, struct TableMetadataBuilder::Impl { // Base metadata (if building from existing metadata) - std::shared_ptr base; + const TableMetadata* base{nullptr}; // Mutable fields that will be used to build the final TableMetadata int8_t format_version; std::string table_uuid; std::string location; - int64_t last_sequence_number; - TimePointMs last_updated_ms; - int32_t last_column_id; + int64_t last_sequence_number{TableMetadata::kInitialSequenceNumber}; + TimePointMs last_updated_ms{std::chrono::milliseconds(0)}; + int32_t last_column_id{0}; std::vector> schemas; std::optional current_schema_id; std::vector> partition_specs; - int32_t default_spec_id; - int32_t last_partition_id; + int32_t default_spec_id{0}; + int32_t last_partition_id{0}; std::unordered_map properties; - int64_t current_snapshot_id; + int64_t current_snapshot_id{-1}; std::vector> snapshots; std::vector snapshot_log; std::vector metadata_log; std::vector> sort_orders; - int32_t default_sort_order_id; + int32_t default_sort_order_id{0}; std::unordered_map> refs; std::vector> statistics; std::vector> partition_statistics; - int64_t next_row_id; + int64_t next_row_id{TableMetadata::kInitialRowId}; // List of changes (MetadataUpdate objects) std::vector> changes; - explicit Impl(int8_t fmt_version) - : format_version(fmt_version), - last_sequence_number(TableMetadata::kInitialSequenceNumber), - last_updated_ms(std::chrono::milliseconds(0)), - last_column_id(0), - default_spec_id(0), - last_partition_id(0), - current_snapshot_id(-1), - default_sort_order_id(0), - next_row_id(TableMetadata::kInitialRowId) {} - - explicit Impl(const std::shared_ptr& base_metadata) + explicit Impl(int8_t fmt_version) : format_version(fmt_version) {} + + explicit Impl(const TableMetadata* base_metadata) : base(base_metadata), format_version(base_metadata->format_version), table_uuid(base_metadata->table_uuid), @@ -270,7 +262,7 @@ struct TableMetadataBuilder::Impl { TableMetadataBuilder::TableMetadataBuilder(int8_t format_version) : impl_(std::make_unique(format_version)) {} -TableMetadataBuilder::TableMetadataBuilder(std::shared_ptr base) +TableMetadataBuilder::TableMetadataBuilder(const TableMetadata* base) : impl_(std::make_unique(base)) {} TableMetadataBuilder::~TableMetadataBuilder() = default; @@ -280,13 +272,15 @@ TableMetadataBuilder::TableMetadataBuilder(TableMetadataBuilder&&) noexcept = de TableMetadataBuilder& TableMetadataBuilder::operator=(TableMetadataBuilder&&) noexcept = default; -TableMetadataBuilder TableMetadataBuilder::BuildFromEmpty(int8_t format_version) { - return TableMetadataBuilder(format_version); +std::unique_ptr TableMetadataBuilder::BuildFromEmpty( + int8_t format_version) { + return std::unique_ptr( + new TableMetadataBuilder(format_version)); // NOLINT } -TableMetadataBuilder TableMetadataBuilder::BuildFrom( - const std::shared_ptr& base) { - return TableMetadataBuilder(base); +std::unique_ptr TableMetadataBuilder::BuildFrom( + const TableMetadata* base) { + return std::unique_ptr(new TableMetadataBuilder(base)); // NOLINT } TableMetadataBuilder& TableMetadataBuilder::AssignUUID() { @@ -294,7 +288,7 @@ TableMetadataBuilder& TableMetadataBuilder::AssignUUID() { return *this; } -TableMetadataBuilder& TableMetadataBuilder::AssignUUID(const std::string& uuid) { +TableMetadataBuilder& TableMetadataBuilder::AssignUUID(std::string_view uuid) { // TODO(gty404): Implement return *this; } diff --git a/src/iceberg/table_metadata.h b/src/iceberg/table_metadata.h index e80a351d9..ab2e5d911 100644 --- a/src/iceberg/table_metadata.h +++ b/src/iceberg/table_metadata.h @@ -29,7 +29,6 @@ #include #include "iceberg/iceberg_export.h" -#include "iceberg/metadata_update.h" #include "iceberg/type_fwd.h" #include "iceberg/util/timepoint.h" @@ -160,14 +159,14 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// /// \param format_version The format version for the table /// \return A new TableMetadataBuilder instance - static TableMetadataBuilder BuildFromEmpty( + static std::unique_ptr BuildFromEmpty( int8_t format_version = TableMetadata::kDefaultTableFormatVersion); /// \brief Create a builder from existing table metadata /// /// \param base The base table metadata to build from /// \return A new TableMetadataBuilder instance initialized with base metadata - static TableMetadataBuilder BuildFrom(const std::shared_ptr& base); + static std::unique_ptr BuildFrom(const TableMetadata* base); /// \brief Assign a UUID to the table /// @@ -179,7 +178,7 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// /// \param uuid The UUID string to assign /// \return Reference to this builder for method chaining - TableMetadataBuilder& AssignUUID(const std::string& uuid); + TableMetadataBuilder& AssignUUID(std::string_view uuid); /// \brief Upgrade the format version of the table /// @@ -192,7 +191,7 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// \param schema The schema to set as current /// \param new_last_column_id The highest column ID in the schema /// \return Reference to this builder for method chaining - TableMetadataBuilder& SetCurrentSchema(std::shared_ptr schema, + TableMetadataBuilder& SetCurrentSchema(std::shared_ptr schema, int32_t new_last_column_id); /// \brief Set the current schema by schema ID @@ -205,14 +204,13 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// /// \param schema The schema to add /// \return Reference to this builder for method chaining - TableMetadataBuilder& AddSchema(std::shared_ptr schema); + TableMetadataBuilder& AddSchema(std::shared_ptr schema); /// \brief Set the default partition spec for the table /// /// \param spec The partition spec to set as default /// \return Reference to this builder for method chaining - TableMetadataBuilder& SetDefaultPartitionSpec( - std::shared_ptr spec); + TableMetadataBuilder& SetDefaultPartitionSpec(std::shared_ptr spec); /// \brief Set the default partition spec by spec ID /// @@ -224,7 +222,7 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// /// \param spec The partition spec to add /// \return Reference to this builder for method chaining - TableMetadataBuilder& AddPartitionSpec(std::shared_ptr spec); + TableMetadataBuilder& AddPartitionSpec(std::shared_ptr spec); /// \brief Remove partition specs from the table /// @@ -242,7 +240,7 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// /// \param order The sort order to set as default /// \return Reference to this builder for method chaining - TableMetadataBuilder& SetDefaultSortOrder(std::shared_ptr order); + TableMetadataBuilder& SetDefaultSortOrder(std::shared_ptr order); /// \brief Set the default sort order by order ID /// @@ -254,13 +252,13 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// /// \param order The sort order to add /// \return Reference to this builder for method chaining - TableMetadataBuilder& AddSortOrder(std::shared_ptr order); + TableMetadataBuilder& AddSortOrder(std::shared_ptr order); /// \brief Add a snapshot to the table /// /// \param snapshot The snapshot to add /// \return Reference to this builder for method chaining - TableMetadataBuilder& AddSnapshot(std::shared_ptr snapshot); + TableMetadataBuilder& AddSnapshot(std::shared_ptr snapshot); /// \brief Set a branch to point to a specific snapshot /// @@ -335,7 +333,7 @@ class ICEBERG_EXPORT TableMetadataBuilder { explicit TableMetadataBuilder(int8_t format_version); /// \brief Private constructor for building from existing metadata - explicit TableMetadataBuilder(std::shared_ptr base); + explicit TableMetadataBuilder(const TableMetadata* base); /// Internal state members struct Impl; diff --git a/src/iceberg/update_requirement.cc b/src/iceberg/update_requirement.cc index 25451f312..53b20ad74 100644 --- a/src/iceberg/update_requirement.cc +++ b/src/iceberg/update_requirement.cc @@ -19,6 +19,7 @@ #include "iceberg/update_requirement.h" +#include "iceberg/snapshot.h" #include "iceberg/table_metadata.h" #include "iceberg/util/string_util.h" From abe3053611993593a058c475589d77f2941d4603 Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Wed, 15 Oct 2025 14:47:13 +0800 Subject: [PATCH 04/14] fix comments --- src/iceberg/util/string_util.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/iceberg/util/string_util.h b/src/iceberg/util/string_util.h index 66bb7b525..499a62ddd 100644 --- a/src/iceberg/util/string_util.h +++ b/src/iceberg/util/string_util.h @@ -46,9 +46,8 @@ class ICEBERG_EXPORT StringUtils { } static bool EqualsIgnoreCase(const std::string& a, const std::string& b) { - return a.size() == b.size() && std::ranges::equal(a, b, [](char ca, char cb) { - return std::tolower(ca) == std::tolower(cb); - }); + return std::ranges::equal( + a, b, [](char ca, char cb) { return std::tolower(ca) == std::tolower(cb); }); } }; From a1411d86e20203d5830b6ad8c74f0261e88e03c4 Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Wed, 15 Oct 2025 22:59:59 +0800 Subject: [PATCH 05/14] fix comments --- src/iceberg/CMakeLists.txt | 1 - src/iceberg/catalog.h | 4 +- .../catalog/memory/in_memory_catalog.cc | 4 +- .../catalog/memory/in_memory_catalog.h | 4 +- src/iceberg/metadata_update.cc | 285 ------------------ src/iceberg/table_metadata.cc | 220 +------------- src/iceberg/table_metadata.h | 73 ++++- ...date_requirement.h => table_requirement.h} | 56 +--- ...te_requirements.h => table_requirements.h} | 42 ++- .../{metadata_update.h => table_update.h} | 154 +++------- src/iceberg/type_fwd.h | 7 +- src/iceberg/update_requirement.cc | 149 --------- src/iceberg/update_requirements.cc | 77 ----- 13 files changed, 153 insertions(+), 923 deletions(-) delete mode 100644 src/iceberg/metadata_update.cc rename src/iceberg/{update_requirement.h => table_requirement.h} (74%) rename src/iceberg/{update_requirements.h => table_requirements.h} (71%) rename src/iceberg/{metadata_update.h => table_update.h} (63%) delete mode 100644 src/iceberg/update_requirement.cc delete mode 100644 src/iceberg/update_requirements.cc diff --git a/src/iceberg/CMakeLists.txt b/src/iceberg/CMakeLists.txt index fc00e8aad..c184cce9b 100644 --- a/src/iceberg/CMakeLists.txt +++ b/src/iceberg/CMakeLists.txt @@ -36,7 +36,6 @@ set(ICEBERG_SOURCES manifest_reader_internal.cc manifest_writer.cc metadata_columns.cc - metadata_update.cc name_mapping.cc partition_field.cc partition_spec.cc diff --git a/src/iceberg/catalog.h b/src/iceberg/catalog.h index 03bd0f6c4..83ea677ab 100644 --- a/src/iceberg/catalog.h +++ b/src/iceberg/catalog.h @@ -123,8 +123,8 @@ class ICEBERG_EXPORT Catalog { /// \return a Table instance or ErrorKind::kAlreadyExists if the table already exists virtual Result> UpdateTable( const TableIdentifier& identifier, - const std::vector>& requirements, - const std::vector>& updates) = 0; + const std::vector>& requirements, + const std::vector>& updates) = 0; /// \brief Start a transaction to create a table /// diff --git a/src/iceberg/catalog/memory/in_memory_catalog.cc b/src/iceberg/catalog/memory/in_memory_catalog.cc index 08a9822c2..c024aac20 100644 --- a/src/iceberg/catalog/memory/in_memory_catalog.cc +++ b/src/iceberg/catalog/memory/in_memory_catalog.cc @@ -392,8 +392,8 @@ Result> InMemoryCatalog::CreateTable( Result> InMemoryCatalog::UpdateTable( const TableIdentifier& identifier, - const std::vector>& requirements, - const std::vector>& updates) { + const std::vector>& requirements, + const std::vector>& updates) { return NotImplemented("update table"); } diff --git a/src/iceberg/catalog/memory/in_memory_catalog.h b/src/iceberg/catalog/memory/in_memory_catalog.h index bde97bae2..59c6d3ad8 100644 --- a/src/iceberg/catalog/memory/in_memory_catalog.h +++ b/src/iceberg/catalog/memory/in_memory_catalog.h @@ -77,8 +77,8 @@ class ICEBERG_EXPORT InMemoryCatalog Result> UpdateTable( const TableIdentifier& identifier, - const std::vector>& requirements, - const std::vector>& updates) override; + const std::vector>& requirements, + const std::vector>& updates) override; Result> StageCreateTable( const TableIdentifier& identifier, const Schema& schema, const PartitionSpec& spec, diff --git a/src/iceberg/metadata_update.cc b/src/iceberg/metadata_update.cc deleted file mode 100644 index cd171c4de..000000000 --- a/src/iceberg/metadata_update.cc +++ /dev/null @@ -1,285 +0,0 @@ -/* - * 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. - */ - -#include "iceberg/metadata_update.h" - -#include "iceberg/table_metadata.h" -#include "iceberg/update_requirements.h" - -namespace iceberg { - -// AssignUUID - -void AssignUUID::ApplyTo(TableMetadataBuilder& builder) const { - builder.AssignUUID(uuid_); -} - -void AssignUUID::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // AssignUUID doesn't generate any requirements -} - -// UpgradeFormatVersion - -void UpgradeFormatVersion::ApplyTo(TableMetadataBuilder& builder) const { - builder.UpgradeFormatVersion(format_version_); -} - -void UpgradeFormatVersion::GenerateRequirements( - UpdateRequirementsContext& /*context*/) const { - // UpgradeFormatVersion doesn't generate any requirements -} - -// AddSchema - -void AddSchema::ApplyTo(TableMetadataBuilder& builder) const { - builder.AddSchema(schema_); -} - -void AddSchema::GenerateRequirements(UpdateRequirementsContext& context) const { - if (context.base() != nullptr) { - context.AddRequirement( - std::make_unique(context.base()->last_column_id)); - } -} - -// SetCurrentSchema - -void SetCurrentSchema::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetCurrentSchema(schema_id_); -} - -void SetCurrentSchema::GenerateRequirements(UpdateRequirementsContext& context) const { - // Require current schema not changed - if (context.base() != nullptr && !context.is_replace()) { - if (context.base()->current_schema_id.has_value()) { - context.AddRequirement(std::make_unique( - context.base()->current_schema_id.value())); - } - } -} - -// AddPartitionSpec - -void AddPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { - builder.AddPartitionSpec(spec_); -} - -void AddPartitionSpec::GenerateRequirements(UpdateRequirementsContext& context) const { - if (context.base() != nullptr) { - context.AddRequirement(std::make_unique( - context.base()->last_partition_id)); - } -} - -// SetDefaultPartitionSpec - -void SetDefaultPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetDefaultPartitionSpec(spec_id_); -} - -void SetDefaultPartitionSpec::GenerateRequirements( - UpdateRequirementsContext& context) const { - // Require default partition spec not changed - if (context.base() != nullptr && !context.is_replace()) { - context.AddRequirement( - std::make_unique(context.base()->default_spec_id)); - } -} - -// RemovePartitionSpecs - -void RemovePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemovePartitionSpecs(spec_ids_); -} - -void RemovePartitionSpecs::GenerateRequirements( - UpdateRequirementsContext& context) const { - // Require default partition spec not changed - if (context.base() != nullptr && !context.is_replace()) { - context.AddRequirement( - std::make_unique(context.base()->default_spec_id)); - } - - // Require that no branches have changed - if (context.base() != nullptr && !context.is_replace()) { - for (const auto& [name, ref] : context.base()->refs) { - if (ref->type() == SnapshotRefType::kBranch && name != "main") { - context.AddRequirement( - std::make_unique(name, ref->snapshot_id)); - } - } - } -} - -// RemoveSchemas - -void RemoveSchemas::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemoveSchemas(schema_ids_); -} - -void RemoveSchemas::GenerateRequirements(UpdateRequirementsContext& context) const { - // Require current schema not changed - if (context.base() != nullptr && !context.is_replace()) { - if (context.base()->current_schema_id.has_value()) { - context.AddRequirement(std::make_unique( - context.base()->current_schema_id.value())); - } - } - - // Require that no branches have changed - if (context.base() != nullptr && !context.is_replace()) { - for (const auto& [name, ref] : context.base()->refs) { - if (ref->type() == SnapshotRefType::kBranch && name != "main") { - context.AddRequirement( - std::make_unique(name, ref->snapshot_id)); - } - } - } -} - -// AddSortOrder - -void AddSortOrder::ApplyTo(TableMetadataBuilder& builder) const { - builder.AddSortOrder(sort_order_); -} - -void AddSortOrder::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // AddSortOrder doesn't generate any requirements -} - -// SetDefaultSortOrder - -void SetDefaultSortOrder::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetDefaultSortOrder(sort_order_id_); -} - -void SetDefaultSortOrder::GenerateRequirements(UpdateRequirementsContext& context) const { - if (context.base() != nullptr && !context.is_replace()) { - context.AddRequirement(std::make_unique( - context.base()->default_sort_order_id)); - } -} - -// AddSnapshot - -void AddSnapshot::ApplyTo(TableMetadataBuilder& builder) const { - builder.AddSnapshot(snapshot_); -} - -void AddSnapshot::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // AddSnapshot doesn't generate any requirements -} - -// RemoveSnapshots - -void RemoveSnapshots::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemoveSnapshots(snapshot_ids_); -} - -void RemoveSnapshots::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // RemoveSnapshots doesn't generate any requirements -} - -// RemoveSnapshotRef - -void RemoveSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemoveRef(ref_name_); -} - -void RemoveSnapshotRef::GenerateRequirements( - UpdateRequirementsContext& /*context*/) const { - // RemoveSnapshotRef doesn't generate any requirements -} - -// SetSnapshotRef - -void SetSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { - // Create a SnapshotRef based on the type - std::shared_ptr ref; - - if (type_ == SnapshotRefType::kBranch) { - SnapshotRef::Branch branch; - branch.min_snapshots_to_keep = min_snapshots_to_keep_; - branch.max_snapshot_age_ms = max_snapshot_age_ms_; - branch.max_ref_age_ms = max_ref_age_ms_; - - ref = std::make_shared(); - ref->snapshot_id = snapshot_id_; - ref->retention = branch; - } else { - SnapshotRef::Tag tag; - tag.max_ref_age_ms = max_ref_age_ms_; - - ref = std::make_shared(); - ref->snapshot_id = snapshot_id_; - ref->retention = tag; - } - - builder.SetRef(ref_name_, ref); -} - -void SetSnapshotRef::GenerateRequirements(UpdateRequirementsContext& context) const { - // Require that the ref is unchanged from the base - if (context.base() != nullptr && !context.is_replace()) { - // Find the reference in the base metadata - auto it = context.base()->refs.find(ref_name_); - std::optional base_snapshot_id; - - if (it != context.base()->refs.end()) { - base_snapshot_id = it->second->snapshot_id; - } - - // Require that the ref does not exist (nullopt) or is the same as the base snapshot - context.AddRequirement( - std::make_unique(ref_name_, base_snapshot_id)); - } -} - -// SetProperties - -void SetProperties::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetProperties(updated_); -} - -void SetProperties::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // SetProperties doesn't generate any requirements -} - -// RemoveProperties - -void RemoveProperties::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemoveProperties(removed_); -} - -void RemoveProperties::GenerateRequirements( - UpdateRequirementsContext& /*context*/) const { - // RemoveProperties doesn't generate any requirements -} - -// SetLocation - -void SetLocation::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetLocation(location_); -} - -void SetLocation::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // SetLocation doesn't generate any requirements -} - -} // namespace iceberg diff --git a/src/iceberg/table_metadata.cc b/src/iceberg/table_metadata.cc index 742e5cc0b..82c9dfddc 100644 --- a/src/iceberg/table_metadata.cc +++ b/src/iceberg/table_metadata.cc @@ -27,12 +27,12 @@ #include "iceberg/file_io.h" #include "iceberg/json_internal.h" -#include "iceberg/metadata_update.h" #include "iceberg/partition_spec.h" #include "iceberg/result.h" #include "iceberg/schema.h" #include "iceberg/snapshot.h" #include "iceberg/sort_order.h" +#include "iceberg/table_update.h" #include "iceberg/util/gzip_internal.h" #include "iceberg/util/macros.h" @@ -198,222 +198,4 @@ Status TableMetadataUtil::Write(FileIO& io, const std::string& location, return io.WriteFile(location, json_string); } -// TableMetadataBuilder implementation - -struct TableMetadataBuilder::Impl { - // Base metadata (if building from existing metadata) - const TableMetadata* base{nullptr}; - - // Mutable fields that will be used to build the final TableMetadata - int8_t format_version; - std::string table_uuid; - std::string location; - int64_t last_sequence_number{TableMetadata::kInitialSequenceNumber}; - TimePointMs last_updated_ms{std::chrono::milliseconds(0)}; - int32_t last_column_id{0}; - std::vector> schemas; - std::optional current_schema_id; - std::vector> partition_specs; - int32_t default_spec_id{0}; - int32_t last_partition_id{0}; - std::unordered_map properties; - int64_t current_snapshot_id{-1}; - std::vector> snapshots; - std::vector snapshot_log; - std::vector metadata_log; - std::vector> sort_orders; - int32_t default_sort_order_id{0}; - std::unordered_map> refs; - std::vector> statistics; - std::vector> partition_statistics; - int64_t next_row_id{TableMetadata::kInitialRowId}; - - // List of changes (MetadataUpdate objects) - std::vector> changes; - - explicit Impl(int8_t fmt_version) : format_version(fmt_version) {} - - explicit Impl(const TableMetadata* base_metadata) - : base(base_metadata), - format_version(base_metadata->format_version), - table_uuid(base_metadata->table_uuid), - location(base_metadata->location), - last_sequence_number(base_metadata->last_sequence_number), - last_updated_ms(base_metadata->last_updated_ms), - last_column_id(base_metadata->last_column_id), - schemas(base_metadata->schemas), - current_schema_id(base_metadata->current_schema_id), - partition_specs(base_metadata->partition_specs), - default_spec_id(base_metadata->default_spec_id), - last_partition_id(base_metadata->last_partition_id), - properties(base_metadata->properties), - current_snapshot_id(base_metadata->current_snapshot_id), - snapshots(base_metadata->snapshots), - snapshot_log(base_metadata->snapshot_log), - metadata_log(base_metadata->metadata_log), - sort_orders(base_metadata->sort_orders), - default_sort_order_id(base_metadata->default_sort_order_id), - refs(base_metadata->refs), - statistics(base_metadata->statistics), - partition_statistics(base_metadata->partition_statistics), - next_row_id(base_metadata->next_row_id) {} -}; - -TableMetadataBuilder::TableMetadataBuilder(int8_t format_version) - : impl_(std::make_unique(format_version)) {} - -TableMetadataBuilder::TableMetadataBuilder(const TableMetadata* base) - : impl_(std::make_unique(base)) {} - -TableMetadataBuilder::~TableMetadataBuilder() = default; - -TableMetadataBuilder::TableMetadataBuilder(TableMetadataBuilder&&) noexcept = default; - -TableMetadataBuilder& TableMetadataBuilder::operator=(TableMetadataBuilder&&) noexcept = - default; - -std::unique_ptr TableMetadataBuilder::BuildFromEmpty( - int8_t format_version) { - return std::unique_ptr( - new TableMetadataBuilder(format_version)); // NOLINT -} - -std::unique_ptr TableMetadataBuilder::BuildFrom( - const TableMetadata* base) { - return std::unique_ptr(new TableMetadataBuilder(base)); // NOLINT -} - -TableMetadataBuilder& TableMetadataBuilder::AssignUUID() { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::AssignUUID(std::string_view uuid) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::UpgradeFormatVersion( - int8_t new_format_version) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetCurrentSchema( - std::shared_ptr schema, int32_t new_last_column_id) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetCurrentSchema(int32_t schema_id) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::AddSchema(std::shared_ptr schema) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetDefaultPartitionSpec( - std::shared_ptr spec) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetDefaultPartitionSpec(int32_t spec_id) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::AddPartitionSpec( - std::shared_ptr spec) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::RemovePartitionSpecs( - const std::vector& spec_ids) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::RemoveSchemas( - const std::vector& schema_ids) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetDefaultSortOrder( - std::shared_ptr order) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetDefaultSortOrder(int32_t order_id) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::AddSortOrder( - std::shared_ptr order) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::AddSnapshot( - std::shared_ptr snapshot) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetBranchSnapshot(int64_t snapshot_id, - const std::string& branch) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetRef(const std::string& name, - std::shared_ptr ref) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::RemoveRef(const std::string& name) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::RemoveSnapshots( - const std::vector& snapshot_ids) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetProperties( - const std::unordered_map& updated) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::RemoveProperties( - const std::vector& removed) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::SetLocation(const std::string& location) { - // TODO(gty404): Implement - return *this; -} - -TableMetadataBuilder& TableMetadataBuilder::DiscardChanges() { - // TODO(gty404): Implement - return *this; -} - -Result> TableMetadataBuilder::Build() { - return NotImplemented("TableMetadataBuilder::Build is not implemented"); -} - } // namespace iceberg diff --git a/src/iceberg/table_metadata.h b/src/iceberg/table_metadata.h index ab2e5d911..6f7a81904 100644 --- a/src/iceberg/table_metadata.h +++ b/src/iceberg/table_metadata.h @@ -153,6 +153,10 @@ ICEBERG_EXPORT std::string ToString(const MetadataLogEntry& entry); /// in a changes list. This allows the builder to maintain a complete history of all /// modifications made to the table metadata, which is important for tracking table /// evolution and for serialization purposes. +/// +/// If a modification violates Iceberg table constraints (e.g., setting a current +/// schema ID that does not exist), an error will be recorded and returned when +/// Build() is called. class ICEBERG_EXPORT TableMetadataBuilder { public: /// \brief Create a builder for a new table @@ -168,6 +172,19 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// \return A new TableMetadataBuilder instance initialized with base metadata static std::unique_ptr BuildFrom(const TableMetadata* base); + /// \brief Set the metadata location of the table + /// + /// \param metadata_location The new metadata location + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetMetadataLocation(std::string_view metadata_location); + + /// \brief Set the previous metadata location of the table + /// + /// \param previous_metadata_location The previous metadata location + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetPreviousMetadataLocation( + std::string_view previous_metadata_location); + /// \brief Assign a UUID to the table /// /// If no UUID is provided, a random UUID will be generated. @@ -280,12 +297,54 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// \return Reference to this builder for method chaining TableMetadataBuilder& RemoveRef(const std::string& name); + /// \brief Remove snapshots from the table + /// + /// \param snapshots_to_remove The snapshots to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemoveSnapshots( + const std::vector>& snapshots_to_remove); + /// \brief Remove snapshots from the table /// /// \param snapshot_ids The IDs of snapshots to remove /// \return Reference to this builder for method chaining TableMetadataBuilder& RemoveSnapshots(const std::vector& snapshot_ids); + /// \brief Suppresses snapshots that are historical, removing the metadata for lazy + /// snapshot loading. + /// + /// Note that the snapshots are not considered removed from metadata and no + /// RemoveSnapshot changes are created. A snapshot is historical if no ref directly + /// references its ID. + /// \return Reference to this builder for method chaining + TableMetadataBuilder& suppressHistoricalSnapshots(); + + /// \brief Set table statistics + /// + /// \param statistics_file The statistics file to set + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetStatistics( + const std::shared_ptr& statistics_file); + + /// \brief Remove table statistics by snapshot ID + /// + /// \param snapshot_id The snapshot ID whose statistics to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemoveStatistics(int64_t snapshot_id); + + /// \brief Set partition statistics + /// + /// \param partition_statistics_file The partition statistics file to set + /// \return Reference to this builder for method chaining + TableMetadataBuilder& SetPartitionStatistics( + const std::shared_ptr& partition_statistics_file); + + /// \brief Remove partition statistics by snapshot ID + /// + /// \param snapshot_id The snapshot ID whose partition statistics to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemovePartitionStatistics(int64_t snapshot_id); + /// \brief Set table properties /// /// \param updated Map of properties to set or update @@ -303,7 +362,19 @@ class ICEBERG_EXPORT TableMetadataBuilder { /// /// \param location The table base location /// \return Reference to this builder for method chaining - TableMetadataBuilder& SetLocation(const std::string& location); + TableMetadataBuilder& SetLocation(std::string_view location); + + /// \brief Add an encryption key to the table + /// + /// \param key The encryption key to add + /// \return Reference to this builder for method chaining + TableMetadataBuilder& AddEncryptionKey(std::shared_ptr key); + + /// \brief Remove an encryption key from the table by key ID + /// + /// \param key_id The ID of the encryption key to remove + /// \return Reference to this builder for method chaining + TableMetadataBuilder& RemoveEncryptionKey(std::string_view key_id); /// \brief Discard all accumulated changes /// diff --git a/src/iceberg/update_requirement.h b/src/iceberg/table_requirement.h similarity index 74% rename from src/iceberg/update_requirement.h rename to src/iceberg/table_requirement.h index 50b570c30..67fe9fa39 100644 --- a/src/iceberg/update_requirement.h +++ b/src/iceberg/table_requirement.h @@ -41,12 +41,9 @@ namespace iceberg { /// Represents a requirement that must be validated before applying /// metadata updates to a table. Each concrete subclass represents /// a specific type of requirement check. -class ICEBERG_EXPORT UpdateRequirement { +class ICEBERG_EXPORT TableRequirement { public: - virtual ~UpdateRequirement() = default; - - /// \brief Clone this update requirement - virtual std::unique_ptr Clone() const = 0; + virtual ~TableRequirement() = default; /// \brief Validate this requirement against table metadata /// @@ -59,14 +56,10 @@ class ICEBERG_EXPORT UpdateRequirement { /// /// This requirement is used when creating a new table to ensure /// it doesn't already exist. -class ICEBERG_EXPORT AssertTableDoesNotExist : public UpdateRequirement { +class ICEBERG_EXPORT AssertTableDoesNotExist : public TableRequirement { public: AssertTableDoesNotExist() = default; - std::unique_ptr Clone() const override { - return std::make_unique(); - } - Status Validate(const TableMetadata* base) const override; }; @@ -74,16 +67,12 @@ class ICEBERG_EXPORT AssertTableDoesNotExist : public UpdateRequirement { /// /// This ensures the table hasn't been replaced or recreated between /// reading the metadata and attempting to update it. -class ICEBERG_EXPORT AssertTableUUID : public UpdateRequirement { +class ICEBERG_EXPORT AssertTableUUID : public TableRequirement { public: explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} const std::string& uuid() const { return uuid_; } - std::unique_ptr Clone() const override { - return std::make_unique(uuid_); - } - Status Validate(const TableMetadata* base) const override; private: @@ -95,17 +84,14 @@ class ICEBERG_EXPORT AssertTableUUID : public UpdateRequirement { /// This requirement validates that a named reference (branch or tag) either: /// - Points to the expected snapshot ID /// - Does not exist (if snapshot_id is nullopt) -class ICEBERG_EXPORT AssertRefSnapshotID : public UpdateRequirement { +class ICEBERG_EXPORT AssertRefSnapshotID : public TableRequirement { public: AssertRefSnapshotID(std::string ref_name, std::optional snapshot_id) : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} const std::string& ref_name() const { return ref_name_; } - const std::optional& snapshot_id() const { return snapshot_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(ref_name_, snapshot_id_); - } + const std::optional& snapshot_id() const { return snapshot_id_; } Status Validate(const TableMetadata* base) const override; @@ -118,17 +104,13 @@ class ICEBERG_EXPORT AssertRefSnapshotID : public UpdateRequirement { /// /// This ensures the schema hasn't been modified (by adding fields) /// since the metadata was read. -class ICEBERG_EXPORT AssertLastAssignedFieldId : public UpdateRequirement { +class ICEBERG_EXPORT AssertLastAssignedFieldId : public TableRequirement { public: explicit AssertLastAssignedFieldId(int32_t last_assigned_field_id) : last_assigned_field_id_(last_assigned_field_id) {} int32_t last_assigned_field_id() const { return last_assigned_field_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(last_assigned_field_id_); - } - Status Validate(const TableMetadata* base) const override; private: @@ -139,16 +121,12 @@ class ICEBERG_EXPORT AssertLastAssignedFieldId : public UpdateRequirement { /// /// This ensures the active schema hasn't changed since the /// metadata was read. -class ICEBERG_EXPORT AssertCurrentSchemaID : public UpdateRequirement { +class ICEBERG_EXPORT AssertCurrentSchemaID : public TableRequirement { public: explicit AssertCurrentSchemaID(int32_t schema_id) : schema_id_(schema_id) {} int32_t schema_id() const { return schema_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(schema_id_); - } - Status Validate(const TableMetadata* base) const override; private: @@ -159,17 +137,13 @@ class ICEBERG_EXPORT AssertCurrentSchemaID : public UpdateRequirement { /// /// This ensures partition specs haven't been modified since the /// metadata was read. -class ICEBERG_EXPORT AssertLastAssignedPartitionId : public UpdateRequirement { +class ICEBERG_EXPORT AssertLastAssignedPartitionId : public TableRequirement { public: explicit AssertLastAssignedPartitionId(int32_t last_assigned_partition_id) : last_assigned_partition_id_(last_assigned_partition_id) {} int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(last_assigned_partition_id_); - } - Status Validate(const TableMetadata* base) const override; private: @@ -180,16 +154,12 @@ class ICEBERG_EXPORT AssertLastAssignedPartitionId : public UpdateRequirement { /// /// This ensures the default partition spec hasn't changed since /// the metadata was read. -class ICEBERG_EXPORT AssertDefaultSpecID : public UpdateRequirement { +class ICEBERG_EXPORT AssertDefaultSpecID : public TableRequirement { public: explicit AssertDefaultSpecID(int32_t spec_id) : spec_id_(spec_id) {} int32_t spec_id() const { return spec_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(spec_id_); - } - Status Validate(const TableMetadata* base) const override; private: @@ -200,17 +170,13 @@ class ICEBERG_EXPORT AssertDefaultSpecID : public UpdateRequirement { /// /// This ensures the default sort order hasn't changed since /// the metadata was read. -class ICEBERG_EXPORT AssertDefaultSortOrderID : public UpdateRequirement { +class ICEBERG_EXPORT AssertDefaultSortOrderID : public TableRequirement { public: explicit AssertDefaultSortOrderID(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} int32_t sort_order_id() const { return sort_order_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(sort_order_id_); - } - Status Validate(const TableMetadata* base) const override; private: diff --git a/src/iceberg/update_requirements.h b/src/iceberg/table_requirements.h similarity index 71% rename from src/iceberg/update_requirements.h rename to src/iceberg/table_requirements.h index d7ad97463..7fd5a3854 100644 --- a/src/iceberg/update_requirements.h +++ b/src/iceberg/table_requirements.h @@ -30,8 +30,8 @@ #include #include "iceberg/iceberg_export.h" +#include "iceberg/table_requirement.h" #include "iceberg/type_fwd.h" -#include "iceberg/update_requirement.h" namespace iceberg { @@ -40,26 +40,24 @@ namespace iceberg { /// This context is passed to each MetadataUpdate's GenerateRequirements method /// and maintains state about what requirements have already been added to avoid /// duplicates. -class ICEBERG_EXPORT UpdateRequirementsContext { +class ICEBERG_EXPORT MetadataUpdateContext { public: /// \brief Construct a context for requirement generation /// /// \param base The base table metadata (may be nullptr for table creation) /// \param is_replace Whether this is a replace operation (more permissive) - UpdateRequirementsContext(const TableMetadata* base, bool is_replace) + MetadataUpdateContext(const TableMetadata* base, bool is_replace) : base_(base), is_replace_(is_replace) {} // Delete copy operations (contains unique_ptr members) - UpdateRequirementsContext(const UpdateRequirementsContext&) = delete; - UpdateRequirementsContext& operator=(const UpdateRequirementsContext&) = delete; + MetadataUpdateContext(const MetadataUpdateContext&) = delete; + MetadataUpdateContext& operator=(const MetadataUpdateContext&) = delete; // Enable move construction only (assignment deleted due to const members) - UpdateRequirementsContext(UpdateRequirementsContext&&) noexcept = default; + MetadataUpdateContext(MetadataUpdateContext&&) noexcept = default; /// \brief Add a requirement to the list - void AddRequirement(std::unique_ptr requirement) { - requirements_.push_back(std::move(requirement)); - } + void AddRequirement(std::unique_ptr requirement); /// \brief Get the base table metadata const TableMetadata* base() const { return base_; } @@ -68,31 +66,29 @@ class ICEBERG_EXPORT UpdateRequirementsContext { bool is_replace() const { return is_replace_; } /// \brief Build and return the list of requirements - std::vector> Build() { - return std::move(requirements_); - } + Result>> Build(); private: const TableMetadata* base_; const bool is_replace_; - std::vector> requirements_; + std::vector> requirements_; }; /// \brief Factory class for generating update requirements /// /// This class analyzes a sequence of metadata updates and generates the /// appropriate update requirements to ensure safe concurrent modifications. -class ICEBERG_EXPORT UpdateRequirements { +class ICEBERG_EXPORT TableRequirements { public: /// \brief Generate requirements for creating a new table /// /// For table creation, this requires that the table does not already exist. /// - /// \param metadata_updates The list of metadata updates for table creation + /// \param table_updates The list of metadata updates for table creation /// \return A list of update requirements to validate before creation - static std::vector> ForCreateTable( - const std::vector>& metadata_updates); + static std::vector> ForCreateTable( + const std::vector>& table_updates); /// \brief Generate requirements for replacing an existing table /// @@ -100,11 +96,11 @@ class ICEBERG_EXPORT UpdateRequirements { /// allows more aggressive changes than a regular update. /// /// \param base The base table metadata - /// \param metadata_updates The list of metadata updates for replacement + /// \param table_updates The list of metadata updates for replacement /// \return A list of update requirements to validate before replacement - static std::vector> ForReplaceTable( + static std::vector> ForReplaceTable( const TableMetadata& base, - const std::vector>& metadata_updates); + const std::vector>& table_updates); /// \brief Generate requirements for updating an existing table /// @@ -112,11 +108,11 @@ class ICEBERG_EXPORT UpdateRequirements { /// metadata properties haven't changed concurrently. /// /// \param base The base table metadata - /// \param metadata_updates The list of metadata updates + /// \param table_updates The list of metadata updates /// \return A list of update requirements to validate before update - static std::vector> ForUpdateTable( + static std::vector> ForUpdateTable( const TableMetadata& base, - const std::vector>& metadata_updates); + const std::vector>& table_updates); }; } // namespace iceberg diff --git a/src/iceberg/metadata_update.h b/src/iceberg/table_update.h similarity index 63% rename from src/iceberg/metadata_update.h rename to src/iceberg/table_update.h index bf5dfc8e2..414a1be69 100644 --- a/src/iceberg/metadata_update.h +++ b/src/iceberg/table_update.h @@ -19,8 +19,8 @@ #pragma once -/// \file iceberg/metadata_update.h -/// Metadata update operations for Iceberg tables. +/// \file iceberg/table_update.h +/// Table metadata update operations for Iceberg tables. #include #include @@ -34,19 +34,13 @@ namespace iceberg { -class TableMetadataBuilder; -class UpdateRequirementsContext; - /// \brief Base class for metadata update operations /// /// Represents a change to table metadata. Each concrete subclass /// represents a specific type of update operation. -class ICEBERG_EXPORT MetadataUpdate { +class ICEBERG_EXPORT TableUpdate { public: - virtual ~MetadataUpdate() = default; - - /// \brief Clone this metadata update - virtual std::unique_ptr Clone() const = 0; + virtual ~TableUpdate() = default; /// \brief Apply this update to a TableMetadataBuilder /// @@ -64,50 +58,42 @@ class ICEBERG_EXPORT MetadataUpdate { /// provides information about the base metadata and operation mode. /// /// \param context The context containing base metadata and operation state - virtual void GenerateRequirements(UpdateRequirementsContext& context) const = 0; + virtual void GenerateRequirements(MetadataUpdateContext& context) const = 0; }; /// \brief Represents an assignment of a UUID to the table -class ICEBERG_EXPORT AssignUUID : public MetadataUpdate { +class ICEBERG_EXPORT AssignUUID : public TableUpdate { public: explicit AssignUUID(std::string uuid) : uuid_(std::move(uuid)) {} const std::string& uuid() const { return uuid_; } - std::unique_ptr Clone() const override { - return std::make_unique(uuid_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::string uuid_; }; /// \brief Represents an upgrade of the table format version -class ICEBERG_EXPORT UpgradeFormatVersion : public MetadataUpdate { +class ICEBERG_EXPORT UpgradeFormatVersion : public TableUpdate { public: explicit UpgradeFormatVersion(int8_t format_version) : format_version_(format_version) {} int8_t format_version() const { return format_version_; } - std::unique_ptr Clone() const override { - return std::make_unique(format_version_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: int8_t format_version_; }; /// \brief Represents adding a new schema to the table -class ICEBERG_EXPORT AddSchema : public MetadataUpdate { +class ICEBERG_EXPORT AddSchema : public TableUpdate { public: explicit AddSchema(std::shared_ptr schema, int32_t last_column_id) : schema_(std::move(schema)), last_column_id_(last_column_id) {} @@ -116,13 +102,9 @@ class ICEBERG_EXPORT AddSchema : public MetadataUpdate { int32_t last_column_id() const { return last_column_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(schema_, last_column_id_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::shared_ptr schema_; @@ -130,203 +112,163 @@ class ICEBERG_EXPORT AddSchema : public MetadataUpdate { }; /// \brief Represents setting the current schema -class ICEBERG_EXPORT SetCurrentSchema : public MetadataUpdate { +class ICEBERG_EXPORT SetCurrentSchema : public TableUpdate { public: explicit SetCurrentSchema(int32_t schema_id) : schema_id_(schema_id) {} int32_t schema_id() const { return schema_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(schema_id_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: int32_t schema_id_; }; /// \brief Represents adding a new partition spec to the table -class ICEBERG_EXPORT AddPartitionSpec : public MetadataUpdate { +class ICEBERG_EXPORT AddPartitionSpec : public TableUpdate { public: explicit AddPartitionSpec(std::shared_ptr spec) : spec_(std::move(spec)) {} const std::shared_ptr& spec() const { return spec_; } - std::unique_ptr Clone() const override { - return std::make_unique(spec_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::shared_ptr spec_; }; /// \brief Represents setting the default partition spec -class ICEBERG_EXPORT SetDefaultPartitionSpec : public MetadataUpdate { +class ICEBERG_EXPORT SetDefaultPartitionSpec : public TableUpdate { public: explicit SetDefaultPartitionSpec(int32_t spec_id) : spec_id_(spec_id) {} int32_t spec_id() const { return spec_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(spec_id_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: int32_t spec_id_; }; /// \brief Represents removing partition specs from the table -class ICEBERG_EXPORT RemovePartitionSpecs : public MetadataUpdate { +class ICEBERG_EXPORT RemovePartitionSpecs : public TableUpdate { public: explicit RemovePartitionSpecs(std::vector spec_ids) : spec_ids_(std::move(spec_ids)) {} const std::vector& spec_ids() const { return spec_ids_; } - std::unique_ptr Clone() const override { - return std::make_unique(spec_ids_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::vector spec_ids_; }; /// \brief Represents removing schemas from the table -class ICEBERG_EXPORT RemoveSchemas : public MetadataUpdate { +class ICEBERG_EXPORT RemoveSchemas : public TableUpdate { public: explicit RemoveSchemas(std::vector schema_ids) : schema_ids_(std::move(schema_ids)) {} const std::vector& schema_ids() const { return schema_ids_; } - std::unique_ptr Clone() const override { - return std::make_unique(schema_ids_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::vector schema_ids_; }; /// \brief Represents adding a new sort order to the table -class ICEBERG_EXPORT AddSortOrder : public MetadataUpdate { +class ICEBERG_EXPORT AddSortOrder : public TableUpdate { public: explicit AddSortOrder(std::shared_ptr sort_order) : sort_order_(std::move(sort_order)) {} const std::shared_ptr& sort_order() const { return sort_order_; } - std::unique_ptr Clone() const override { - return std::make_unique(sort_order_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::shared_ptr sort_order_; }; /// \brief Represents setting the default sort order -class ICEBERG_EXPORT SetDefaultSortOrder : public MetadataUpdate { +class ICEBERG_EXPORT SetDefaultSortOrder : public TableUpdate { public: explicit SetDefaultSortOrder(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} int32_t sort_order_id() const { return sort_order_id_; } - std::unique_ptr Clone() const override { - return std::make_unique(sort_order_id_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: int32_t sort_order_id_; }; /// \brief Represents adding a snapshot to the table -class ICEBERG_EXPORT AddSnapshot : public MetadataUpdate { +class ICEBERG_EXPORT AddSnapshot : public TableUpdate { public: explicit AddSnapshot(std::shared_ptr snapshot) : snapshot_(std::move(snapshot)) {} const std::shared_ptr& snapshot() const { return snapshot_; } - std::unique_ptr Clone() const override { - return std::make_unique(snapshot_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::shared_ptr snapshot_; }; /// \brief Represents removing snapshots from the table -class ICEBERG_EXPORT RemoveSnapshots : public MetadataUpdate { +class ICEBERG_EXPORT RemoveSnapshots : public TableUpdate { public: explicit RemoveSnapshots(std::vector snapshot_ids) : snapshot_ids_(std::move(snapshot_ids)) {} const std::vector& snapshot_ids() const { return snapshot_ids_; } - std::unique_ptr Clone() const override { - return std::make_unique(snapshot_ids_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::vector snapshot_ids_; }; /// \brief Represents removing a snapshot reference -class ICEBERG_EXPORT RemoveSnapshotRef : public MetadataUpdate { +class ICEBERG_EXPORT RemoveSnapshotRef : public TableUpdate { public: explicit RemoveSnapshotRef(std::string ref_name) : ref_name_(std::move(ref_name)) {} const std::string& ref_name() const { return ref_name_; } - std::unique_ptr Clone() const override { - return std::make_unique(ref_name_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::string ref_name_; }; /// \brief Represents setting a snapshot reference -class ICEBERG_EXPORT SetSnapshotRef : public MetadataUpdate { +class ICEBERG_EXPORT SetSnapshotRef : public TableUpdate { public: SetSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, std::optional min_snapshots_to_keep = std::nullopt, @@ -350,15 +292,9 @@ class ICEBERG_EXPORT SetSnapshotRef : public MetadataUpdate { } const std::optional& max_ref_age_ms() const { return max_ref_age_ms_; } - std::unique_ptr Clone() const override { - return std::make_unique(ref_name_, snapshot_id_, type_, - min_snapshots_to_keep_, max_snapshot_age_ms_, - max_ref_age_ms_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::string ref_name_; @@ -370,59 +306,47 @@ class ICEBERG_EXPORT SetSnapshotRef : public MetadataUpdate { }; /// \brief Represents setting table properties -class ICEBERG_EXPORT SetProperties : public MetadataUpdate { +class ICEBERG_EXPORT SetProperties : public TableUpdate { public: explicit SetProperties(std::unordered_map updated) : updated_(std::move(updated)) {} const std::unordered_map& updated() const { return updated_; } - std::unique_ptr Clone() const override { - return std::make_unique(updated_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::unordered_map updated_; }; /// \brief Represents removing table properties -class ICEBERG_EXPORT RemoveProperties : public MetadataUpdate { +class ICEBERG_EXPORT RemoveProperties : public TableUpdate { public: explicit RemoveProperties(std::vector removed) : removed_(std::move(removed)) {} const std::vector& removed() const { return removed_; } - std::unique_ptr Clone() const override { - return std::make_unique(removed_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::vector removed_; }; /// \brief Represents setting the table location -class ICEBERG_EXPORT SetLocation : public MetadataUpdate { +class ICEBERG_EXPORT SetLocation : public TableUpdate { public: explicit SetLocation(std::string location) : location_(std::move(location)) {} const std::string& location() const { return location_; } - std::unique_ptr Clone() const override { - return std::make_unique(location_); - } - void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(UpdateRequirementsContext& context) const override; + void GenerateRequirements(MetadataUpdateContext& context) const override; private: std::string location_; diff --git a/src/iceberg/type_fwd.h b/src/iceberg/type_fwd.h index 24eaef535..652ef8c50 100644 --- a/src/iceberg/type_fwd.h +++ b/src/iceberg/type_fwd.h @@ -142,13 +142,16 @@ class StructLike; class ArrayLike; class MapLike; -class MetadataUpdate; -class UpdateRequirement; +class TableUpdate; +class TableRequirement; +class TableMetadataBuilder; +class MetadataUpdateContext; /// ---------------------------------------------------------------------------- /// TODO: Forward declarations below are not added yet. /// ---------------------------------------------------------------------------- class AppendFiles; +class EncryptedKey; } // namespace iceberg diff --git a/src/iceberg/update_requirement.cc b/src/iceberg/update_requirement.cc deleted file mode 100644 index 53b20ad74..000000000 --- a/src/iceberg/update_requirement.cc +++ /dev/null @@ -1,149 +0,0 @@ -/* - * 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. - */ - -#include "iceberg/update_requirement.h" - -#include "iceberg/snapshot.h" -#include "iceberg/table_metadata.h" -#include "iceberg/util/string_util.h" - -namespace iceberg { - -Status AssertTableDoesNotExist::Validate(const TableMetadata* base) const { - if (base != nullptr) { - return CommitFailed("Requirement failed: table already exists"); - } - return {}; -} - -Status AssertTableUUID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - if (uuid_.empty()) { - return CommitFailed("Requirement failed: expected non-empty UUID"); - } - - if (!StringUtils::EqualsIgnoreCase(uuid_, base->table_uuid)) { - return CommitFailed("Requirement failed: UUID does not match: expected {} != {}", - base->table_uuid, uuid_); - } - - return {}; -} - -Status AssertRefSnapshotID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - // Find the reference in the table metadata - auto it = base->refs.find(ref_name_); - - if (it != base->refs.end()) { - // Reference exists - const auto& ref = it->second; - std::string type = (ref->type() == SnapshotRefType::kBranch) ? "branch" : "tag"; - - if (!snapshot_id_.has_value()) { - // A null snapshot ID means the ref should not exist already - return CommitFailed("Requirement failed: {} {} was created concurrently", type, - ref_name_); - } - if (snapshot_id_.value() != ref->snapshot_id) { - return CommitFailed("Requirement failed: {} {} has changed: expected id {} != {}", - type, ref_name_, snapshot_id_.value(), ref->snapshot_id); - } - } else { - // Reference does not exist - if (snapshot_id_.has_value()) { - return CommitFailed("Requirement failed: branch or tag {} is missing, expected {}", - ref_name_, snapshot_id_.value()); - } - } - - return {}; -} - -Status AssertLastAssignedFieldId::Validate(const TableMetadata* base) const { - if (base != nullptr && base->last_column_id != last_assigned_field_id_) { - return CommitFailed( - "Requirement failed: last assigned field id changed: expected id {} != {}", - last_assigned_field_id_, base->last_column_id); - } - return {}; -} - -Status AssertCurrentSchemaID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - if (!base->current_schema_id.has_value()) { - return CommitFailed("Requirement failed: table has no current schema"); - } - - if (schema_id_ != base->current_schema_id.value()) { - return CommitFailed( - "Requirement failed: current schema changed: expected id {} != {}", schema_id_, - base->current_schema_id.value()); - } - - return {}; -} - -Status AssertLastAssignedPartitionId::Validate(const TableMetadata* base) const { - if (base != nullptr && base->last_partition_id != last_assigned_partition_id_) { - return CommitFailed( - "Requirement failed: last assigned partition id changed: expected id {} != {}", - last_assigned_partition_id_, base->last_partition_id); - } - return {}; -} - -Status AssertDefaultSpecID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - if (spec_id_ != base->default_spec_id) { - return CommitFailed( - "Requirement failed: default partition spec changed: expected id {} != {}", - spec_id_, base->default_spec_id); - } - - return {}; -} - -Status AssertDefaultSortOrderID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - if (sort_order_id_ != base->default_sort_order_id) { - return CommitFailed( - "Requirement failed: default sort order changed: expected id {} != {}", - sort_order_id_, base->default_sort_order_id); - } - - return {}; -} - -} // namespace iceberg diff --git a/src/iceberg/update_requirements.cc b/src/iceberg/update_requirements.cc deleted file mode 100644 index d3c2df959..000000000 --- a/src/iceberg/update_requirements.cc +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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. - */ - -#include "iceberg/update_requirements.h" - -#include "iceberg/metadata_update.h" -#include "iceberg/table_metadata.h" - -namespace iceberg { - -std::vector> UpdateRequirements::ForCreateTable( - const std::vector>& metadata_updates) { - // Create context for table creation (no base metadata) - UpdateRequirementsContext context(nullptr, false); - - // Add requirement that table does not exist - context.AddRequirement(std::make_unique()); - - // Let each metadata update generate its requirements - for (const auto& update : metadata_updates) { - update->GenerateRequirements(context); - } - - return context.Build(); -} - -std::vector> UpdateRequirements::ForReplaceTable( - const TableMetadata& base, - const std::vector>& metadata_updates) { - // Create context for table replacement (is_replace = true) - UpdateRequirementsContext context(&base, true); - - // Add requirement that UUID matches - context.AddRequirement(std::make_unique(base.table_uuid)); - - // Let each metadata update generate its requirements - for (const auto& update : metadata_updates) { - update->GenerateRequirements(context); - } - - return context.Build(); -} - -std::vector> UpdateRequirements::ForUpdateTable( - const TableMetadata& base, - const std::vector>& metadata_updates) { - // Create context for table update (is_replace = false) - UpdateRequirementsContext context(&base, false); - - // Add requirement that UUID matches - context.AddRequirement(std::make_unique(base.table_uuid)); - - // Let each metadata update generate its requirements - for (const auto& update : metadata_updates) { - update->GenerateRequirements(context); - } - - return context.Build(); -} - -} // namespace iceberg From f040c014ec5aac8b52f5aab99de1b867f84881ae Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Wed, 15 Oct 2025 23:30:30 +0800 Subject: [PATCH 06/14] fix build error --- src/iceberg/CMakeLists.txt | 3 + src/iceberg/table_metadata.cc | 182 +++++++++++++++++++++++++++++- src/iceberg/table_requirement.cc | 58 ++++++++++ src/iceberg/table_requirement.h | 5 +- src/iceberg/table_requirements.cc | 53 +++++++++ src/iceberg/table_requirements.h | 6 +- src/iceberg/table_update.cc | 164 +++++++++++++++++++++++++++ src/iceberg/table_update.h | 37 +++--- src/iceberg/test/mock_catalog.h | 4 +- 9 files changed, 485 insertions(+), 27 deletions(-) create mode 100644 src/iceberg/table_requirement.cc create mode 100644 src/iceberg/table_requirements.cc create mode 100644 src/iceberg/table_update.cc diff --git a/src/iceberg/CMakeLists.txt b/src/iceberg/CMakeLists.txt index c184cce9b..eaaf6248c 100644 --- a/src/iceberg/CMakeLists.txt +++ b/src/iceberg/CMakeLists.txt @@ -52,7 +52,10 @@ set(ICEBERG_SOURCES table.cc table_metadata.cc table_properties.cc + table_requirement.cc + table_requirements.cc table_scan.cc + table_update.cc transform.cc transform_function.cc type.cc diff --git a/src/iceberg/table_metadata.cc b/src/iceberg/table_metadata.cc index 82c9dfddc..7f1d41eaa 100644 --- a/src/iceberg/table_metadata.cc +++ b/src/iceberg/table_metadata.cc @@ -32,7 +32,6 @@ #include "iceberg/schema.h" #include "iceberg/snapshot.h" #include "iceberg/sort_order.h" -#include "iceberg/table_update.h" #include "iceberg/util/gzip_internal.h" #include "iceberg/util/macros.h" @@ -198,4 +197,185 @@ Status TableMetadataUtil::Write(FileIO& io, const std::string& location, return io.WriteFile(location, json_string); } +// TableMetadataBuilder implementation + +struct TableMetadataBuilder::Impl {}; + +TableMetadataBuilder::TableMetadataBuilder(int8_t format_version) + : impl_(std::make_unique()) {} + +TableMetadataBuilder::TableMetadataBuilder(const TableMetadata* base) + : impl_(std::make_unique()) {} + +TableMetadataBuilder::~TableMetadataBuilder() = default; + +TableMetadataBuilder::TableMetadataBuilder(TableMetadataBuilder&&) noexcept = default; + +TableMetadataBuilder& TableMetadataBuilder::operator=(TableMetadataBuilder&&) noexcept = + default; + +std::unique_ptr TableMetadataBuilder::BuildFromEmpty( + int8_t format_version) { + return std::unique_ptr( + new TableMetadataBuilder(format_version)); // NOLINT +} + +std::unique_ptr TableMetadataBuilder::BuildFrom( + const TableMetadata* base) { + return std::unique_ptr(new TableMetadataBuilder(base)); // NOLINT +} + +TableMetadataBuilder& TableMetadataBuilder::SetMetadataLocation( + std::string_view metadata_location) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetPreviousMetadataLocation( + std::string_view previous_metadata_location) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AssignUUID() { return *this; } + +TableMetadataBuilder& TableMetadataBuilder::AssignUUID(std::string_view uuid) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::UpgradeFormatVersion( + int8_t new_format_version) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetCurrentSchema( + std::shared_ptr schema, int32_t new_last_column_id) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetCurrentSchema(int32_t schema_id) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddSchema(std::shared_ptr schema) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetDefaultPartitionSpec( + std::shared_ptr spec) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetDefaultPartitionSpec(int32_t spec_id) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddPartitionSpec( + std::shared_ptr spec) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemovePartitionSpecs( + const std::vector& spec_ids) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveSchemas( + const std::vector& schema_ids) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetDefaultSortOrder( + std::shared_ptr order) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetDefaultSortOrder(int32_t order_id) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddSortOrder( + std::shared_ptr order) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddSnapshot( + std::shared_ptr snapshot) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetBranchSnapshot(int64_t snapshot_id, + const std::string& branch) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetRef(const std::string& name, + std::shared_ptr ref) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveRef(const std::string& name) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveSnapshots( + const std::vector>& snapshots_to_remove) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveSnapshots( + const std::vector& snapshot_ids) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::suppressHistoricalSnapshots() { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetStatistics( + const std::shared_ptr& statistics_file) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveStatistics(int64_t snapshot_id) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetPartitionStatistics( + const std::shared_ptr& partition_statistics_file) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemovePartitionStatistics( + int64_t snapshot_id) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetProperties( + const std::unordered_map& updated) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveProperties( + const std::vector& removed) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::SetLocation(std::string_view location) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::AddEncryptionKey( + std::shared_ptr key) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::RemoveEncryptionKey(std::string_view key_id) { + return *this; +} + +TableMetadataBuilder& TableMetadataBuilder::DiscardChanges() { return *this; } + +Result> TableMetadataBuilder::Build() { + return NotImplemented("TableMetadataBuilder::Build not implemented"); +} + } // namespace iceberg diff --git a/src/iceberg/table_requirement.cc b/src/iceberg/table_requirement.cc new file mode 100644 index 000000000..26d08591e --- /dev/null +++ b/src/iceberg/table_requirement.cc @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "iceberg/table_requirement.h" + +#include "iceberg/table_metadata.h" + +namespace iceberg { + +Status AssertTableDoesNotExist::Validate(const TableMetadata* base) const { + return NotImplemented("AssertTableDoesNotExist::Validate not implemented"); +} + +Status AssertTableUUID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertTableUUID::Validate not implemented"); +} + +Status AssertRefSnapshotID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertRefSnapshotID::Validate not implemented"); +} + +Status AssertLastAssignedFieldId::Validate(const TableMetadata* base) const { + return NotImplemented("AssertLastAssignedFieldId::Validate not implemented"); +} + +Status AssertCurrentSchemaID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertCurrentSchemaID::Validate not implemented"); +} + +Status AssertLastAssignedPartitionId::Validate(const TableMetadata* base) const { + return NotImplemented("AssertLastAssignedPartitionId::Validate not implemented"); +} + +Status AssertDefaultSpecID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertDefaultSpecID::Validate not implemented"); +} + +Status AssertDefaultSortOrderID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertDefaultSortOrderID::Validate not implemented"); +} + +} // namespace iceberg diff --git a/src/iceberg/table_requirement.h b/src/iceberg/table_requirement.h index 67fe9fa39..6046b1178 100644 --- a/src/iceberg/table_requirement.h +++ b/src/iceberg/table_requirement.h @@ -19,14 +19,13 @@ #pragma once -/// \file iceberg/update_requirement.h +/// \file iceberg/table_requirement.h /// Update requirements for Iceberg table operations. /// -/// Update requirements are conditions that must be satisfied before +/// Table requirements are conditions that must be satisfied before /// applying metadata updates to a table. They are used for optimistic /// concurrency control in table operations. -#include #include #include diff --git a/src/iceberg/table_requirements.cc b/src/iceberg/table_requirements.cc new file mode 100644 index 000000000..ea8f606e5 --- /dev/null +++ b/src/iceberg/table_requirements.cc @@ -0,0 +1,53 @@ +/* + * 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. + */ + +#include "iceberg/table_requirements.h" + +#include "iceberg/table_metadata.h" +#include "iceberg/table_update.h" + +namespace iceberg { + +void MetadataUpdateContext::AddRequirement( + std::unique_ptr requirement) { + requirements_.push_back(std::move(requirement)); +} + +Result>> MetadataUpdateContext::Build() { + return std::move(requirements_); +} + +Result>> TableRequirements::ForCreateTable( + const std::vector>& table_updates) { + return NotImplemented("TableRequirements::ForCreateTable not implemented"); +} + +Result>> TableRequirements::ForReplaceTable( + const TableMetadata& base, + const std::vector>& table_updates) { + return NotImplemented("TableRequirements::ForReplaceTable not implemented"); +} + +Result>> TableRequirements::ForUpdateTable( + const TableMetadata& base, + const std::vector>& table_updates) { + return NotImplemented("TableRequirements::ForUpdateTable not implemented"); +} + +} // namespace iceberg diff --git a/src/iceberg/table_requirements.h b/src/iceberg/table_requirements.h index 7fd5a3854..e92426e32 100644 --- a/src/iceberg/table_requirements.h +++ b/src/iceberg/table_requirements.h @@ -87,7 +87,7 @@ class ICEBERG_EXPORT TableRequirements { /// /// \param table_updates The list of metadata updates for table creation /// \return A list of update requirements to validate before creation - static std::vector> ForCreateTable( + static Result>> ForCreateTable( const std::vector>& table_updates); /// \brief Generate requirements for replacing an existing table @@ -98,7 +98,7 @@ class ICEBERG_EXPORT TableRequirements { /// \param base The base table metadata /// \param table_updates The list of metadata updates for replacement /// \return A list of update requirements to validate before replacement - static std::vector> ForReplaceTable( + static Result>> ForReplaceTable( const TableMetadata& base, const std::vector>& table_updates); @@ -110,7 +110,7 @@ class ICEBERG_EXPORT TableRequirements { /// \param base The base table metadata /// \param table_updates The list of metadata updates /// \return A list of update requirements to validate before update - static std::vector> ForUpdateTable( + static Result>> ForUpdateTable( const TableMetadata& base, const std::vector>& table_updates); }; diff --git a/src/iceberg/table_update.cc b/src/iceberg/table_update.cc new file mode 100644 index 000000000..13f5b30a3 --- /dev/null +++ b/src/iceberg/table_update.cc @@ -0,0 +1,164 @@ +/* + * 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. + */ + +#include "iceberg/table_update.h" + +#include "iceberg/table_metadata.h" +#include "iceberg/table_requirements.h" + +namespace iceberg { + +// AssignUUID + +void AssignUUID::ApplyTo(TableMetadataBuilder& builder) const {} + +Status AssignUUID::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("AssignUUID::GenerateRequirements not implemented"); +} + +// UpgradeFormatVersion + +void UpgradeFormatVersion::ApplyTo(TableMetadataBuilder& builder) const {} + +Status UpgradeFormatVersion::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("UpgradeFormatVersion::GenerateRequirements not implemented"); +} + +// AddSchema + +void AddSchema::ApplyTo(TableMetadataBuilder& builder) const {} + +Status AddSchema::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("AddSchema::GenerateRequirements not implemented"); +} + +// SetCurrentSchema + +void SetCurrentSchema::ApplyTo(TableMetadataBuilder& builder) const {} + +Status SetCurrentSchema::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("SetCurrentSchema::GenerateRequirements not implemented"); +} + +// AddPartitionSpec + +void AddPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const {} + +Status AddPartitionSpec::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("AddPartitionSpec::GenerateRequirements not implemented"); +} + +// SetDefaultPartitionSpec + +void SetDefaultPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const {} + +Status SetDefaultPartitionSpec::GenerateRequirements( + MetadataUpdateContext& context) const { + return NotImplemented("SetDefaultPartitionSpec::GenerateRequirements not implemented"); +} + +// RemovePartitionSpecs + +void RemovePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const {} + +Status RemovePartitionSpecs::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("RemovePartitionSpecs::GenerateRequirements not implemented"); +} + +// RemoveSchemas + +void RemoveSchemas::ApplyTo(TableMetadataBuilder& builder) const {} + +Status RemoveSchemas::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("RemoveSchemas::GenerateRequirements not implemented"); +} + +// AddSortOrder + +void AddSortOrder::ApplyTo(TableMetadataBuilder& builder) const {} + +Status AddSortOrder::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("AddSortOrder::GenerateRequirements not implemented"); +} + +// SetDefaultSortOrder + +void SetDefaultSortOrder::ApplyTo(TableMetadataBuilder& builder) const {} + +Status SetDefaultSortOrder::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("SetDefaultSortOrder::GenerateRequirements not implemented"); +} + +// AddSnapshot + +void AddSnapshot::ApplyTo(TableMetadataBuilder& builder) const {} + +Status AddSnapshot::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("AddSnapshot::GenerateRequirements not implemented"); +} + +// RemoveSnapshots + +void RemoveSnapshots::ApplyTo(TableMetadataBuilder& builder) const {} + +Status RemoveSnapshots::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("RemoveSnapshots::GenerateRequirements not implemented"); +} + +// RemoveSnapshotRef + +void RemoveSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const {} + +Status RemoveSnapshotRef::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("RemoveSnapshotRef::GenerateRequirements not implemented"); +} + +// SetSnapshotRef + +void SetSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const {} + +Status SetSnapshotRef::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("SetSnapshotRef::GenerateRequirements not implemented"); +} + +// SetProperties + +void SetProperties::ApplyTo(TableMetadataBuilder& builder) const {} + +Status SetProperties::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("SetProperties::GenerateRequirements not implemented"); +} + +// RemoveProperties + +void RemoveProperties::ApplyTo(TableMetadataBuilder& builder) const {} + +Status RemoveProperties::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("RemoveProperties::GenerateRequirements not implemented"); +} + +// SetLocation + +void SetLocation::ApplyTo(TableMetadataBuilder& builder) const {} + +Status SetLocation::GenerateRequirements(MetadataUpdateContext& context) const { + return NotImplemented("SetLocation::GenerateRequirements not implemented"); +} + +} // namespace iceberg diff --git a/src/iceberg/table_update.h b/src/iceberg/table_update.h index 414a1be69..3bfcd3cf5 100644 --- a/src/iceberg/table_update.h +++ b/src/iceberg/table_update.h @@ -58,7 +58,8 @@ class ICEBERG_EXPORT TableUpdate { /// provides information about the base metadata and operation mode. /// /// \param context The context containing base metadata and operation state - virtual void GenerateRequirements(MetadataUpdateContext& context) const = 0; + /// \return Status indicating success or failure with error details + virtual Status GenerateRequirements(MetadataUpdateContext& context) const = 0; }; /// \brief Represents an assignment of a UUID to the table @@ -70,7 +71,7 @@ class ICEBERG_EXPORT AssignUUID : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::string uuid_; @@ -86,7 +87,7 @@ class ICEBERG_EXPORT UpgradeFormatVersion : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: int8_t format_version_; @@ -104,7 +105,7 @@ class ICEBERG_EXPORT AddSchema : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::shared_ptr schema_; @@ -120,7 +121,7 @@ class ICEBERG_EXPORT SetCurrentSchema : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: int32_t schema_id_; @@ -136,7 +137,7 @@ class ICEBERG_EXPORT AddPartitionSpec : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::shared_ptr spec_; @@ -151,7 +152,7 @@ class ICEBERG_EXPORT SetDefaultPartitionSpec : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: int32_t spec_id_; @@ -167,7 +168,7 @@ class ICEBERG_EXPORT RemovePartitionSpecs : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::vector spec_ids_; @@ -183,7 +184,7 @@ class ICEBERG_EXPORT RemoveSchemas : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::vector schema_ids_; @@ -199,7 +200,7 @@ class ICEBERG_EXPORT AddSortOrder : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::shared_ptr sort_order_; @@ -214,7 +215,7 @@ class ICEBERG_EXPORT SetDefaultSortOrder : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: int32_t sort_order_id_; @@ -230,7 +231,7 @@ class ICEBERG_EXPORT AddSnapshot : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::shared_ptr snapshot_; @@ -246,7 +247,7 @@ class ICEBERG_EXPORT RemoveSnapshots : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::vector snapshot_ids_; @@ -261,7 +262,7 @@ class ICEBERG_EXPORT RemoveSnapshotRef : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::string ref_name_; @@ -294,7 +295,7 @@ class ICEBERG_EXPORT SetSnapshotRef : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::string ref_name_; @@ -315,7 +316,7 @@ class ICEBERG_EXPORT SetProperties : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::unordered_map updated_; @@ -331,7 +332,7 @@ class ICEBERG_EXPORT RemoveProperties : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::vector removed_; @@ -346,7 +347,7 @@ class ICEBERG_EXPORT SetLocation : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - void GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(MetadataUpdateContext& context) const override; private: std::string location_; diff --git a/src/iceberg/test/mock_catalog.h b/src/iceberg/test/mock_catalog.h index 5363f1c7b..f54982bb2 100644 --- a/src/iceberg/test/mock_catalog.h +++ b/src/iceberg/test/mock_catalog.h @@ -62,8 +62,8 @@ class MockCatalog : public Catalog { MOCK_METHOD((Result>), UpdateTable, (const TableIdentifier&, - (const std::vector>&), - (const std::vector>&)), + (const std::vector>&), + (const std::vector>&)), (override)); MOCK_METHOD((Result>), StageCreateTable, From d8a736eec89f8bb383d5ec8c86e1406f63b13e99 Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Sat, 11 Oct 2025 21:59:07 +0800 Subject: [PATCH 07/14] feat: implement metadata updates and update requirements This commit introduces the metadata update and update requirements system for table metadata modifications. --- src/iceberg/CMakeLists.txt | 1 + src/iceberg/metadata_update.cc | 285 +++++++++++++++++++ src/iceberg/metadata_update.h | 431 +++++++++++++++++++++++++++++ src/iceberg/table_metadata.h | 1 + src/iceberg/update_requirement.cc | 148 ++++++++++ src/iceberg/update_requirement.h | 220 +++++++++++++++ src/iceberg/update_requirements.cc | 77 ++++++ src/iceberg/update_requirements.h | 122 ++++++++ 8 files changed, 1285 insertions(+) create mode 100644 src/iceberg/metadata_update.cc create mode 100644 src/iceberg/metadata_update.h create mode 100644 src/iceberg/update_requirement.cc create mode 100644 src/iceberg/update_requirement.h create mode 100644 src/iceberg/update_requirements.cc create mode 100644 src/iceberg/update_requirements.h diff --git a/src/iceberg/CMakeLists.txt b/src/iceberg/CMakeLists.txt index eaaf6248c..500d23686 100644 --- a/src/iceberg/CMakeLists.txt +++ b/src/iceberg/CMakeLists.txt @@ -36,6 +36,7 @@ set(ICEBERG_SOURCES manifest_reader_internal.cc manifest_writer.cc metadata_columns.cc + metadata_update.cc name_mapping.cc partition_field.cc partition_spec.cc diff --git a/src/iceberg/metadata_update.cc b/src/iceberg/metadata_update.cc new file mode 100644 index 000000000..cd171c4de --- /dev/null +++ b/src/iceberg/metadata_update.cc @@ -0,0 +1,285 @@ +/* + * 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. + */ + +#include "iceberg/metadata_update.h" + +#include "iceberg/table_metadata.h" +#include "iceberg/update_requirements.h" + +namespace iceberg { + +// AssignUUID + +void AssignUUID::ApplyTo(TableMetadataBuilder& builder) const { + builder.AssignUUID(uuid_); +} + +void AssignUUID::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // AssignUUID doesn't generate any requirements +} + +// UpgradeFormatVersion + +void UpgradeFormatVersion::ApplyTo(TableMetadataBuilder& builder) const { + builder.UpgradeFormatVersion(format_version_); +} + +void UpgradeFormatVersion::GenerateRequirements( + UpdateRequirementsContext& /*context*/) const { + // UpgradeFormatVersion doesn't generate any requirements +} + +// AddSchema + +void AddSchema::ApplyTo(TableMetadataBuilder& builder) const { + builder.AddSchema(schema_); +} + +void AddSchema::GenerateRequirements(UpdateRequirementsContext& context) const { + if (context.base() != nullptr) { + context.AddRequirement( + std::make_unique(context.base()->last_column_id)); + } +} + +// SetCurrentSchema + +void SetCurrentSchema::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetCurrentSchema(schema_id_); +} + +void SetCurrentSchema::GenerateRequirements(UpdateRequirementsContext& context) const { + // Require current schema not changed + if (context.base() != nullptr && !context.is_replace()) { + if (context.base()->current_schema_id.has_value()) { + context.AddRequirement(std::make_unique( + context.base()->current_schema_id.value())); + } + } +} + +// AddPartitionSpec + +void AddPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { + builder.AddPartitionSpec(spec_); +} + +void AddPartitionSpec::GenerateRequirements(UpdateRequirementsContext& context) const { + if (context.base() != nullptr) { + context.AddRequirement(std::make_unique( + context.base()->last_partition_id)); + } +} + +// SetDefaultPartitionSpec + +void SetDefaultPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetDefaultPartitionSpec(spec_id_); +} + +void SetDefaultPartitionSpec::GenerateRequirements( + UpdateRequirementsContext& context) const { + // Require default partition spec not changed + if (context.base() != nullptr && !context.is_replace()) { + context.AddRequirement( + std::make_unique(context.base()->default_spec_id)); + } +} + +// RemovePartitionSpecs + +void RemovePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemovePartitionSpecs(spec_ids_); +} + +void RemovePartitionSpecs::GenerateRequirements( + UpdateRequirementsContext& context) const { + // Require default partition spec not changed + if (context.base() != nullptr && !context.is_replace()) { + context.AddRequirement( + std::make_unique(context.base()->default_spec_id)); + } + + // Require that no branches have changed + if (context.base() != nullptr && !context.is_replace()) { + for (const auto& [name, ref] : context.base()->refs) { + if (ref->type() == SnapshotRefType::kBranch && name != "main") { + context.AddRequirement( + std::make_unique(name, ref->snapshot_id)); + } + } + } +} + +// RemoveSchemas + +void RemoveSchemas::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemoveSchemas(schema_ids_); +} + +void RemoveSchemas::GenerateRequirements(UpdateRequirementsContext& context) const { + // Require current schema not changed + if (context.base() != nullptr && !context.is_replace()) { + if (context.base()->current_schema_id.has_value()) { + context.AddRequirement(std::make_unique( + context.base()->current_schema_id.value())); + } + } + + // Require that no branches have changed + if (context.base() != nullptr && !context.is_replace()) { + for (const auto& [name, ref] : context.base()->refs) { + if (ref->type() == SnapshotRefType::kBranch && name != "main") { + context.AddRequirement( + std::make_unique(name, ref->snapshot_id)); + } + } + } +} + +// AddSortOrder + +void AddSortOrder::ApplyTo(TableMetadataBuilder& builder) const { + builder.AddSortOrder(sort_order_); +} + +void AddSortOrder::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // AddSortOrder doesn't generate any requirements +} + +// SetDefaultSortOrder + +void SetDefaultSortOrder::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetDefaultSortOrder(sort_order_id_); +} + +void SetDefaultSortOrder::GenerateRequirements(UpdateRequirementsContext& context) const { + if (context.base() != nullptr && !context.is_replace()) { + context.AddRequirement(std::make_unique( + context.base()->default_sort_order_id)); + } +} + +// AddSnapshot + +void AddSnapshot::ApplyTo(TableMetadataBuilder& builder) const { + builder.AddSnapshot(snapshot_); +} + +void AddSnapshot::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // AddSnapshot doesn't generate any requirements +} + +// RemoveSnapshots + +void RemoveSnapshots::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemoveSnapshots(snapshot_ids_); +} + +void RemoveSnapshots::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // RemoveSnapshots doesn't generate any requirements +} + +// RemoveSnapshotRef + +void RemoveSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemoveRef(ref_name_); +} + +void RemoveSnapshotRef::GenerateRequirements( + UpdateRequirementsContext& /*context*/) const { + // RemoveSnapshotRef doesn't generate any requirements +} + +// SetSnapshotRef + +void SetSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { + // Create a SnapshotRef based on the type + std::shared_ptr ref; + + if (type_ == SnapshotRefType::kBranch) { + SnapshotRef::Branch branch; + branch.min_snapshots_to_keep = min_snapshots_to_keep_; + branch.max_snapshot_age_ms = max_snapshot_age_ms_; + branch.max_ref_age_ms = max_ref_age_ms_; + + ref = std::make_shared(); + ref->snapshot_id = snapshot_id_; + ref->retention = branch; + } else { + SnapshotRef::Tag tag; + tag.max_ref_age_ms = max_ref_age_ms_; + + ref = std::make_shared(); + ref->snapshot_id = snapshot_id_; + ref->retention = tag; + } + + builder.SetRef(ref_name_, ref); +} + +void SetSnapshotRef::GenerateRequirements(UpdateRequirementsContext& context) const { + // Require that the ref is unchanged from the base + if (context.base() != nullptr && !context.is_replace()) { + // Find the reference in the base metadata + auto it = context.base()->refs.find(ref_name_); + std::optional base_snapshot_id; + + if (it != context.base()->refs.end()) { + base_snapshot_id = it->second->snapshot_id; + } + + // Require that the ref does not exist (nullopt) or is the same as the base snapshot + context.AddRequirement( + std::make_unique(ref_name_, base_snapshot_id)); + } +} + +// SetProperties + +void SetProperties::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetProperties(updated_); +} + +void SetProperties::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // SetProperties doesn't generate any requirements +} + +// RemoveProperties + +void RemoveProperties::ApplyTo(TableMetadataBuilder& builder) const { + builder.RemoveProperties(removed_); +} + +void RemoveProperties::GenerateRequirements( + UpdateRequirementsContext& /*context*/) const { + // RemoveProperties doesn't generate any requirements +} + +// SetLocation + +void SetLocation::ApplyTo(TableMetadataBuilder& builder) const { + builder.SetLocation(location_); +} + +void SetLocation::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { + // SetLocation doesn't generate any requirements +} + +} // namespace iceberg diff --git a/src/iceberg/metadata_update.h b/src/iceberg/metadata_update.h new file mode 100644 index 000000000..8aeaea3a6 --- /dev/null +++ b/src/iceberg/metadata_update.h @@ -0,0 +1,431 @@ +/* + * 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 + +/// \file iceberg/metadata_update.h +/// Metadata update operations for Iceberg tables. + +#include +#include +#include +#include +#include + +#include "iceberg/iceberg_export.h" +#include "iceberg/snapshot.h" +#include "iceberg/type_fwd.h" + +namespace iceberg { + +class TableMetadataBuilder; +class UpdateRequirementsContext; + +/// \brief Base class for metadata update operations +/// +/// Represents a change to table metadata. Each concrete subclass +/// represents a specific type of update operation. +class ICEBERG_EXPORT MetadataUpdate { + public: + virtual ~MetadataUpdate() = default; + + /// \brief Clone this metadata update + virtual std::unique_ptr Clone() const = 0; + + /// \brief Apply this update to a TableMetadataBuilder + /// + /// This method modifies the builder by applying the update operation + /// it represents. Each subclass implements this to apply its specific + /// type of update. + /// + /// \param builder The builder to apply this update to + virtual void ApplyTo(TableMetadataBuilder& builder) const = 0; + + /// \brief Generate update requirements for this metadata update + /// + /// This method generates the appropriate UpdateRequirement instances + /// that must be validated before this update can be applied. The context + /// provides information about the base metadata and operation mode. + /// + /// \param context The context containing base metadata and operation state + virtual void GenerateRequirements(UpdateRequirementsContext& context) const = 0; +}; + +/// \brief Represents an assignment of a UUID to the table +class ICEBERG_EXPORT AssignUUID : public MetadataUpdate { + public: + explicit AssignUUID(std::string uuid) : uuid_(std::move(uuid)) {} + + const std::string& uuid() const { return uuid_; } + + std::unique_ptr Clone() const override { + return std::make_unique(uuid_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::string uuid_; +}; + +/// \brief Represents an upgrade of the table format version +class ICEBERG_EXPORT UpgradeFormatVersion : public MetadataUpdate { + public: + explicit UpgradeFormatVersion(int8_t format_version) + : format_version_(format_version) {} + + int8_t format_version() const { return format_version_; } + + std::unique_ptr Clone() const override { + return std::make_unique(format_version_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + int8_t format_version_; +}; + +/// \brief Represents adding a new schema to the table +class ICEBERG_EXPORT AddSchema : public MetadataUpdate { + public: + explicit AddSchema(std::shared_ptr schema, int32_t last_column_id) + : schema_(std::move(schema)), last_column_id_(last_column_id) {} + + const std::shared_ptr& schema() const { return schema_; } + + int32_t last_column_id() const { return last_column_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(schema_, last_column_id_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::shared_ptr schema_; + int32_t last_column_id_; +}; + +/// \brief Represents setting the current schema +class ICEBERG_EXPORT SetCurrentSchema : public MetadataUpdate { + public: + explicit SetCurrentSchema(int32_t schema_id) : schema_id_(schema_id) {} + + int32_t schema_id() const { return schema_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(schema_id_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + int32_t schema_id_; +}; + +/// \brief Represents adding a new partition spec to the table +class ICEBERG_EXPORT AddPartitionSpec : public MetadataUpdate { + public: + explicit AddPartitionSpec(std::shared_ptr spec) + : spec_(std::move(spec)) {} + + const std::shared_ptr& spec() const { return spec_; } + + std::unique_ptr Clone() const override { + return std::make_unique(spec_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::shared_ptr spec_; +}; + +/// \brief Represents setting the default partition spec +class ICEBERG_EXPORT SetDefaultPartitionSpec : public MetadataUpdate { + public: + explicit SetDefaultPartitionSpec(int32_t spec_id) : spec_id_(spec_id) {} + + int32_t spec_id() const { return spec_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(spec_id_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + int32_t spec_id_; +}; + +/// \brief Represents removing partition specs from the table +class ICEBERG_EXPORT RemovePartitionSpecs : public MetadataUpdate { + public: + explicit RemovePartitionSpecs(std::vector spec_ids) + : spec_ids_(std::move(spec_ids)) {} + + const std::vector& spec_ids() const { return spec_ids_; } + + std::unique_ptr Clone() const override { + return std::make_unique(spec_ids_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::vector spec_ids_; +}; + +/// \brief Represents removing schemas from the table +class ICEBERG_EXPORT RemoveSchemas : public MetadataUpdate { + public: + explicit RemoveSchemas(std::vector schema_ids) + : schema_ids_(std::move(schema_ids)) {} + + const std::vector& schema_ids() const { return schema_ids_; } + + std::unique_ptr Clone() const override { + return std::make_unique(schema_ids_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::vector schema_ids_; +}; + +/// \brief Represents adding a new sort order to the table +class ICEBERG_EXPORT AddSortOrder : public MetadataUpdate { + public: + explicit AddSortOrder(std::shared_ptr sort_order) + : sort_order_(std::move(sort_order)) {} + + const std::shared_ptr& sort_order() const { return sort_order_; } + + std::unique_ptr Clone() const override { + return std::make_unique(sort_order_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::shared_ptr sort_order_; +}; + +/// \brief Represents setting the default sort order +class ICEBERG_EXPORT SetDefaultSortOrder : public MetadataUpdate { + public: + explicit SetDefaultSortOrder(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} + + int32_t sort_order_id() const { return sort_order_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(sort_order_id_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + int32_t sort_order_id_; +}; + +/// \brief Represents adding a snapshot to the table +class ICEBERG_EXPORT AddSnapshot : public MetadataUpdate { + public: + explicit AddSnapshot(std::shared_ptr snapshot) + : snapshot_(std::move(snapshot)) {} + + const std::shared_ptr& snapshot() const { return snapshot_; } + + std::unique_ptr Clone() const override { + return std::make_unique(snapshot_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::shared_ptr snapshot_; +}; + +/// \brief Represents removing snapshots from the table +class ICEBERG_EXPORT RemoveSnapshots : public MetadataUpdate { + public: + explicit RemoveSnapshots(std::vector snapshot_ids) + : snapshot_ids_(std::move(snapshot_ids)) {} + + const std::vector& snapshot_ids() const { return snapshot_ids_; } + + std::unique_ptr Clone() const override { + return std::make_unique(snapshot_ids_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::vector snapshot_ids_; +}; + +/// \brief Represents removing a snapshot reference +class ICEBERG_EXPORT RemoveSnapshotRef : public MetadataUpdate { + public: + explicit RemoveSnapshotRef(std::string ref_name) : ref_name_(std::move(ref_name)) {} + + const std::string& ref_name() const { return ref_name_; } + + std::unique_ptr Clone() const override { + return std::make_unique(ref_name_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::string ref_name_; +}; + +/// \brief Represents setting a snapshot reference +class ICEBERG_EXPORT SetSnapshotRef : public MetadataUpdate { + public: + SetSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, + std::optional min_snapshots_to_keep = std::nullopt, + std::optional max_snapshot_age_ms = std::nullopt, + std::optional max_ref_age_ms = std::nullopt) + : ref_name_(std::move(ref_name)), + snapshot_id_(snapshot_id), + type_(type), + min_snapshots_to_keep_(min_snapshots_to_keep), + max_snapshot_age_ms_(max_snapshot_age_ms), + max_ref_age_ms_(max_ref_age_ms) {} + + const std::string& ref_name() const { return ref_name_; } + int64_t snapshot_id() const { return snapshot_id_; } + SnapshotRefType type() const { return type_; } + const std::optional& min_snapshots_to_keep() const { + return min_snapshots_to_keep_; + } + const std::optional& max_snapshot_age_ms() const { + return max_snapshot_age_ms_; + } + const std::optional& max_ref_age_ms() const { return max_ref_age_ms_; } + + std::unique_ptr Clone() const override { + return std::make_unique(ref_name_, snapshot_id_, type_, + min_snapshots_to_keep_, max_snapshot_age_ms_, + max_ref_age_ms_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::string ref_name_; + int64_t snapshot_id_; + SnapshotRefType type_; + std::optional min_snapshots_to_keep_; + std::optional max_snapshot_age_ms_; + std::optional max_ref_age_ms_; +}; + +/// \brief Represents setting table properties +class ICEBERG_EXPORT SetProperties : public MetadataUpdate { + public: + explicit SetProperties(std::unordered_map updated) + : updated_(std::move(updated)) {} + + const std::unordered_map& updated() const { return updated_; } + + std::unique_ptr Clone() const override { + return std::make_unique(updated_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::unordered_map updated_; +}; + +/// \brief Represents removing table properties +class ICEBERG_EXPORT RemoveProperties : public MetadataUpdate { + public: + explicit RemoveProperties(std::vector removed) + : removed_(std::move(removed)) {} + + const std::vector& removed() const { return removed_; } + + std::unique_ptr Clone() const override { + return std::make_unique(removed_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::vector removed_; +}; + +/// \brief Represents setting the table location +class ICEBERG_EXPORT SetLocation : public MetadataUpdate { + public: + explicit SetLocation(std::string location) : location_(std::move(location)) {} + + const std::string& location() const { return location_; } + + std::unique_ptr Clone() const override { + return std::make_unique(location_); + } + + void ApplyTo(TableMetadataBuilder& builder) const override; + + void GenerateRequirements(UpdateRequirementsContext& context) const override; + + private: + std::string location_; +}; + +} // namespace iceberg diff --git a/src/iceberg/table_metadata.h b/src/iceberg/table_metadata.h index 6f7a81904..8a3015d21 100644 --- a/src/iceberg/table_metadata.h +++ b/src/iceberg/table_metadata.h @@ -29,6 +29,7 @@ #include #include "iceberg/iceberg_export.h" +#include "iceberg/metadata_update.h" #include "iceberg/type_fwd.h" #include "iceberg/util/timepoint.h" diff --git a/src/iceberg/update_requirement.cc b/src/iceberg/update_requirement.cc new file mode 100644 index 000000000..25451f312 --- /dev/null +++ b/src/iceberg/update_requirement.cc @@ -0,0 +1,148 @@ +/* + * 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. + */ + +#include "iceberg/update_requirement.h" + +#include "iceberg/table_metadata.h" +#include "iceberg/util/string_util.h" + +namespace iceberg { + +Status AssertTableDoesNotExist::Validate(const TableMetadata* base) const { + if (base != nullptr) { + return CommitFailed("Requirement failed: table already exists"); + } + return {}; +} + +Status AssertTableUUID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + if (uuid_.empty()) { + return CommitFailed("Requirement failed: expected non-empty UUID"); + } + + if (!StringUtils::EqualsIgnoreCase(uuid_, base->table_uuid)) { + return CommitFailed("Requirement failed: UUID does not match: expected {} != {}", + base->table_uuid, uuid_); + } + + return {}; +} + +Status AssertRefSnapshotID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + // Find the reference in the table metadata + auto it = base->refs.find(ref_name_); + + if (it != base->refs.end()) { + // Reference exists + const auto& ref = it->second; + std::string type = (ref->type() == SnapshotRefType::kBranch) ? "branch" : "tag"; + + if (!snapshot_id_.has_value()) { + // A null snapshot ID means the ref should not exist already + return CommitFailed("Requirement failed: {} {} was created concurrently", type, + ref_name_); + } + if (snapshot_id_.value() != ref->snapshot_id) { + return CommitFailed("Requirement failed: {} {} has changed: expected id {} != {}", + type, ref_name_, snapshot_id_.value(), ref->snapshot_id); + } + } else { + // Reference does not exist + if (snapshot_id_.has_value()) { + return CommitFailed("Requirement failed: branch or tag {} is missing, expected {}", + ref_name_, snapshot_id_.value()); + } + } + + return {}; +} + +Status AssertLastAssignedFieldId::Validate(const TableMetadata* base) const { + if (base != nullptr && base->last_column_id != last_assigned_field_id_) { + return CommitFailed( + "Requirement failed: last assigned field id changed: expected id {} != {}", + last_assigned_field_id_, base->last_column_id); + } + return {}; +} + +Status AssertCurrentSchemaID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + if (!base->current_schema_id.has_value()) { + return CommitFailed("Requirement failed: table has no current schema"); + } + + if (schema_id_ != base->current_schema_id.value()) { + return CommitFailed( + "Requirement failed: current schema changed: expected id {} != {}", schema_id_, + base->current_schema_id.value()); + } + + return {}; +} + +Status AssertLastAssignedPartitionId::Validate(const TableMetadata* base) const { + if (base != nullptr && base->last_partition_id != last_assigned_partition_id_) { + return CommitFailed( + "Requirement failed: last assigned partition id changed: expected id {} != {}", + last_assigned_partition_id_, base->last_partition_id); + } + return {}; +} + +Status AssertDefaultSpecID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + if (spec_id_ != base->default_spec_id) { + return CommitFailed( + "Requirement failed: default partition spec changed: expected id {} != {}", + spec_id_, base->default_spec_id); + } + + return {}; +} + +Status AssertDefaultSortOrderID::Validate(const TableMetadata* base) const { + if (base == nullptr) { + return CommitFailed("Requirement failed: table does not exist"); + } + + if (sort_order_id_ != base->default_sort_order_id) { + return CommitFailed( + "Requirement failed: default sort order changed: expected id {} != {}", + sort_order_id_, base->default_sort_order_id); + } + + return {}; +} + +} // namespace iceberg diff --git a/src/iceberg/update_requirement.h b/src/iceberg/update_requirement.h new file mode 100644 index 000000000..50b570c30 --- /dev/null +++ b/src/iceberg/update_requirement.h @@ -0,0 +1,220 @@ +/* + * 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 + +/// \file iceberg/update_requirement.h +/// Update requirements for Iceberg table operations. +/// +/// Update requirements are conditions that must be satisfied before +/// applying metadata updates to a table. They are used for optimistic +/// concurrency control in table operations. + +#include +#include +#include + +#include "iceberg/iceberg_export.h" +#include "iceberg/result.h" +#include "iceberg/type_fwd.h" + +namespace iceberg { + +/// \brief Base class for update requirement operations +/// +/// Represents a requirement that must be validated before applying +/// metadata updates to a table. Each concrete subclass represents +/// a specific type of requirement check. +class ICEBERG_EXPORT UpdateRequirement { + public: + virtual ~UpdateRequirement() = default; + + /// \brief Clone this update requirement + virtual std::unique_ptr Clone() const = 0; + + /// \brief Validate this requirement against table metadata + /// + /// \param base The base table metadata to validate against (may be nullptr) + /// \return Status indicating success or failure with error details + virtual Status Validate(const TableMetadata* base) const = 0; +}; + +/// \brief Requirement that the table does not exist +/// +/// This requirement is used when creating a new table to ensure +/// it doesn't already exist. +class ICEBERG_EXPORT AssertTableDoesNotExist : public UpdateRequirement { + public: + AssertTableDoesNotExist() = default; + + std::unique_ptr Clone() const override { + return std::make_unique(); + } + + Status Validate(const TableMetadata* base) const override; +}; + +/// \brief Requirement that the table UUID matches the expected value +/// +/// This ensures the table hasn't been replaced or recreated between +/// reading the metadata and attempting to update it. +class ICEBERG_EXPORT AssertTableUUID : public UpdateRequirement { + public: + explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} + + const std::string& uuid() const { return uuid_; } + + std::unique_ptr Clone() const override { + return std::make_unique(uuid_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + std::string uuid_; +}; + +/// \brief Requirement that a reference (branch or tag) points to a specific snapshot +/// +/// This requirement validates that a named reference (branch or tag) either: +/// - Points to the expected snapshot ID +/// - Does not exist (if snapshot_id is nullopt) +class ICEBERG_EXPORT AssertRefSnapshotID : public UpdateRequirement { + public: + AssertRefSnapshotID(std::string ref_name, std::optional snapshot_id) + : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} + + const std::string& ref_name() const { return ref_name_; } + const std::optional& snapshot_id() const { return snapshot_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(ref_name_, snapshot_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + std::string ref_name_; + std::optional snapshot_id_; +}; + +/// \brief Requirement that the last assigned field ID matches +/// +/// This ensures the schema hasn't been modified (by adding fields) +/// since the metadata was read. +class ICEBERG_EXPORT AssertLastAssignedFieldId : public UpdateRequirement { + public: + explicit AssertLastAssignedFieldId(int32_t last_assigned_field_id) + : last_assigned_field_id_(last_assigned_field_id) {} + + int32_t last_assigned_field_id() const { return last_assigned_field_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(last_assigned_field_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t last_assigned_field_id_; +}; + +/// \brief Requirement that the current schema ID matches +/// +/// This ensures the active schema hasn't changed since the +/// metadata was read. +class ICEBERG_EXPORT AssertCurrentSchemaID : public UpdateRequirement { + public: + explicit AssertCurrentSchemaID(int32_t schema_id) : schema_id_(schema_id) {} + + int32_t schema_id() const { return schema_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(schema_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t schema_id_; +}; + +/// \brief Requirement that the last assigned partition ID matches +/// +/// This ensures partition specs haven't been modified since the +/// metadata was read. +class ICEBERG_EXPORT AssertLastAssignedPartitionId : public UpdateRequirement { + public: + explicit AssertLastAssignedPartitionId(int32_t last_assigned_partition_id) + : last_assigned_partition_id_(last_assigned_partition_id) {} + + int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(last_assigned_partition_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t last_assigned_partition_id_; +}; + +/// \brief Requirement that the default partition spec ID matches +/// +/// This ensures the default partition spec hasn't changed since +/// the metadata was read. +class ICEBERG_EXPORT AssertDefaultSpecID : public UpdateRequirement { + public: + explicit AssertDefaultSpecID(int32_t spec_id) : spec_id_(spec_id) {} + + int32_t spec_id() const { return spec_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(spec_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t spec_id_; +}; + +/// \brief Requirement that the default sort order ID matches +/// +/// This ensures the default sort order hasn't changed since +/// the metadata was read. +class ICEBERG_EXPORT AssertDefaultSortOrderID : public UpdateRequirement { + public: + explicit AssertDefaultSortOrderID(int32_t sort_order_id) + : sort_order_id_(sort_order_id) {} + + int32_t sort_order_id() const { return sort_order_id_; } + + std::unique_ptr Clone() const override { + return std::make_unique(sort_order_id_); + } + + Status Validate(const TableMetadata* base) const override; + + private: + int32_t sort_order_id_; +}; + +} // namespace iceberg diff --git a/src/iceberg/update_requirements.cc b/src/iceberg/update_requirements.cc new file mode 100644 index 000000000..d3c2df959 --- /dev/null +++ b/src/iceberg/update_requirements.cc @@ -0,0 +1,77 @@ +/* + * 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. + */ + +#include "iceberg/update_requirements.h" + +#include "iceberg/metadata_update.h" +#include "iceberg/table_metadata.h" + +namespace iceberg { + +std::vector> UpdateRequirements::ForCreateTable( + const std::vector>& metadata_updates) { + // Create context for table creation (no base metadata) + UpdateRequirementsContext context(nullptr, false); + + // Add requirement that table does not exist + context.AddRequirement(std::make_unique()); + + // Let each metadata update generate its requirements + for (const auto& update : metadata_updates) { + update->GenerateRequirements(context); + } + + return context.Build(); +} + +std::vector> UpdateRequirements::ForReplaceTable( + const TableMetadata& base, + const std::vector>& metadata_updates) { + // Create context for table replacement (is_replace = true) + UpdateRequirementsContext context(&base, true); + + // Add requirement that UUID matches + context.AddRequirement(std::make_unique(base.table_uuid)); + + // Let each metadata update generate its requirements + for (const auto& update : metadata_updates) { + update->GenerateRequirements(context); + } + + return context.Build(); +} + +std::vector> UpdateRequirements::ForUpdateTable( + const TableMetadata& base, + const std::vector>& metadata_updates) { + // Create context for table update (is_replace = false) + UpdateRequirementsContext context(&base, false); + + // Add requirement that UUID matches + context.AddRequirement(std::make_unique(base.table_uuid)); + + // Let each metadata update generate its requirements + for (const auto& update : metadata_updates) { + update->GenerateRequirements(context); + } + + return context.Build(); +} + +} // namespace iceberg diff --git a/src/iceberg/update_requirements.h b/src/iceberg/update_requirements.h new file mode 100644 index 000000000..d7ad97463 --- /dev/null +++ b/src/iceberg/update_requirements.h @@ -0,0 +1,122 @@ +/* + * 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 + +/// \file iceberg/update_requirements.h +/// Factory for generating update requirements from metadata updates. +/// +/// This utility class generates the appropriate UpdateRequirement instances +/// based on a list of MetadataUpdate operations. The requirements are used +/// for optimistic concurrency control when committing table changes. + +#include +#include + +#include "iceberg/iceberg_export.h" +#include "iceberg/type_fwd.h" +#include "iceberg/update_requirement.h" + +namespace iceberg { + +/// \brief Context for generating update requirements +/// +/// This context is passed to each MetadataUpdate's GenerateRequirements method +/// and maintains state about what requirements have already been added to avoid +/// duplicates. +class ICEBERG_EXPORT UpdateRequirementsContext { + public: + /// \brief Construct a context for requirement generation + /// + /// \param base The base table metadata (may be nullptr for table creation) + /// \param is_replace Whether this is a replace operation (more permissive) + UpdateRequirementsContext(const TableMetadata* base, bool is_replace) + : base_(base), is_replace_(is_replace) {} + + // Delete copy operations (contains unique_ptr members) + UpdateRequirementsContext(const UpdateRequirementsContext&) = delete; + UpdateRequirementsContext& operator=(const UpdateRequirementsContext&) = delete; + + // Enable move construction only (assignment deleted due to const members) + UpdateRequirementsContext(UpdateRequirementsContext&&) noexcept = default; + + /// \brief Add a requirement to the list + void AddRequirement(std::unique_ptr requirement) { + requirements_.push_back(std::move(requirement)); + } + + /// \brief Get the base table metadata + const TableMetadata* base() const { return base_; } + + /// \brief Check if this is a replace operation + bool is_replace() const { return is_replace_; } + + /// \brief Build and return the list of requirements + std::vector> Build() { + return std::move(requirements_); + } + + private: + const TableMetadata* base_; + const bool is_replace_; + + std::vector> requirements_; +}; + +/// \brief Factory class for generating update requirements +/// +/// This class analyzes a sequence of metadata updates and generates the +/// appropriate update requirements to ensure safe concurrent modifications. +class ICEBERG_EXPORT UpdateRequirements { + public: + /// \brief Generate requirements for creating a new table + /// + /// For table creation, this requires that the table does not already exist. + /// + /// \param metadata_updates The list of metadata updates for table creation + /// \return A list of update requirements to validate before creation + static std::vector> ForCreateTable( + const std::vector>& metadata_updates); + + /// \brief Generate requirements for replacing an existing table + /// + /// For table replacement, this requires that the table UUID matches but + /// allows more aggressive changes than a regular update. + /// + /// \param base The base table metadata + /// \param metadata_updates The list of metadata updates for replacement + /// \return A list of update requirements to validate before replacement + static std::vector> ForReplaceTable( + const TableMetadata& base, + const std::vector>& metadata_updates); + + /// \brief Generate requirements for updating an existing table + /// + /// For table updates, this generates requirements to ensure that key + /// metadata properties haven't changed concurrently. + /// + /// \param base The base table metadata + /// \param metadata_updates The list of metadata updates + /// \return A list of update requirements to validate before update + static std::vector> ForUpdateTable( + const TableMetadata& base, + const std::vector>& metadata_updates); +}; + +} // namespace iceberg From e92f45cabd342add9c408a7c658725406f851c0e Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Wed, 15 Oct 2025 22:59:59 +0800 Subject: [PATCH 08/14] fix comments --- src/iceberg/CMakeLists.txt | 1 - src/iceberg/metadata_update.cc | 285 ------------------- src/iceberg/metadata_update.h | 431 ----------------------------- src/iceberg/table_metadata.cc | 1 + src/iceberg/table_requirement.h | 232 ++++++++-------- src/iceberg/update_requirement.h | 220 --------------- src/iceberg/update_requirements.cc | 77 ------ src/iceberg/update_requirements.h | 122 -------- 8 files changed, 117 insertions(+), 1252 deletions(-) delete mode 100644 src/iceberg/metadata_update.cc delete mode 100644 src/iceberg/metadata_update.h delete mode 100644 src/iceberg/update_requirement.h delete mode 100644 src/iceberg/update_requirements.cc delete mode 100644 src/iceberg/update_requirements.h diff --git a/src/iceberg/CMakeLists.txt b/src/iceberg/CMakeLists.txt index 500d23686..eaaf6248c 100644 --- a/src/iceberg/CMakeLists.txt +++ b/src/iceberg/CMakeLists.txt @@ -36,7 +36,6 @@ set(ICEBERG_SOURCES manifest_reader_internal.cc manifest_writer.cc metadata_columns.cc - metadata_update.cc name_mapping.cc partition_field.cc partition_spec.cc diff --git a/src/iceberg/metadata_update.cc b/src/iceberg/metadata_update.cc deleted file mode 100644 index cd171c4de..000000000 --- a/src/iceberg/metadata_update.cc +++ /dev/null @@ -1,285 +0,0 @@ -/* - * 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. - */ - -#include "iceberg/metadata_update.h" - -#include "iceberg/table_metadata.h" -#include "iceberg/update_requirements.h" - -namespace iceberg { - -// AssignUUID - -void AssignUUID::ApplyTo(TableMetadataBuilder& builder) const { - builder.AssignUUID(uuid_); -} - -void AssignUUID::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // AssignUUID doesn't generate any requirements -} - -// UpgradeFormatVersion - -void UpgradeFormatVersion::ApplyTo(TableMetadataBuilder& builder) const { - builder.UpgradeFormatVersion(format_version_); -} - -void UpgradeFormatVersion::GenerateRequirements( - UpdateRequirementsContext& /*context*/) const { - // UpgradeFormatVersion doesn't generate any requirements -} - -// AddSchema - -void AddSchema::ApplyTo(TableMetadataBuilder& builder) const { - builder.AddSchema(schema_); -} - -void AddSchema::GenerateRequirements(UpdateRequirementsContext& context) const { - if (context.base() != nullptr) { - context.AddRequirement( - std::make_unique(context.base()->last_column_id)); - } -} - -// SetCurrentSchema - -void SetCurrentSchema::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetCurrentSchema(schema_id_); -} - -void SetCurrentSchema::GenerateRequirements(UpdateRequirementsContext& context) const { - // Require current schema not changed - if (context.base() != nullptr && !context.is_replace()) { - if (context.base()->current_schema_id.has_value()) { - context.AddRequirement(std::make_unique( - context.base()->current_schema_id.value())); - } - } -} - -// AddPartitionSpec - -void AddPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { - builder.AddPartitionSpec(spec_); -} - -void AddPartitionSpec::GenerateRequirements(UpdateRequirementsContext& context) const { - if (context.base() != nullptr) { - context.AddRequirement(std::make_unique( - context.base()->last_partition_id)); - } -} - -// SetDefaultPartitionSpec - -void SetDefaultPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetDefaultPartitionSpec(spec_id_); -} - -void SetDefaultPartitionSpec::GenerateRequirements( - UpdateRequirementsContext& context) const { - // Require default partition spec not changed - if (context.base() != nullptr && !context.is_replace()) { - context.AddRequirement( - std::make_unique(context.base()->default_spec_id)); - } -} - -// RemovePartitionSpecs - -void RemovePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemovePartitionSpecs(spec_ids_); -} - -void RemovePartitionSpecs::GenerateRequirements( - UpdateRequirementsContext& context) const { - // Require default partition spec not changed - if (context.base() != nullptr && !context.is_replace()) { - context.AddRequirement( - std::make_unique(context.base()->default_spec_id)); - } - - // Require that no branches have changed - if (context.base() != nullptr && !context.is_replace()) { - for (const auto& [name, ref] : context.base()->refs) { - if (ref->type() == SnapshotRefType::kBranch && name != "main") { - context.AddRequirement( - std::make_unique(name, ref->snapshot_id)); - } - } - } -} - -// RemoveSchemas - -void RemoveSchemas::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemoveSchemas(schema_ids_); -} - -void RemoveSchemas::GenerateRequirements(UpdateRequirementsContext& context) const { - // Require current schema not changed - if (context.base() != nullptr && !context.is_replace()) { - if (context.base()->current_schema_id.has_value()) { - context.AddRequirement(std::make_unique( - context.base()->current_schema_id.value())); - } - } - - // Require that no branches have changed - if (context.base() != nullptr && !context.is_replace()) { - for (const auto& [name, ref] : context.base()->refs) { - if (ref->type() == SnapshotRefType::kBranch && name != "main") { - context.AddRequirement( - std::make_unique(name, ref->snapshot_id)); - } - } - } -} - -// AddSortOrder - -void AddSortOrder::ApplyTo(TableMetadataBuilder& builder) const { - builder.AddSortOrder(sort_order_); -} - -void AddSortOrder::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // AddSortOrder doesn't generate any requirements -} - -// SetDefaultSortOrder - -void SetDefaultSortOrder::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetDefaultSortOrder(sort_order_id_); -} - -void SetDefaultSortOrder::GenerateRequirements(UpdateRequirementsContext& context) const { - if (context.base() != nullptr && !context.is_replace()) { - context.AddRequirement(std::make_unique( - context.base()->default_sort_order_id)); - } -} - -// AddSnapshot - -void AddSnapshot::ApplyTo(TableMetadataBuilder& builder) const { - builder.AddSnapshot(snapshot_); -} - -void AddSnapshot::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // AddSnapshot doesn't generate any requirements -} - -// RemoveSnapshots - -void RemoveSnapshots::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemoveSnapshots(snapshot_ids_); -} - -void RemoveSnapshots::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // RemoveSnapshots doesn't generate any requirements -} - -// RemoveSnapshotRef - -void RemoveSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemoveRef(ref_name_); -} - -void RemoveSnapshotRef::GenerateRequirements( - UpdateRequirementsContext& /*context*/) const { - // RemoveSnapshotRef doesn't generate any requirements -} - -// SetSnapshotRef - -void SetSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { - // Create a SnapshotRef based on the type - std::shared_ptr ref; - - if (type_ == SnapshotRefType::kBranch) { - SnapshotRef::Branch branch; - branch.min_snapshots_to_keep = min_snapshots_to_keep_; - branch.max_snapshot_age_ms = max_snapshot_age_ms_; - branch.max_ref_age_ms = max_ref_age_ms_; - - ref = std::make_shared(); - ref->snapshot_id = snapshot_id_; - ref->retention = branch; - } else { - SnapshotRef::Tag tag; - tag.max_ref_age_ms = max_ref_age_ms_; - - ref = std::make_shared(); - ref->snapshot_id = snapshot_id_; - ref->retention = tag; - } - - builder.SetRef(ref_name_, ref); -} - -void SetSnapshotRef::GenerateRequirements(UpdateRequirementsContext& context) const { - // Require that the ref is unchanged from the base - if (context.base() != nullptr && !context.is_replace()) { - // Find the reference in the base metadata - auto it = context.base()->refs.find(ref_name_); - std::optional base_snapshot_id; - - if (it != context.base()->refs.end()) { - base_snapshot_id = it->second->snapshot_id; - } - - // Require that the ref does not exist (nullopt) or is the same as the base snapshot - context.AddRequirement( - std::make_unique(ref_name_, base_snapshot_id)); - } -} - -// SetProperties - -void SetProperties::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetProperties(updated_); -} - -void SetProperties::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // SetProperties doesn't generate any requirements -} - -// RemoveProperties - -void RemoveProperties::ApplyTo(TableMetadataBuilder& builder) const { - builder.RemoveProperties(removed_); -} - -void RemoveProperties::GenerateRequirements( - UpdateRequirementsContext& /*context*/) const { - // RemoveProperties doesn't generate any requirements -} - -// SetLocation - -void SetLocation::ApplyTo(TableMetadataBuilder& builder) const { - builder.SetLocation(location_); -} - -void SetLocation::GenerateRequirements(UpdateRequirementsContext& /*context*/) const { - // SetLocation doesn't generate any requirements -} - -} // namespace iceberg diff --git a/src/iceberg/metadata_update.h b/src/iceberg/metadata_update.h deleted file mode 100644 index 8aeaea3a6..000000000 --- a/src/iceberg/metadata_update.h +++ /dev/null @@ -1,431 +0,0 @@ -/* - * 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 - -/// \file iceberg/metadata_update.h -/// Metadata update operations for Iceberg tables. - -#include -#include -#include -#include -#include - -#include "iceberg/iceberg_export.h" -#include "iceberg/snapshot.h" -#include "iceberg/type_fwd.h" - -namespace iceberg { - -class TableMetadataBuilder; -class UpdateRequirementsContext; - -/// \brief Base class for metadata update operations -/// -/// Represents a change to table metadata. Each concrete subclass -/// represents a specific type of update operation. -class ICEBERG_EXPORT MetadataUpdate { - public: - virtual ~MetadataUpdate() = default; - - /// \brief Clone this metadata update - virtual std::unique_ptr Clone() const = 0; - - /// \brief Apply this update to a TableMetadataBuilder - /// - /// This method modifies the builder by applying the update operation - /// it represents. Each subclass implements this to apply its specific - /// type of update. - /// - /// \param builder The builder to apply this update to - virtual void ApplyTo(TableMetadataBuilder& builder) const = 0; - - /// \brief Generate update requirements for this metadata update - /// - /// This method generates the appropriate UpdateRequirement instances - /// that must be validated before this update can be applied. The context - /// provides information about the base metadata and operation mode. - /// - /// \param context The context containing base metadata and operation state - virtual void GenerateRequirements(UpdateRequirementsContext& context) const = 0; -}; - -/// \brief Represents an assignment of a UUID to the table -class ICEBERG_EXPORT AssignUUID : public MetadataUpdate { - public: - explicit AssignUUID(std::string uuid) : uuid_(std::move(uuid)) {} - - const std::string& uuid() const { return uuid_; } - - std::unique_ptr Clone() const override { - return std::make_unique(uuid_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::string uuid_; -}; - -/// \brief Represents an upgrade of the table format version -class ICEBERG_EXPORT UpgradeFormatVersion : public MetadataUpdate { - public: - explicit UpgradeFormatVersion(int8_t format_version) - : format_version_(format_version) {} - - int8_t format_version() const { return format_version_; } - - std::unique_ptr Clone() const override { - return std::make_unique(format_version_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - int8_t format_version_; -}; - -/// \brief Represents adding a new schema to the table -class ICEBERG_EXPORT AddSchema : public MetadataUpdate { - public: - explicit AddSchema(std::shared_ptr schema, int32_t last_column_id) - : schema_(std::move(schema)), last_column_id_(last_column_id) {} - - const std::shared_ptr& schema() const { return schema_; } - - int32_t last_column_id() const { return last_column_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(schema_, last_column_id_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::shared_ptr schema_; - int32_t last_column_id_; -}; - -/// \brief Represents setting the current schema -class ICEBERG_EXPORT SetCurrentSchema : public MetadataUpdate { - public: - explicit SetCurrentSchema(int32_t schema_id) : schema_id_(schema_id) {} - - int32_t schema_id() const { return schema_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(schema_id_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - int32_t schema_id_; -}; - -/// \brief Represents adding a new partition spec to the table -class ICEBERG_EXPORT AddPartitionSpec : public MetadataUpdate { - public: - explicit AddPartitionSpec(std::shared_ptr spec) - : spec_(std::move(spec)) {} - - const std::shared_ptr& spec() const { return spec_; } - - std::unique_ptr Clone() const override { - return std::make_unique(spec_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::shared_ptr spec_; -}; - -/// \brief Represents setting the default partition spec -class ICEBERG_EXPORT SetDefaultPartitionSpec : public MetadataUpdate { - public: - explicit SetDefaultPartitionSpec(int32_t spec_id) : spec_id_(spec_id) {} - - int32_t spec_id() const { return spec_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(spec_id_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - int32_t spec_id_; -}; - -/// \brief Represents removing partition specs from the table -class ICEBERG_EXPORT RemovePartitionSpecs : public MetadataUpdate { - public: - explicit RemovePartitionSpecs(std::vector spec_ids) - : spec_ids_(std::move(spec_ids)) {} - - const std::vector& spec_ids() const { return spec_ids_; } - - std::unique_ptr Clone() const override { - return std::make_unique(spec_ids_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::vector spec_ids_; -}; - -/// \brief Represents removing schemas from the table -class ICEBERG_EXPORT RemoveSchemas : public MetadataUpdate { - public: - explicit RemoveSchemas(std::vector schema_ids) - : schema_ids_(std::move(schema_ids)) {} - - const std::vector& schema_ids() const { return schema_ids_; } - - std::unique_ptr Clone() const override { - return std::make_unique(schema_ids_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::vector schema_ids_; -}; - -/// \brief Represents adding a new sort order to the table -class ICEBERG_EXPORT AddSortOrder : public MetadataUpdate { - public: - explicit AddSortOrder(std::shared_ptr sort_order) - : sort_order_(std::move(sort_order)) {} - - const std::shared_ptr& sort_order() const { return sort_order_; } - - std::unique_ptr Clone() const override { - return std::make_unique(sort_order_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::shared_ptr sort_order_; -}; - -/// \brief Represents setting the default sort order -class ICEBERG_EXPORT SetDefaultSortOrder : public MetadataUpdate { - public: - explicit SetDefaultSortOrder(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} - - int32_t sort_order_id() const { return sort_order_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(sort_order_id_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - int32_t sort_order_id_; -}; - -/// \brief Represents adding a snapshot to the table -class ICEBERG_EXPORT AddSnapshot : public MetadataUpdate { - public: - explicit AddSnapshot(std::shared_ptr snapshot) - : snapshot_(std::move(snapshot)) {} - - const std::shared_ptr& snapshot() const { return snapshot_; } - - std::unique_ptr Clone() const override { - return std::make_unique(snapshot_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::shared_ptr snapshot_; -}; - -/// \brief Represents removing snapshots from the table -class ICEBERG_EXPORT RemoveSnapshots : public MetadataUpdate { - public: - explicit RemoveSnapshots(std::vector snapshot_ids) - : snapshot_ids_(std::move(snapshot_ids)) {} - - const std::vector& snapshot_ids() const { return snapshot_ids_; } - - std::unique_ptr Clone() const override { - return std::make_unique(snapshot_ids_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::vector snapshot_ids_; -}; - -/// \brief Represents removing a snapshot reference -class ICEBERG_EXPORT RemoveSnapshotRef : public MetadataUpdate { - public: - explicit RemoveSnapshotRef(std::string ref_name) : ref_name_(std::move(ref_name)) {} - - const std::string& ref_name() const { return ref_name_; } - - std::unique_ptr Clone() const override { - return std::make_unique(ref_name_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::string ref_name_; -}; - -/// \brief Represents setting a snapshot reference -class ICEBERG_EXPORT SetSnapshotRef : public MetadataUpdate { - public: - SetSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, - std::optional min_snapshots_to_keep = std::nullopt, - std::optional max_snapshot_age_ms = std::nullopt, - std::optional max_ref_age_ms = std::nullopt) - : ref_name_(std::move(ref_name)), - snapshot_id_(snapshot_id), - type_(type), - min_snapshots_to_keep_(min_snapshots_to_keep), - max_snapshot_age_ms_(max_snapshot_age_ms), - max_ref_age_ms_(max_ref_age_ms) {} - - const std::string& ref_name() const { return ref_name_; } - int64_t snapshot_id() const { return snapshot_id_; } - SnapshotRefType type() const { return type_; } - const std::optional& min_snapshots_to_keep() const { - return min_snapshots_to_keep_; - } - const std::optional& max_snapshot_age_ms() const { - return max_snapshot_age_ms_; - } - const std::optional& max_ref_age_ms() const { return max_ref_age_ms_; } - - std::unique_ptr Clone() const override { - return std::make_unique(ref_name_, snapshot_id_, type_, - min_snapshots_to_keep_, max_snapshot_age_ms_, - max_ref_age_ms_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::string ref_name_; - int64_t snapshot_id_; - SnapshotRefType type_; - std::optional min_snapshots_to_keep_; - std::optional max_snapshot_age_ms_; - std::optional max_ref_age_ms_; -}; - -/// \brief Represents setting table properties -class ICEBERG_EXPORT SetProperties : public MetadataUpdate { - public: - explicit SetProperties(std::unordered_map updated) - : updated_(std::move(updated)) {} - - const std::unordered_map& updated() const { return updated_; } - - std::unique_ptr Clone() const override { - return std::make_unique(updated_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::unordered_map updated_; -}; - -/// \brief Represents removing table properties -class ICEBERG_EXPORT RemoveProperties : public MetadataUpdate { - public: - explicit RemoveProperties(std::vector removed) - : removed_(std::move(removed)) {} - - const std::vector& removed() const { return removed_; } - - std::unique_ptr Clone() const override { - return std::make_unique(removed_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::vector removed_; -}; - -/// \brief Represents setting the table location -class ICEBERG_EXPORT SetLocation : public MetadataUpdate { - public: - explicit SetLocation(std::string location) : location_(std::move(location)) {} - - const std::string& location() const { return location_; } - - std::unique_ptr Clone() const override { - return std::make_unique(location_); - } - - void ApplyTo(TableMetadataBuilder& builder) const override; - - void GenerateRequirements(UpdateRequirementsContext& context) const override; - - private: - std::string location_; -}; - -} // namespace iceberg diff --git a/src/iceberg/table_metadata.cc b/src/iceberg/table_metadata.cc index 7f1d41eaa..c501f2ab2 100644 --- a/src/iceberg/table_metadata.cc +++ b/src/iceberg/table_metadata.cc @@ -32,6 +32,7 @@ #include "iceberg/schema.h" #include "iceberg/snapshot.h" #include "iceberg/sort_order.h" +#include "iceberg/table_update.h" #include "iceberg/util/gzip_internal.h" #include "iceberg/util/macros.h" diff --git a/src/iceberg/table_requirement.h b/src/iceberg/table_requirement.h index 6046b1178..948709fde 100644 --- a/src/iceberg/table_requirement.h +++ b/src/iceberg/table_requirement.h @@ -33,153 +33,153 @@ #include "iceberg/result.h" #include "iceberg/type_fwd.h" -namespace iceberg { + namespace iceberg { -/// \brief Base class for update requirement operations -/// -/// Represents a requirement that must be validated before applying -/// metadata updates to a table. Each concrete subclass represents -/// a specific type of requirement check. -class ICEBERG_EXPORT TableRequirement { - public: - virtual ~TableRequirement() = default; - - /// \brief Validate this requirement against table metadata + /// \brief Base class for update requirement operations /// - /// \param base The base table metadata to validate against (may be nullptr) - /// \return Status indicating success or failure with error details - virtual Status Validate(const TableMetadata* base) const = 0; -}; - -/// \brief Requirement that the table does not exist -/// -/// This requirement is used when creating a new table to ensure -/// it doesn't already exist. -class ICEBERG_EXPORT AssertTableDoesNotExist : public TableRequirement { - public: - AssertTableDoesNotExist() = default; + /// Represents a requirement that must be validated before applying + /// metadata updates to a table. Each concrete subclass represents + /// a specific type of requirement check. + class ICEBERG_EXPORT TableRequirement { + public: + virtual ~TableRequirement() = default; + + /// \brief Validate this requirement against table metadata + /// + /// \param base The base table metadata to validate against (may be nullptr) + /// \return Status indicating success or failure with error details + virtual Status Validate(const TableMetadata* base) const = 0; + }; + + /// \brief Requirement that the table does not exist + /// + /// This requirement is used when creating a new table to ensure + /// it doesn't already exist. + class ICEBERG_EXPORT AssertTableDoesNotExist : public TableRequirement { + public: + AssertTableDoesNotExist() = default; - Status Validate(const TableMetadata* base) const override; -}; + Status Validate(const TableMetadata* base) const override; + }; -/// \brief Requirement that the table UUID matches the expected value -/// -/// This ensures the table hasn't been replaced or recreated between -/// reading the metadata and attempting to update it. -class ICEBERG_EXPORT AssertTableUUID : public TableRequirement { - public: - explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} + /// \brief Requirement that the table UUID matches the expected value + /// + /// This ensures the table hasn't been replaced or recreated between + /// reading the metadata and attempting to update it. + class ICEBERG_EXPORT AssertTableUUID : public TableRequirement { + public: + explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} - const std::string& uuid() const { return uuid_; } + const std::string& uuid() const { return uuid_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - std::string uuid_; -}; + private: + std::string uuid_; + }; -/// \brief Requirement that a reference (branch or tag) points to a specific snapshot -/// -/// This requirement validates that a named reference (branch or tag) either: -/// - Points to the expected snapshot ID -/// - Does not exist (if snapshot_id is nullopt) -class ICEBERG_EXPORT AssertRefSnapshotID : public TableRequirement { - public: - AssertRefSnapshotID(std::string ref_name, std::optional snapshot_id) - : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} + /// \brief Requirement that a reference (branch or tag) points to a specific snapshot + /// + /// This requirement validates that a named reference (branch or tag) either: + /// - Points to the expected snapshot ID + /// - Does not exist (if snapshot_id is nullopt) + class ICEBERG_EXPORT AssertRefSnapshotID : public TableRequirement { + public: + AssertRefSnapshotID(std::string ref_name, std::optional snapshot_id) + : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} - const std::string& ref_name() const { return ref_name_; } + const std::string& ref_name() const { return ref_name_; } - const std::optional& snapshot_id() const { return snapshot_id_; } + const std::optional& snapshot_id() const { return snapshot_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - std::string ref_name_; - std::optional snapshot_id_; -}; + private: + std::string ref_name_; + std::optional snapshot_id_; + }; -/// \brief Requirement that the last assigned field ID matches -/// -/// This ensures the schema hasn't been modified (by adding fields) -/// since the metadata was read. -class ICEBERG_EXPORT AssertLastAssignedFieldId : public TableRequirement { - public: - explicit AssertLastAssignedFieldId(int32_t last_assigned_field_id) - : last_assigned_field_id_(last_assigned_field_id) {} + /// \brief Requirement that the last assigned field ID matches + /// + /// This ensures the schema hasn't been modified (by adding fields) + /// since the metadata was read. + class ICEBERG_EXPORT AssertLastAssignedFieldId : public TableRequirement { + public: + explicit AssertLastAssignedFieldId(int32_t last_assigned_field_id) + : last_assigned_field_id_(last_assigned_field_id) {} - int32_t last_assigned_field_id() const { return last_assigned_field_id_; } + int32_t last_assigned_field_id() const { return last_assigned_field_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t last_assigned_field_id_; -}; + private: + int32_t last_assigned_field_id_; + }; -/// \brief Requirement that the current schema ID matches -/// -/// This ensures the active schema hasn't changed since the -/// metadata was read. -class ICEBERG_EXPORT AssertCurrentSchemaID : public TableRequirement { - public: - explicit AssertCurrentSchemaID(int32_t schema_id) : schema_id_(schema_id) {} + /// \brief Requirement that the current schema ID matches + /// + /// This ensures the active schema hasn't changed since the + /// metadata was read. + class ICEBERG_EXPORT AssertCurrentSchemaID : public TableRequirement { + public: + explicit AssertCurrentSchemaID(int32_t schema_id) : schema_id_(schema_id) {} - int32_t schema_id() const { return schema_id_; } + int32_t schema_id() const { return schema_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t schema_id_; -}; + private: + int32_t schema_id_; + }; -/// \brief Requirement that the last assigned partition ID matches -/// -/// This ensures partition specs haven't been modified since the -/// metadata was read. -class ICEBERG_EXPORT AssertLastAssignedPartitionId : public TableRequirement { - public: - explicit AssertLastAssignedPartitionId(int32_t last_assigned_partition_id) - : last_assigned_partition_id_(last_assigned_partition_id) {} + /// \brief Requirement that the last assigned partition ID matches + /// + /// This ensures partition specs haven't been modified since the + /// metadata was read. + class ICEBERG_EXPORT AssertLastAssignedPartitionId : public TableRequirement { + public: + explicit AssertLastAssignedPartitionId(int32_t last_assigned_partition_id) + : last_assigned_partition_id_(last_assigned_partition_id) {} - int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } + int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t last_assigned_partition_id_; -}; + private: + int32_t last_assigned_partition_id_; + }; -/// \brief Requirement that the default partition spec ID matches -/// -/// This ensures the default partition spec hasn't changed since -/// the metadata was read. -class ICEBERG_EXPORT AssertDefaultSpecID : public TableRequirement { - public: - explicit AssertDefaultSpecID(int32_t spec_id) : spec_id_(spec_id) {} + /// \brief Requirement that the default partition spec ID matches + /// + /// This ensures the default partition spec hasn't changed since + /// the metadata was read. + class ICEBERG_EXPORT AssertDefaultSpecID : public TableRequirement { + public: + explicit AssertDefaultSpecID(int32_t spec_id) : spec_id_(spec_id) {} - int32_t spec_id() const { return spec_id_; } + int32_t spec_id() const { return spec_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t spec_id_; -}; + private: + int32_t spec_id_; + }; -/// \brief Requirement that the default sort order ID matches -/// -/// This ensures the default sort order hasn't changed since -/// the metadata was read. -class ICEBERG_EXPORT AssertDefaultSortOrderID : public TableRequirement { - public: - explicit AssertDefaultSortOrderID(int32_t sort_order_id) - : sort_order_id_(sort_order_id) {} + /// \brief Requirement that the default sort order ID matches + /// + /// This ensures the default sort order hasn't changed since + /// the metadata was read. + class ICEBERG_EXPORT AssertDefaultSortOrderID : public TableRequirement { + public: + explicit AssertDefaultSortOrderID(int32_t sort_order_id) + : sort_order_id_(sort_order_id) {} - int32_t sort_order_id() const { return sort_order_id_; } + int32_t sort_order_id() const { return sort_order_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t sort_order_id_; -}; + private: + int32_t sort_order_id_; + }; } // namespace iceberg diff --git a/src/iceberg/update_requirement.h b/src/iceberg/update_requirement.h deleted file mode 100644 index 50b570c30..000000000 --- a/src/iceberg/update_requirement.h +++ /dev/null @@ -1,220 +0,0 @@ -/* - * 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 - -/// \file iceberg/update_requirement.h -/// Update requirements for Iceberg table operations. -/// -/// Update requirements are conditions that must be satisfied before -/// applying metadata updates to a table. They are used for optimistic -/// concurrency control in table operations. - -#include -#include -#include - -#include "iceberg/iceberg_export.h" -#include "iceberg/result.h" -#include "iceberg/type_fwd.h" - -namespace iceberg { - -/// \brief Base class for update requirement operations -/// -/// Represents a requirement that must be validated before applying -/// metadata updates to a table. Each concrete subclass represents -/// a specific type of requirement check. -class ICEBERG_EXPORT UpdateRequirement { - public: - virtual ~UpdateRequirement() = default; - - /// \brief Clone this update requirement - virtual std::unique_ptr Clone() const = 0; - - /// \brief Validate this requirement against table metadata - /// - /// \param base The base table metadata to validate against (may be nullptr) - /// \return Status indicating success or failure with error details - virtual Status Validate(const TableMetadata* base) const = 0; -}; - -/// \brief Requirement that the table does not exist -/// -/// This requirement is used when creating a new table to ensure -/// it doesn't already exist. -class ICEBERG_EXPORT AssertTableDoesNotExist : public UpdateRequirement { - public: - AssertTableDoesNotExist() = default; - - std::unique_ptr Clone() const override { - return std::make_unique(); - } - - Status Validate(const TableMetadata* base) const override; -}; - -/// \brief Requirement that the table UUID matches the expected value -/// -/// This ensures the table hasn't been replaced or recreated between -/// reading the metadata and attempting to update it. -class ICEBERG_EXPORT AssertTableUUID : public UpdateRequirement { - public: - explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} - - const std::string& uuid() const { return uuid_; } - - std::unique_ptr Clone() const override { - return std::make_unique(uuid_); - } - - Status Validate(const TableMetadata* base) const override; - - private: - std::string uuid_; -}; - -/// \brief Requirement that a reference (branch or tag) points to a specific snapshot -/// -/// This requirement validates that a named reference (branch or tag) either: -/// - Points to the expected snapshot ID -/// - Does not exist (if snapshot_id is nullopt) -class ICEBERG_EXPORT AssertRefSnapshotID : public UpdateRequirement { - public: - AssertRefSnapshotID(std::string ref_name, std::optional snapshot_id) - : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} - - const std::string& ref_name() const { return ref_name_; } - const std::optional& snapshot_id() const { return snapshot_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(ref_name_, snapshot_id_); - } - - Status Validate(const TableMetadata* base) const override; - - private: - std::string ref_name_; - std::optional snapshot_id_; -}; - -/// \brief Requirement that the last assigned field ID matches -/// -/// This ensures the schema hasn't been modified (by adding fields) -/// since the metadata was read. -class ICEBERG_EXPORT AssertLastAssignedFieldId : public UpdateRequirement { - public: - explicit AssertLastAssignedFieldId(int32_t last_assigned_field_id) - : last_assigned_field_id_(last_assigned_field_id) {} - - int32_t last_assigned_field_id() const { return last_assigned_field_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(last_assigned_field_id_); - } - - Status Validate(const TableMetadata* base) const override; - - private: - int32_t last_assigned_field_id_; -}; - -/// \brief Requirement that the current schema ID matches -/// -/// This ensures the active schema hasn't changed since the -/// metadata was read. -class ICEBERG_EXPORT AssertCurrentSchemaID : public UpdateRequirement { - public: - explicit AssertCurrentSchemaID(int32_t schema_id) : schema_id_(schema_id) {} - - int32_t schema_id() const { return schema_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(schema_id_); - } - - Status Validate(const TableMetadata* base) const override; - - private: - int32_t schema_id_; -}; - -/// \brief Requirement that the last assigned partition ID matches -/// -/// This ensures partition specs haven't been modified since the -/// metadata was read. -class ICEBERG_EXPORT AssertLastAssignedPartitionId : public UpdateRequirement { - public: - explicit AssertLastAssignedPartitionId(int32_t last_assigned_partition_id) - : last_assigned_partition_id_(last_assigned_partition_id) {} - - int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(last_assigned_partition_id_); - } - - Status Validate(const TableMetadata* base) const override; - - private: - int32_t last_assigned_partition_id_; -}; - -/// \brief Requirement that the default partition spec ID matches -/// -/// This ensures the default partition spec hasn't changed since -/// the metadata was read. -class ICEBERG_EXPORT AssertDefaultSpecID : public UpdateRequirement { - public: - explicit AssertDefaultSpecID(int32_t spec_id) : spec_id_(spec_id) {} - - int32_t spec_id() const { return spec_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(spec_id_); - } - - Status Validate(const TableMetadata* base) const override; - - private: - int32_t spec_id_; -}; - -/// \brief Requirement that the default sort order ID matches -/// -/// This ensures the default sort order hasn't changed since -/// the metadata was read. -class ICEBERG_EXPORT AssertDefaultSortOrderID : public UpdateRequirement { - public: - explicit AssertDefaultSortOrderID(int32_t sort_order_id) - : sort_order_id_(sort_order_id) {} - - int32_t sort_order_id() const { return sort_order_id_; } - - std::unique_ptr Clone() const override { - return std::make_unique(sort_order_id_); - } - - Status Validate(const TableMetadata* base) const override; - - private: - int32_t sort_order_id_; -}; - -} // namespace iceberg diff --git a/src/iceberg/update_requirements.cc b/src/iceberg/update_requirements.cc deleted file mode 100644 index d3c2df959..000000000 --- a/src/iceberg/update_requirements.cc +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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. - */ - -#include "iceberg/update_requirements.h" - -#include "iceberg/metadata_update.h" -#include "iceberg/table_metadata.h" - -namespace iceberg { - -std::vector> UpdateRequirements::ForCreateTable( - const std::vector>& metadata_updates) { - // Create context for table creation (no base metadata) - UpdateRequirementsContext context(nullptr, false); - - // Add requirement that table does not exist - context.AddRequirement(std::make_unique()); - - // Let each metadata update generate its requirements - for (const auto& update : metadata_updates) { - update->GenerateRequirements(context); - } - - return context.Build(); -} - -std::vector> UpdateRequirements::ForReplaceTable( - const TableMetadata& base, - const std::vector>& metadata_updates) { - // Create context for table replacement (is_replace = true) - UpdateRequirementsContext context(&base, true); - - // Add requirement that UUID matches - context.AddRequirement(std::make_unique(base.table_uuid)); - - // Let each metadata update generate its requirements - for (const auto& update : metadata_updates) { - update->GenerateRequirements(context); - } - - return context.Build(); -} - -std::vector> UpdateRequirements::ForUpdateTable( - const TableMetadata& base, - const std::vector>& metadata_updates) { - // Create context for table update (is_replace = false) - UpdateRequirementsContext context(&base, false); - - // Add requirement that UUID matches - context.AddRequirement(std::make_unique(base.table_uuid)); - - // Let each metadata update generate its requirements - for (const auto& update : metadata_updates) { - update->GenerateRequirements(context); - } - - return context.Build(); -} - -} // namespace iceberg diff --git a/src/iceberg/update_requirements.h b/src/iceberg/update_requirements.h deleted file mode 100644 index d7ad97463..000000000 --- a/src/iceberg/update_requirements.h +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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 - -/// \file iceberg/update_requirements.h -/// Factory for generating update requirements from metadata updates. -/// -/// This utility class generates the appropriate UpdateRequirement instances -/// based on a list of MetadataUpdate operations. The requirements are used -/// for optimistic concurrency control when committing table changes. - -#include -#include - -#include "iceberg/iceberg_export.h" -#include "iceberg/type_fwd.h" -#include "iceberg/update_requirement.h" - -namespace iceberg { - -/// \brief Context for generating update requirements -/// -/// This context is passed to each MetadataUpdate's GenerateRequirements method -/// and maintains state about what requirements have already been added to avoid -/// duplicates. -class ICEBERG_EXPORT UpdateRequirementsContext { - public: - /// \brief Construct a context for requirement generation - /// - /// \param base The base table metadata (may be nullptr for table creation) - /// \param is_replace Whether this is a replace operation (more permissive) - UpdateRequirementsContext(const TableMetadata* base, bool is_replace) - : base_(base), is_replace_(is_replace) {} - - // Delete copy operations (contains unique_ptr members) - UpdateRequirementsContext(const UpdateRequirementsContext&) = delete; - UpdateRequirementsContext& operator=(const UpdateRequirementsContext&) = delete; - - // Enable move construction only (assignment deleted due to const members) - UpdateRequirementsContext(UpdateRequirementsContext&&) noexcept = default; - - /// \brief Add a requirement to the list - void AddRequirement(std::unique_ptr requirement) { - requirements_.push_back(std::move(requirement)); - } - - /// \brief Get the base table metadata - const TableMetadata* base() const { return base_; } - - /// \brief Check if this is a replace operation - bool is_replace() const { return is_replace_; } - - /// \brief Build and return the list of requirements - std::vector> Build() { - return std::move(requirements_); - } - - private: - const TableMetadata* base_; - const bool is_replace_; - - std::vector> requirements_; -}; - -/// \brief Factory class for generating update requirements -/// -/// This class analyzes a sequence of metadata updates and generates the -/// appropriate update requirements to ensure safe concurrent modifications. -class ICEBERG_EXPORT UpdateRequirements { - public: - /// \brief Generate requirements for creating a new table - /// - /// For table creation, this requires that the table does not already exist. - /// - /// \param metadata_updates The list of metadata updates for table creation - /// \return A list of update requirements to validate before creation - static std::vector> ForCreateTable( - const std::vector>& metadata_updates); - - /// \brief Generate requirements for replacing an existing table - /// - /// For table replacement, this requires that the table UUID matches but - /// allows more aggressive changes than a regular update. - /// - /// \param base The base table metadata - /// \param metadata_updates The list of metadata updates for replacement - /// \return A list of update requirements to validate before replacement - static std::vector> ForReplaceTable( - const TableMetadata& base, - const std::vector>& metadata_updates); - - /// \brief Generate requirements for updating an existing table - /// - /// For table updates, this generates requirements to ensure that key - /// metadata properties haven't changed concurrently. - /// - /// \param base The base table metadata - /// \param metadata_updates The list of metadata updates - /// \return A list of update requirements to validate before update - static std::vector> ForUpdateTable( - const TableMetadata& base, - const std::vector>& metadata_updates); -}; - -} // namespace iceberg From 7f1b7d995a13853babd4b178deaef81981b9bd6b Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Fri, 17 Oct 2025 11:09:02 +0800 Subject: [PATCH 09/14] rename some class name and fix comments --- src/iceberg/CMakeLists.txt | 2 - src/iceberg/meson.build | 6 + src/iceberg/table_metadata.cc | 73 +++++++------ src/iceberg/table_requirement.cc | 27 +++-- src/iceberg/table_requirement.h | 2 +- src/iceberg/table_requirements.cc | 10 +- src/iceberg/table_requirements.h | 42 +++---- src/iceberg/table_update.cc | 176 ++++++++++++++++++------------ src/iceberg/table_update.h | 112 +++++++++---------- src/iceberg/type_fwd.h | 2 +- src/iceberg/util/string_util.h | 4 +- 11 files changed, 255 insertions(+), 201 deletions(-) diff --git a/src/iceberg/CMakeLists.txt b/src/iceberg/CMakeLists.txt index eaaf6248c..e37095043 100644 --- a/src/iceberg/CMakeLists.txt +++ b/src/iceberg/CMakeLists.txt @@ -59,8 +59,6 @@ set(ICEBERG_SOURCES transform.cc transform_function.cc type.cc - update_requirement.cc - update_requirements.cc util/conversions.cc util/decimal.cc util/gzip_internal.cc diff --git a/src/iceberg/meson.build b/src/iceberg/meson.build index df64ae023..25bfdc647 100644 --- a/src/iceberg/meson.build +++ b/src/iceberg/meson.build @@ -74,7 +74,10 @@ iceberg_sources = files( 'table.cc', 'table_metadata.cc', 'table_properties.cc', + 'table_requirement.cc', + 'table_requirements.cc', 'table_scan.cc', + 'table_update.cc', 'transform.cc', 'transform_function.cc', 'type.cc', @@ -169,7 +172,10 @@ install_headers( 'table.h', 'table_identifier.h', 'table_metadata.h', + 'table_requirement.h', + 'table_requirements.h', 'table_scan.h', + 'table_update.h', 'transaction.h', 'transform_function.h', 'transform.h', diff --git a/src/iceberg/table_metadata.cc b/src/iceberg/table_metadata.cc index c501f2ab2..9c9653bb4 100644 --- a/src/iceberg/table_metadata.cc +++ b/src/iceberg/table_metadata.cc @@ -25,6 +25,8 @@ #include +#include "../../build/_deps/fmt-src/include/fmt/color.h" +#include "iceberg/exception.h" #include "iceberg/file_io.h" #include "iceberg/json_internal.h" #include "iceberg/partition_spec.h" @@ -228,152 +230,157 @@ std::unique_ptr TableMetadataBuilder::BuildFrom( TableMetadataBuilder& TableMetadataBuilder::SetMetadataLocation( std::string_view metadata_location) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetPreviousMetadataLocation( std::string_view previous_metadata_location) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -TableMetadataBuilder& TableMetadataBuilder::AssignUUID() { return *this; } +TableMetadataBuilder& TableMetadataBuilder::AssignUUID() { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} TableMetadataBuilder& TableMetadataBuilder::AssignUUID(std::string_view uuid) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); + ; } TableMetadataBuilder& TableMetadataBuilder::UpgradeFormatVersion( int8_t new_format_version) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetCurrentSchema( std::shared_ptr schema, int32_t new_last_column_id) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetCurrentSchema(int32_t schema_id) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::AddSchema(std::shared_ptr schema) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetDefaultPartitionSpec( std::shared_ptr spec) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetDefaultPartitionSpec(int32_t spec_id) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::AddPartitionSpec( std::shared_ptr spec) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemovePartitionSpecs( const std::vector& spec_ids) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemoveSchemas( const std::vector& schema_ids) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetDefaultSortOrder( std::shared_ptr order) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetDefaultSortOrder(int32_t order_id) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::AddSortOrder( std::shared_ptr order) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::AddSnapshot( std::shared_ptr snapshot) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetBranchSnapshot(int64_t snapshot_id, const std::string& branch) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetRef(const std::string& name, std::shared_ptr ref) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemoveRef(const std::string& name) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemoveSnapshots( const std::vector>& snapshots_to_remove) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemoveSnapshots( const std::vector& snapshot_ids) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::suppressHistoricalSnapshots() { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetStatistics( const std::shared_ptr& statistics_file) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemoveStatistics(int64_t snapshot_id) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetPartitionStatistics( const std::shared_ptr& partition_statistics_file) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemovePartitionStatistics( int64_t snapshot_id) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetProperties( const std::unordered_map& updated) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemoveProperties( const std::vector& removed) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::SetLocation(std::string_view location) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::AddEncryptionKey( std::shared_ptr key) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } TableMetadataBuilder& TableMetadataBuilder::RemoveEncryptionKey(std::string_view key_id) { - return *this; + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -TableMetadataBuilder& TableMetadataBuilder::DiscardChanges() { return *this; } +TableMetadataBuilder& TableMetadataBuilder::DiscardChanges() { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} Result> TableMetadataBuilder::Build() { return NotImplemented("TableMetadataBuilder::Build not implemented"); diff --git a/src/iceberg/table_requirement.cc b/src/iceberg/table_requirement.cc index 26d08591e..38b0cf509 100644 --- a/src/iceberg/table_requirement.cc +++ b/src/iceberg/table_requirement.cc @@ -31,28 +31,31 @@ Status AssertTableUUID::Validate(const TableMetadata* base) const { return NotImplemented("AssertTableUUID::Validate not implemented"); } -Status AssertRefSnapshotID::Validate(const TableMetadata* base) const { - return NotImplemented("AssertRefSnapshotID::Validate not implemented"); +Status AssertTableRefSnapshotID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertTableRefSnapshotID::Validate not implemented"); } -Status AssertLastAssignedFieldId::Validate(const TableMetadata* base) const { - return NotImplemented("AssertLastAssignedFieldId::Validate not implemented"); +Status AssertCurrentTableLastAssignedFieldId::Validate(const TableMetadata* base) const { + return NotImplemented( + "AssertCurrentTableLastAssignedFieldId::Validate not implemented"); } -Status AssertCurrentSchemaID::Validate(const TableMetadata* base) const { - return NotImplemented("AssertCurrentSchemaID::Validate not implemented"); +Status AssertCurrentTableSchemaID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertCurrentTableSchemaID::Validate not implemented"); } -Status AssertLastAssignedPartitionId::Validate(const TableMetadata* base) const { - return NotImplemented("AssertLastAssignedPartitionId::Validate not implemented"); +Status AssertCurrentTableLastAssignedPartitionId::Validate( + const TableMetadata* base) const { + return NotImplemented( + "AssertCurrentTableLastAssignedPartitionId::Validate not implemented"); } -Status AssertDefaultSpecID::Validate(const TableMetadata* base) const { - return NotImplemented("AssertDefaultSpecID::Validate not implemented"); +Status AssertDefaultTableSpecID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertDefaultTableSpecID::Validate not implemented"); } -Status AssertDefaultSortOrderID::Validate(const TableMetadata* base) const { - return NotImplemented("AssertDefaultSortOrderID::Validate not implemented"); +Status AssertDefaultTableSortOrderID::Validate(const TableMetadata* base) const { + return NotImplemented("AssertDefaultTableSortOrderID::Validate not implemented"); } } // namespace iceberg diff --git a/src/iceberg/table_requirement.h b/src/iceberg/table_requirement.h index 948709fde..53be79677 100644 --- a/src/iceberg/table_requirement.h +++ b/src/iceberg/table_requirement.h @@ -33,7 +33,7 @@ #include "iceberg/result.h" #include "iceberg/type_fwd.h" - namespace iceberg { + namespace iceberg { /// \brief Base class for update requirement operations /// diff --git a/src/iceberg/table_requirements.cc b/src/iceberg/table_requirements.cc index ea8f606e5..1eb870cb4 100644 --- a/src/iceberg/table_requirements.cc +++ b/src/iceberg/table_requirements.cc @@ -19,18 +19,18 @@ #include "iceberg/table_requirements.h" +#include "iceberg/exception.h" #include "iceberg/table_metadata.h" #include "iceberg/table_update.h" namespace iceberg { -void MetadataUpdateContext::AddRequirement( - std::unique_ptr requirement) { - requirements_.push_back(std::move(requirement)); +void TableUpdateContext::AddRequirement(std::unique_ptr requirement) { + throw IcebergError("TableUpdateContext::AddRequirement not implemented"); } -Result>> MetadataUpdateContext::Build() { - return std::move(requirements_); +Result>> TableUpdateContext::Build() { + return NotImplemented("TableUpdateContext::Build not implemented"); } Result>> TableRequirements::ForCreateTable( diff --git a/src/iceberg/table_requirements.h b/src/iceberg/table_requirements.h index e92426e32..327e7d53f 100644 --- a/src/iceberg/table_requirements.h +++ b/src/iceberg/table_requirements.h @@ -19,11 +19,11 @@ #pragma once -/// \file iceberg/update_requirements.h -/// Factory for generating update requirements from metadata updates. +/// \file iceberg/table_requirements.h +/// Factory for generating table requirements from metadata updates. /// -/// This utility class generates the appropriate UpdateRequirement instances -/// based on a list of MetadataUpdate operations. The requirements are used +/// This utility class generates the appropriate TableRequirement instances +/// based on a list of TableUpdate operations. The requirements are used /// for optimistic concurrency control when committing table changes. #include @@ -35,26 +35,26 @@ namespace iceberg { -/// \brief Context for generating update requirements +/// \brief Context for generating table requirements /// -/// This context is passed to each MetadataUpdate's GenerateRequirements method +/// This context is passed to each TableUpdate's GenerateRequirements method /// and maintains state about what requirements have already been added to avoid /// duplicates. -class ICEBERG_EXPORT MetadataUpdateContext { +class ICEBERG_EXPORT TableUpdateContext { public: /// \brief Construct a context for requirement generation /// - /// \param base The base table metadata (may be nullptr for table creation) + /// \param base The base table metadata (maybe nullptr for table creation) /// \param is_replace Whether this is a replace operation (more permissive) - MetadataUpdateContext(const TableMetadata* base, bool is_replace) + TableUpdateContext(const TableMetadata* base, bool is_replace) : base_(base), is_replace_(is_replace) {} // Delete copy operations (contains unique_ptr members) - MetadataUpdateContext(const MetadataUpdateContext&) = delete; - MetadataUpdateContext& operator=(const MetadataUpdateContext&) = delete; + TableUpdateContext(const TableUpdateContext&) = delete; + TableUpdateContext& operator=(const TableUpdateContext&) = delete; // Enable move construction only (assignment deleted due to const members) - MetadataUpdateContext(MetadataUpdateContext&&) noexcept = default; + TableUpdateContext(TableUpdateContext&&) noexcept = default; /// \brief Add a requirement to the list void AddRequirement(std::unique_ptr requirement); @@ -75,18 +75,18 @@ class ICEBERG_EXPORT MetadataUpdateContext { std::vector> requirements_; }; -/// \brief Factory class for generating update requirements +/// \brief Factory class for generating table requirements /// -/// This class analyzes a sequence of metadata updates and generates the -/// appropriate update requirements to ensure safe concurrent modifications. +/// This class analyzes a sequence of table updates and generates the +/// appropriate table requirements to ensure safe concurrent modifications. class ICEBERG_EXPORT TableRequirements { public: /// \brief Generate requirements for creating a new table /// /// For table creation, this requires that the table does not already exist. /// - /// \param table_updates The list of metadata updates for table creation - /// \return A list of update requirements to validate before creation + /// \param table_updates The list of table updates for table creation + /// \return A list of table requirements to validate before creation static Result>> ForCreateTable( const std::vector>& table_updates); @@ -96,8 +96,8 @@ class ICEBERG_EXPORT TableRequirements { /// allows more aggressive changes than a regular update. /// /// \param base The base table metadata - /// \param table_updates The list of metadata updates for replacement - /// \return A list of update requirements to validate before replacement + /// \param table_updates The list of table updates for replacement + /// \return A list of table requirements to validate before replacement static Result>> ForReplaceTable( const TableMetadata& base, const std::vector>& table_updates); @@ -108,8 +108,8 @@ class ICEBERG_EXPORT TableRequirements { /// metadata properties haven't changed concurrently. /// /// \param base The base table metadata - /// \param table_updates The list of metadata updates - /// \return A list of update requirements to validate before update + /// \param table_updates The list of table updates + /// \return A list of table requirements to validate before update static Result>> ForUpdateTable( const TableMetadata& base, const std::vector>& table_updates); diff --git a/src/iceberg/table_update.cc b/src/iceberg/table_update.cc index 13f5b30a3..7be6f9440 100644 --- a/src/iceberg/table_update.cc +++ b/src/iceberg/table_update.cc @@ -19,146 +19,184 @@ #include "iceberg/table_update.h" +#include "iceberg/exception.h" #include "iceberg/table_metadata.h" #include "iceberg/table_requirements.h" namespace iceberg { -// AssignUUID +// AssignTableUUID -void AssignUUID::ApplyTo(TableMetadataBuilder& builder) const {} +void AssignTableUUID::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status AssignUUID::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("AssignUUID::GenerateRequirements not implemented"); +Status AssignTableUUID::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("AssignTableUUID::GenerateRequirements not implemented"); } -// UpgradeFormatVersion +// UpgradeTableFormatVersion -void UpgradeFormatVersion::ApplyTo(TableMetadataBuilder& builder) const {} +void UpgradeTableFormatVersion::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status UpgradeFormatVersion::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("UpgradeFormatVersion::GenerateRequirements not implemented"); +Status UpgradeTableFormatVersion::GenerateRequirements( + TableUpdateContext& context) const { + return NotImplemented( + "UpgradeTableFormatVersion::GenerateRequirements not implemented"); } -// AddSchema +// AddTableSchema -void AddSchema::ApplyTo(TableMetadataBuilder& builder) const {} +void AddTableSchema::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status AddSchema::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("AddSchema::GenerateRequirements not implemented"); +Status AddTableSchema::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("AddTableSchema::GenerateRequirements not implemented"); } -// SetCurrentSchema +// SetCurrentTableSchema -void SetCurrentSchema::ApplyTo(TableMetadataBuilder& builder) const {} +void SetCurrentTableSchema::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status SetCurrentSchema::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("SetCurrentSchema::GenerateRequirements not implemented"); +Status SetCurrentTableSchema::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("SetCurrentTableSchema::GenerateRequirements not implemented"); } -// AddPartitionSpec +// AddTablePartitionSpec -void AddPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const {} +void AddTablePartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status AddPartitionSpec::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("AddPartitionSpec::GenerateRequirements not implemented"); +Status AddTablePartitionSpec::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("AddTablePartitionSpec::GenerateRequirements not implemented"); } -// SetDefaultPartitionSpec +// SetDefaultTablePartitionSpec -void SetDefaultPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const {} +void SetDefaultTablePartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status SetDefaultPartitionSpec::GenerateRequirements( - MetadataUpdateContext& context) const { - return NotImplemented("SetDefaultPartitionSpec::GenerateRequirements not implemented"); +Status SetDefaultTablePartitionSpec::GenerateRequirements( + TableUpdateContext& context) const { + return NotImplemented( + "SetDefaultTablePartitionSpec::GenerateRequirements not implemented"); } -// RemovePartitionSpecs +// RemoveTablePartitionSpecs -void RemovePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const {} +void RemoveTablePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status RemovePartitionSpecs::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("RemovePartitionSpecs::GenerateRequirements not implemented"); +Status RemoveTablePartitionSpecs::GenerateRequirements( + TableUpdateContext& context) const { + return NotImplemented( + "RemoveTablePartitionSpecs::GenerateRequirements not implemented"); } -// RemoveSchemas +// RemoveTableSchemas -void RemoveSchemas::ApplyTo(TableMetadataBuilder& builder) const {} +void RemoveTableSchemas::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status RemoveSchemas::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("RemoveSchemas::GenerateRequirements not implemented"); +Status RemoveTableSchemas::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("RemoveTableSchemas::GenerateRequirements not implemented"); } -// AddSortOrder +// AddTableSortOrder -void AddSortOrder::ApplyTo(TableMetadataBuilder& builder) const {} +void AddTableSortOrder::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status AddSortOrder::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("AddSortOrder::GenerateRequirements not implemented"); +Status AddTableSortOrder::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("AddTableSortOrder::GenerateRequirements not implemented"); } -// SetDefaultSortOrder +// SetDefaultTableSortOrder -void SetDefaultSortOrder::ApplyTo(TableMetadataBuilder& builder) const {} +void SetDefaultTableSortOrder::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status SetDefaultSortOrder::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("SetDefaultSortOrder::GenerateRequirements not implemented"); +Status SetDefaultTableSortOrder::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("SetDefaultTableSortOrder::GenerateRequirements not implemented"); } -// AddSnapshot +// AddTableSnapshot -void AddSnapshot::ApplyTo(TableMetadataBuilder& builder) const {} +void AddTableSnapshot::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status AddSnapshot::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("AddSnapshot::GenerateRequirements not implemented"); +Status AddTableSnapshot::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("AddTableSnapshot::GenerateRequirements not implemented"); } -// RemoveSnapshots +// RemoveTableSnapshots -void RemoveSnapshots::ApplyTo(TableMetadataBuilder& builder) const {} +void RemoveTableSnapshots::ApplyTo(TableMetadataBuilder& builder) const {} -Status RemoveSnapshots::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("RemoveSnapshots::GenerateRequirements not implemented"); +Status RemoveTableSnapshots::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("RemoveTableSnapshots::GenerateRequirements not implemented"); } -// RemoveSnapshotRef +// RemoveTableSnapshotRef -void RemoveSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const {} +void RemoveTableSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status RemoveSnapshotRef::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("RemoveSnapshotRef::GenerateRequirements not implemented"); +Status RemoveTableSnapshotRef::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("RemoveTableSnapshotRef::GenerateRequirements not implemented"); } -// SetSnapshotRef +// SetTableSnapshotRef -void SetSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const {} +void SetTableSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status SetSnapshotRef::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("SetSnapshotRef::GenerateRequirements not implemented"); +Status SetTableSnapshotRef::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("SetTableSnapshotRef::GenerateRequirements not implemented"); } -// SetProperties +// SetTableProperties -void SetProperties::ApplyTo(TableMetadataBuilder& builder) const {} +void SetTableProperties::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status SetProperties::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("SetProperties::GenerateRequirements not implemented"); +Status SetTableProperties::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("SetTableProperties::GenerateRequirements not implemented"); } -// RemoveProperties +// RemoveTableProperties -void RemoveProperties::ApplyTo(TableMetadataBuilder& builder) const {} +void RemoveTableProperties::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status RemoveProperties::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("RemoveProperties::GenerateRequirements not implemented"); +Status RemoveTableProperties::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("RemoveTableProperties::GenerateRequirements not implemented"); } -// SetLocation +// SetTableLocation -void SetLocation::ApplyTo(TableMetadataBuilder& builder) const {} +void SetTableLocation::ApplyTo(TableMetadataBuilder& builder) const { + throw IcebergError(std::format("{} not implemented", __FUNCTION__)); +} -Status SetLocation::GenerateRequirements(MetadataUpdateContext& context) const { - return NotImplemented("SetLocation::GenerateRequirements not implemented"); +Status SetTableLocation::GenerateRequirements(TableUpdateContext& context) const { + return NotImplemented("SetTableLocation::GenerateRequirements not implemented"); } } // namespace iceberg diff --git a/src/iceberg/table_update.h b/src/iceberg/table_update.h index 3bfcd3cf5..9b747728a 100644 --- a/src/iceberg/table_update.h +++ b/src/iceberg/table_update.h @@ -59,44 +59,44 @@ class ICEBERG_EXPORT TableUpdate { /// /// \param context The context containing base metadata and operation state /// \return Status indicating success or failure with error details - virtual Status GenerateRequirements(MetadataUpdateContext& context) const = 0; + virtual Status GenerateRequirements(TableUpdateContext& context) const = 0; }; /// \brief Represents an assignment of a UUID to the table -class ICEBERG_EXPORT AssignUUID : public TableUpdate { +class ICEBERG_EXPORT AssignTableUUID : public TableUpdate { public: - explicit AssignUUID(std::string uuid) : uuid_(std::move(uuid)) {} + explicit AssignTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} const std::string& uuid() const { return uuid_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::string uuid_; }; /// \brief Represents an upgrade of the table format version -class ICEBERG_EXPORT UpgradeFormatVersion : public TableUpdate { +class ICEBERG_EXPORT UpgradeTableFormatVersion : public TableUpdate { public: - explicit UpgradeFormatVersion(int8_t format_version) + explicit UpgradeTableFormatVersion(int8_t format_version) : format_version_(format_version) {} int8_t format_version() const { return format_version_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: int8_t format_version_; }; /// \brief Represents adding a new schema to the table -class ICEBERG_EXPORT AddSchema : public TableUpdate { +class ICEBERG_EXPORT AddTableSchema : public TableUpdate { public: - explicit AddSchema(std::shared_ptr schema, int32_t last_column_id) + explicit AddTableSchema(std::shared_ptr schema, int32_t last_column_id) : schema_(std::move(schema)), last_column_id_(last_column_id) {} const std::shared_ptr& schema() const { return schema_; } @@ -105,7 +105,7 @@ class ICEBERG_EXPORT AddSchema : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::shared_ptr schema_; @@ -113,168 +113,170 @@ class ICEBERG_EXPORT AddSchema : public TableUpdate { }; /// \brief Represents setting the current schema -class ICEBERG_EXPORT SetCurrentSchema : public TableUpdate { +class ICEBERG_EXPORT SetCurrentTableSchema : public TableUpdate { public: - explicit SetCurrentSchema(int32_t schema_id) : schema_id_(schema_id) {} + explicit SetCurrentTableSchema(int32_t schema_id) : schema_id_(schema_id) {} int32_t schema_id() const { return schema_id_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: int32_t schema_id_; }; /// \brief Represents adding a new partition spec to the table -class ICEBERG_EXPORT AddPartitionSpec : public TableUpdate { +class ICEBERG_EXPORT AddTablePartitionSpec : public TableUpdate { public: - explicit AddPartitionSpec(std::shared_ptr spec) + explicit AddTablePartitionSpec(std::shared_ptr spec) : spec_(std::move(spec)) {} const std::shared_ptr& spec() const { return spec_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::shared_ptr spec_; }; /// \brief Represents setting the default partition spec -class ICEBERG_EXPORT SetDefaultPartitionSpec : public TableUpdate { +class ICEBERG_EXPORT SetDefaultTablePartitionSpec : public TableUpdate { public: - explicit SetDefaultPartitionSpec(int32_t spec_id) : spec_id_(spec_id) {} + explicit SetDefaultTablePartitionSpec(int32_t spec_id) : spec_id_(spec_id) {} int32_t spec_id() const { return spec_id_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: int32_t spec_id_; }; /// \brief Represents removing partition specs from the table -class ICEBERG_EXPORT RemovePartitionSpecs : public TableUpdate { +class ICEBERG_EXPORT RemoveTablePartitionSpecs : public TableUpdate { public: - explicit RemovePartitionSpecs(std::vector spec_ids) + explicit RemoveTablePartitionSpecs(std::vector spec_ids) : spec_ids_(std::move(spec_ids)) {} const std::vector& spec_ids() const { return spec_ids_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::vector spec_ids_; }; /// \brief Represents removing schemas from the table -class ICEBERG_EXPORT RemoveSchemas : public TableUpdate { +class ICEBERG_EXPORT RemoveTableSchemas : public TableUpdate { public: - explicit RemoveSchemas(std::vector schema_ids) + explicit RemoveTableSchemas(std::vector schema_ids) : schema_ids_(std::move(schema_ids)) {} const std::vector& schema_ids() const { return schema_ids_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::vector schema_ids_; }; /// \brief Represents adding a new sort order to the table -class ICEBERG_EXPORT AddSortOrder : public TableUpdate { +class ICEBERG_EXPORT AddTableSortOrder : public TableUpdate { public: - explicit AddSortOrder(std::shared_ptr sort_order) + explicit AddTableSortOrder(std::shared_ptr sort_order) : sort_order_(std::move(sort_order)) {} const std::shared_ptr& sort_order() const { return sort_order_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::shared_ptr sort_order_; }; /// \brief Represents setting the default sort order -class ICEBERG_EXPORT SetDefaultSortOrder : public TableUpdate { +class ICEBERG_EXPORT SetDefaultTableSortOrder : public TableUpdate { public: - explicit SetDefaultSortOrder(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} + explicit SetDefaultTableSortOrder(int32_t sort_order_id) + : sort_order_id_(sort_order_id) {} int32_t sort_order_id() const { return sort_order_id_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: int32_t sort_order_id_; }; /// \brief Represents adding a snapshot to the table -class ICEBERG_EXPORT AddSnapshot : public TableUpdate { +class ICEBERG_EXPORT AddTableSnapshot : public TableUpdate { public: - explicit AddSnapshot(std::shared_ptr snapshot) + explicit AddTableSnapshot(std::shared_ptr snapshot) : snapshot_(std::move(snapshot)) {} const std::shared_ptr& snapshot() const { return snapshot_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::shared_ptr snapshot_; }; /// \brief Represents removing snapshots from the table -class ICEBERG_EXPORT RemoveSnapshots : public TableUpdate { +class ICEBERG_EXPORT RemoveTableSnapshots : public TableUpdate { public: - explicit RemoveSnapshots(std::vector snapshot_ids) + explicit RemoveTableSnapshots(std::vector snapshot_ids) : snapshot_ids_(std::move(snapshot_ids)) {} const std::vector& snapshot_ids() const { return snapshot_ids_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::vector snapshot_ids_; }; /// \brief Represents removing a snapshot reference -class ICEBERG_EXPORT RemoveSnapshotRef : public TableUpdate { +class ICEBERG_EXPORT RemoveTableSnapshotRef : public TableUpdate { public: - explicit RemoveSnapshotRef(std::string ref_name) : ref_name_(std::move(ref_name)) {} + explicit RemoveTableSnapshotRef(std::string ref_name) + : ref_name_(std::move(ref_name)) {} const std::string& ref_name() const { return ref_name_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::string ref_name_; }; /// \brief Represents setting a snapshot reference -class ICEBERG_EXPORT SetSnapshotRef : public TableUpdate { +class ICEBERG_EXPORT SetTableSnapshotRef : public TableUpdate { public: - SetSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, - std::optional min_snapshots_to_keep = std::nullopt, - std::optional max_snapshot_age_ms = std::nullopt, - std::optional max_ref_age_ms = std::nullopt) + SetTableSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, + std::optional min_snapshots_to_keep = std::nullopt, + std::optional max_snapshot_age_ms = std::nullopt, + std::optional max_ref_age_ms = std::nullopt) : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id), type_(type), @@ -295,7 +297,7 @@ class ICEBERG_EXPORT SetSnapshotRef : public TableUpdate { void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::string ref_name_; @@ -307,47 +309,47 @@ class ICEBERG_EXPORT SetSnapshotRef : public TableUpdate { }; /// \brief Represents setting table properties -class ICEBERG_EXPORT SetProperties : public TableUpdate { +class ICEBERG_EXPORT SetTableProperties : public TableUpdate { public: - explicit SetProperties(std::unordered_map updated) + explicit SetTableProperties(std::unordered_map updated) : updated_(std::move(updated)) {} const std::unordered_map& updated() const { return updated_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::unordered_map updated_; }; /// \brief Represents removing table properties -class ICEBERG_EXPORT RemoveProperties : public TableUpdate { +class ICEBERG_EXPORT RemoveTableProperties : public TableUpdate { public: - explicit RemoveProperties(std::vector removed) + explicit RemoveTableProperties(std::vector removed) : removed_(std::move(removed)) {} const std::vector& removed() const { return removed_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::vector removed_; }; /// \brief Represents setting the table location -class ICEBERG_EXPORT SetLocation : public TableUpdate { +class ICEBERG_EXPORT SetTableLocation : public TableUpdate { public: - explicit SetLocation(std::string location) : location_(std::move(location)) {} + explicit SetTableLocation(std::string location) : location_(std::move(location)) {} const std::string& location() const { return location_; } void ApplyTo(TableMetadataBuilder& builder) const override; - Status GenerateRequirements(MetadataUpdateContext& context) const override; + Status GenerateRequirements(TableUpdateContext& context) const override; private: std::string location_; diff --git a/src/iceberg/type_fwd.h b/src/iceberg/type_fwd.h index 652ef8c50..3bd067d0c 100644 --- a/src/iceberg/type_fwd.h +++ b/src/iceberg/type_fwd.h @@ -145,7 +145,7 @@ class MapLike; class TableUpdate; class TableRequirement; class TableMetadataBuilder; -class MetadataUpdateContext; +class TableUpdateContext; /// ---------------------------------------------------------------------------- /// TODO: Forward declarations below are not added yet. diff --git a/src/iceberg/util/string_util.h b/src/iceberg/util/string_util.h index 499a62ddd..a22aa7a5d 100644 --- a/src/iceberg/util/string_util.h +++ b/src/iceberg/util/string_util.h @@ -45,9 +45,9 @@ class ICEBERG_EXPORT StringUtils { return input; } - static bool EqualsIgnoreCase(const std::string& a, const std::string& b) { + static bool EqualsIgnoreCase(const std::string& lhs, const std::string& rhs) { return std::ranges::equal( - a, b, [](char ca, char cb) { return std::tolower(ca) == std::tolower(cb); }); + lhs, rhs, [](char lc, char rc) { return std::tolower(lc) == std::tolower(rc); }); } }; From 4439f8d90d9b3005cbcb1e8d41a44e76c8d2eaa5 Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Fri, 17 Oct 2025 11:27:48 +0800 Subject: [PATCH 10/14] fix conflict --- src/iceberg/table_metadata.h | 1 - src/iceberg/table_requirement.h | 232 ++++++++++++++++---------------- 2 files changed, 116 insertions(+), 117 deletions(-) diff --git a/src/iceberg/table_metadata.h b/src/iceberg/table_metadata.h index 8a3015d21..6f7a81904 100644 --- a/src/iceberg/table_metadata.h +++ b/src/iceberg/table_metadata.h @@ -29,7 +29,6 @@ #include #include "iceberg/iceberg_export.h" -#include "iceberg/metadata_update.h" #include "iceberg/type_fwd.h" #include "iceberg/util/timepoint.h" diff --git a/src/iceberg/table_requirement.h b/src/iceberg/table_requirement.h index 53be79677..4bd062570 100644 --- a/src/iceberg/table_requirement.h +++ b/src/iceberg/table_requirement.h @@ -33,153 +33,153 @@ #include "iceberg/result.h" #include "iceberg/type_fwd.h" - namespace iceberg { +namespace iceberg { - /// \brief Base class for update requirement operations - /// - /// Represents a requirement that must be validated before applying - /// metadata updates to a table. Each concrete subclass represents - /// a specific type of requirement check. - class ICEBERG_EXPORT TableRequirement { - public: - virtual ~TableRequirement() = default; - - /// \brief Validate this requirement against table metadata - /// - /// \param base The base table metadata to validate against (may be nullptr) - /// \return Status indicating success or failure with error details - virtual Status Validate(const TableMetadata* base) const = 0; - }; - - /// \brief Requirement that the table does not exist +/// \brief Base class for update requirement operations +/// +/// Represents a requirement that must be validated before applying +/// metadata updates to a table. Each concrete subclass represents +/// a specific type of requirement check. +class ICEBERG_EXPORT TableRequirement { + public: + virtual ~TableRequirement() = default; + + /// \brief Validate this requirement against table metadata /// - /// This requirement is used when creating a new table to ensure - /// it doesn't already exist. - class ICEBERG_EXPORT AssertTableDoesNotExist : public TableRequirement { - public: - AssertTableDoesNotExist() = default; + /// \param base The base table metadata to validate against (may be nullptr) + /// \return Status indicating success or failure with error details + virtual Status Validate(const TableMetadata* base) const = 0; +}; - Status Validate(const TableMetadata* base) const override; - }; +/// \brief Requirement that the table does not exist +/// +/// This requirement is used when creating a new table to ensure +/// it doesn't already exist. +class ICEBERG_EXPORT AssertTableDoesNotExist : public TableRequirement { + public: + AssertTableDoesNotExist() = default; - /// \brief Requirement that the table UUID matches the expected value - /// - /// This ensures the table hasn't been replaced or recreated between - /// reading the metadata and attempting to update it. - class ICEBERG_EXPORT AssertTableUUID : public TableRequirement { - public: - explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} + Status Validate(const TableMetadata* base) const override; +}; - const std::string& uuid() const { return uuid_; } +/// \brief Requirement that the table UUID matches the expected value +/// +/// This ensures the table hasn't been replaced or recreated between +/// reading the metadata and attempting to update it. +class ICEBERG_EXPORT AssertTableUUID : public TableRequirement { + public: + explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} - Status Validate(const TableMetadata* base) const override; + const std::string& uuid() const { return uuid_; } - private: - std::string uuid_; - }; + Status Validate(const TableMetadata* base) const override; - /// \brief Requirement that a reference (branch or tag) points to a specific snapshot - /// - /// This requirement validates that a named reference (branch or tag) either: - /// - Points to the expected snapshot ID - /// - Does not exist (if snapshot_id is nullopt) - class ICEBERG_EXPORT AssertRefSnapshotID : public TableRequirement { - public: - AssertRefSnapshotID(std::string ref_name, std::optional snapshot_id) - : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} + private: + std::string uuid_; +}; + +/// \brief Requirement that a reference (branch or tag) points to a specific snapshot +/// +/// This requirement validates that a named reference (branch or tag) either: +/// - Points to the expected snapshot ID +/// - Does not exist (if snapshot_id is nullopt) +class ICEBERG_EXPORT AssertTableRefSnapshotID : public TableRequirement { + public: + AssertTableRefSnapshotID(std::string ref_name, std::optional snapshot_id) + : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} - const std::string& ref_name() const { return ref_name_; } + const std::string& ref_name() const { return ref_name_; } - const std::optional& snapshot_id() const { return snapshot_id_; } + const std::optional& snapshot_id() const { return snapshot_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - std::string ref_name_; - std::optional snapshot_id_; - }; + private: + std::string ref_name_; + std::optional snapshot_id_; +}; - /// \brief Requirement that the last assigned field ID matches - /// - /// This ensures the schema hasn't been modified (by adding fields) - /// since the metadata was read. - class ICEBERG_EXPORT AssertLastAssignedFieldId : public TableRequirement { - public: - explicit AssertLastAssignedFieldId(int32_t last_assigned_field_id) - : last_assigned_field_id_(last_assigned_field_id) {} +/// \brief Requirement that the last assigned field ID matches +/// +/// This ensures the schema hasn't been modified (by adding fields) +/// since the metadata was read. +class ICEBERG_EXPORT AssertCurrentTableLastAssignedFieldId : public TableRequirement { + public: + explicit AssertCurrentTableLastAssignedFieldId(int32_t last_assigned_field_id) + : last_assigned_field_id_(last_assigned_field_id) {} - int32_t last_assigned_field_id() const { return last_assigned_field_id_; } + int32_t last_assigned_field_id() const { return last_assigned_field_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t last_assigned_field_id_; - }; + private: + int32_t last_assigned_field_id_; +}; - /// \brief Requirement that the current schema ID matches - /// - /// This ensures the active schema hasn't changed since the - /// metadata was read. - class ICEBERG_EXPORT AssertCurrentSchemaID : public TableRequirement { - public: - explicit AssertCurrentSchemaID(int32_t schema_id) : schema_id_(schema_id) {} +/// \brief Requirement that the current schema ID matches +/// +/// This ensures the active schema hasn't changed since the +/// metadata was read. +class ICEBERG_EXPORT AssertCurrentTableSchemaID : public TableRequirement { + public: + explicit AssertCurrentTableSchemaID(int32_t schema_id) : schema_id_(schema_id) {} - int32_t schema_id() const { return schema_id_; } + int32_t schema_id() const { return schema_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t schema_id_; - }; + private: + int32_t schema_id_; +}; - /// \brief Requirement that the last assigned partition ID matches - /// - /// This ensures partition specs haven't been modified since the - /// metadata was read. - class ICEBERG_EXPORT AssertLastAssignedPartitionId : public TableRequirement { - public: - explicit AssertLastAssignedPartitionId(int32_t last_assigned_partition_id) - : last_assigned_partition_id_(last_assigned_partition_id) {} +/// \brief Requirement that the last assigned partition ID matches +/// +/// This ensures partition specs haven't been modified since the +/// metadata was read. +class ICEBERG_EXPORT AssertCurrentTableLastAssignedPartitionId : public TableRequirement { + public: + explicit AssertCurrentTableLastAssignedPartitionId(int32_t last_assigned_partition_id) + : last_assigned_partition_id_(last_assigned_partition_id) {} - int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } + int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t last_assigned_partition_id_; - }; + private: + int32_t last_assigned_partition_id_; +}; - /// \brief Requirement that the default partition spec ID matches - /// - /// This ensures the default partition spec hasn't changed since - /// the metadata was read. - class ICEBERG_EXPORT AssertDefaultSpecID : public TableRequirement { - public: - explicit AssertDefaultSpecID(int32_t spec_id) : spec_id_(spec_id) {} +/// \brief Requirement that the default partition spec ID matches +/// +/// This ensures the default partition spec hasn't changed since +/// the metadata was read. +class ICEBERG_EXPORT AssertDefaultTableSpecID : public TableRequirement { + public: + explicit AssertDefaultTableSpecID(int32_t spec_id) : spec_id_(spec_id) {} - int32_t spec_id() const { return spec_id_; } + int32_t spec_id() const { return spec_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t spec_id_; - }; + private: + int32_t spec_id_; +}; - /// \brief Requirement that the default sort order ID matches - /// - /// This ensures the default sort order hasn't changed since - /// the metadata was read. - class ICEBERG_EXPORT AssertDefaultSortOrderID : public TableRequirement { - public: - explicit AssertDefaultSortOrderID(int32_t sort_order_id) - : sort_order_id_(sort_order_id) {} +/// \brief Requirement that the default sort order ID matches +/// +/// This ensures the default sort order hasn't changed since +/// the metadata was read. +class ICEBERG_EXPORT AssertDefaultTableSortOrderID : public TableRequirement { + public: + explicit AssertDefaultTableSortOrderID(int32_t sort_order_id) + : sort_order_id_(sort_order_id) {} - int32_t sort_order_id() const { return sort_order_id_; } + int32_t sort_order_id() const { return sort_order_id_; } - Status Validate(const TableMetadata* base) const override; + Status Validate(const TableMetadata* base) const override; - private: - int32_t sort_order_id_; - }; + private: + int32_t sort_order_id_; +}; } // namespace iceberg From 35c22e27793b6218ef6cd26de7d11c0010ca5e22 Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Fri, 17 Oct 2025 11:40:42 +0800 Subject: [PATCH 11/14] fix conflict --- src/iceberg/table_metadata.cc | 1 - src/iceberg/update_requirement.cc | 148 ------------------------------ 2 files changed, 149 deletions(-) delete mode 100644 src/iceberg/update_requirement.cc diff --git a/src/iceberg/table_metadata.cc b/src/iceberg/table_metadata.cc index 9c9653bb4..e32e75ee1 100644 --- a/src/iceberg/table_metadata.cc +++ b/src/iceberg/table_metadata.cc @@ -25,7 +25,6 @@ #include -#include "../../build/_deps/fmt-src/include/fmt/color.h" #include "iceberg/exception.h" #include "iceberg/file_io.h" #include "iceberg/json_internal.h" diff --git a/src/iceberg/update_requirement.cc b/src/iceberg/update_requirement.cc deleted file mode 100644 index 25451f312..000000000 --- a/src/iceberg/update_requirement.cc +++ /dev/null @@ -1,148 +0,0 @@ -/* - * 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. - */ - -#include "iceberg/update_requirement.h" - -#include "iceberg/table_metadata.h" -#include "iceberg/util/string_util.h" - -namespace iceberg { - -Status AssertTableDoesNotExist::Validate(const TableMetadata* base) const { - if (base != nullptr) { - return CommitFailed("Requirement failed: table already exists"); - } - return {}; -} - -Status AssertTableUUID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - if (uuid_.empty()) { - return CommitFailed("Requirement failed: expected non-empty UUID"); - } - - if (!StringUtils::EqualsIgnoreCase(uuid_, base->table_uuid)) { - return CommitFailed("Requirement failed: UUID does not match: expected {} != {}", - base->table_uuid, uuid_); - } - - return {}; -} - -Status AssertRefSnapshotID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - // Find the reference in the table metadata - auto it = base->refs.find(ref_name_); - - if (it != base->refs.end()) { - // Reference exists - const auto& ref = it->second; - std::string type = (ref->type() == SnapshotRefType::kBranch) ? "branch" : "tag"; - - if (!snapshot_id_.has_value()) { - // A null snapshot ID means the ref should not exist already - return CommitFailed("Requirement failed: {} {} was created concurrently", type, - ref_name_); - } - if (snapshot_id_.value() != ref->snapshot_id) { - return CommitFailed("Requirement failed: {} {} has changed: expected id {} != {}", - type, ref_name_, snapshot_id_.value(), ref->snapshot_id); - } - } else { - // Reference does not exist - if (snapshot_id_.has_value()) { - return CommitFailed("Requirement failed: branch or tag {} is missing, expected {}", - ref_name_, snapshot_id_.value()); - } - } - - return {}; -} - -Status AssertLastAssignedFieldId::Validate(const TableMetadata* base) const { - if (base != nullptr && base->last_column_id != last_assigned_field_id_) { - return CommitFailed( - "Requirement failed: last assigned field id changed: expected id {} != {}", - last_assigned_field_id_, base->last_column_id); - } - return {}; -} - -Status AssertCurrentSchemaID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - if (!base->current_schema_id.has_value()) { - return CommitFailed("Requirement failed: table has no current schema"); - } - - if (schema_id_ != base->current_schema_id.value()) { - return CommitFailed( - "Requirement failed: current schema changed: expected id {} != {}", schema_id_, - base->current_schema_id.value()); - } - - return {}; -} - -Status AssertLastAssignedPartitionId::Validate(const TableMetadata* base) const { - if (base != nullptr && base->last_partition_id != last_assigned_partition_id_) { - return CommitFailed( - "Requirement failed: last assigned partition id changed: expected id {} != {}", - last_assigned_partition_id_, base->last_partition_id); - } - return {}; -} - -Status AssertDefaultSpecID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - if (spec_id_ != base->default_spec_id) { - return CommitFailed( - "Requirement failed: default partition spec changed: expected id {} != {}", - spec_id_, base->default_spec_id); - } - - return {}; -} - -Status AssertDefaultSortOrderID::Validate(const TableMetadata* base) const { - if (base == nullptr) { - return CommitFailed("Requirement failed: table does not exist"); - } - - if (sort_order_id_ != base->default_sort_order_id) { - return CommitFailed( - "Requirement failed: default sort order changed: expected id {} != {}", - sort_order_id_, base->default_sort_order_id); - } - - return {}; -} - -} // namespace iceberg From 718b62fb6baba9b0769f031696419d5290e73bec Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Fri, 17 Oct 2025 16:10:22 +0800 Subject: [PATCH 12/14] move subclass of TableUpdate/TableRequirement to sub namespace --- src/iceberg/table_requirement.cc | 20 +++--- src/iceberg/table_requirement.h | 36 ++++++----- src/iceberg/table_update.cc | 106 ++++++++++++++++--------------- src/iceberg/table_update.h | 72 +++++++++++---------- 4 files changed, 125 insertions(+), 109 deletions(-) diff --git a/src/iceberg/table_requirement.cc b/src/iceberg/table_requirement.cc index 38b0cf509..6c90efbe7 100644 --- a/src/iceberg/table_requirement.cc +++ b/src/iceberg/table_requirement.cc @@ -23,39 +23,43 @@ namespace iceberg { -Status AssertTableDoesNotExist::Validate(const TableMetadata* base) const { +namespace table { + +Status AssertDoesNotExist::Validate(const TableMetadata* base) const { return NotImplemented("AssertTableDoesNotExist::Validate not implemented"); } -Status AssertTableUUID::Validate(const TableMetadata* base) const { +Status AssertUUID::Validate(const TableMetadata* base) const { return NotImplemented("AssertTableUUID::Validate not implemented"); } -Status AssertTableRefSnapshotID::Validate(const TableMetadata* base) const { +Status AssertRefSnapshotID::Validate(const TableMetadata* base) const { return NotImplemented("AssertTableRefSnapshotID::Validate not implemented"); } -Status AssertCurrentTableLastAssignedFieldId::Validate(const TableMetadata* base) const { +Status AssertLastAssignedFieldId::Validate(const TableMetadata* base) const { return NotImplemented( "AssertCurrentTableLastAssignedFieldId::Validate not implemented"); } -Status AssertCurrentTableSchemaID::Validate(const TableMetadata* base) const { +Status AssertCurrentSchemaID::Validate(const TableMetadata* base) const { return NotImplemented("AssertCurrentTableSchemaID::Validate not implemented"); } -Status AssertCurrentTableLastAssignedPartitionId::Validate( +Status AssertLastAssignedPartitionId::Validate( const TableMetadata* base) const { return NotImplemented( "AssertCurrentTableLastAssignedPartitionId::Validate not implemented"); } -Status AssertDefaultTableSpecID::Validate(const TableMetadata* base) const { +Status AssertDefaultSpecID::Validate(const TableMetadata* base) const { return NotImplemented("AssertDefaultTableSpecID::Validate not implemented"); } -Status AssertDefaultTableSortOrderID::Validate(const TableMetadata* base) const { +Status AssertDefaultSortOrderID::Validate(const TableMetadata* base) const { return NotImplemented("AssertDefaultTableSortOrderID::Validate not implemented"); } +} // namespace table + } // namespace iceberg diff --git a/src/iceberg/table_requirement.h b/src/iceberg/table_requirement.h index 4bd062570..c054532ac 100644 --- a/src/iceberg/table_requirement.h +++ b/src/iceberg/table_requirement.h @@ -51,13 +51,15 @@ class ICEBERG_EXPORT TableRequirement { virtual Status Validate(const TableMetadata* base) const = 0; }; +namespace table { + /// \brief Requirement that the table does not exist /// /// This requirement is used when creating a new table to ensure /// it doesn't already exist. -class ICEBERG_EXPORT AssertTableDoesNotExist : public TableRequirement { +class ICEBERG_EXPORT AssertDoesNotExist : public TableRequirement { public: - AssertTableDoesNotExist() = default; + AssertDoesNotExist() = default; Status Validate(const TableMetadata* base) const override; }; @@ -66,9 +68,9 @@ class ICEBERG_EXPORT AssertTableDoesNotExist : public TableRequirement { /// /// This ensures the table hasn't been replaced or recreated between /// reading the metadata and attempting to update it. -class ICEBERG_EXPORT AssertTableUUID : public TableRequirement { +class ICEBERG_EXPORT AssertUUID : public TableRequirement { public: - explicit AssertTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} + explicit AssertUUID(std::string uuid) : uuid_(std::move(uuid)) {} const std::string& uuid() const { return uuid_; } @@ -83,9 +85,9 @@ class ICEBERG_EXPORT AssertTableUUID : public TableRequirement { /// This requirement validates that a named reference (branch or tag) either: /// - Points to the expected snapshot ID /// - Does not exist (if snapshot_id is nullopt) -class ICEBERG_EXPORT AssertTableRefSnapshotID : public TableRequirement { +class ICEBERG_EXPORT AssertRefSnapshotID : public TableRequirement { public: - AssertTableRefSnapshotID(std::string ref_name, std::optional snapshot_id) + AssertRefSnapshotID(std::string ref_name, std::optional snapshot_id) : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id) {} const std::string& ref_name() const { return ref_name_; } @@ -103,9 +105,9 @@ class ICEBERG_EXPORT AssertTableRefSnapshotID : public TableRequirement { /// /// This ensures the schema hasn't been modified (by adding fields) /// since the metadata was read. -class ICEBERG_EXPORT AssertCurrentTableLastAssignedFieldId : public TableRequirement { +class ICEBERG_EXPORT AssertLastAssignedFieldId : public TableRequirement { public: - explicit AssertCurrentTableLastAssignedFieldId(int32_t last_assigned_field_id) + explicit AssertLastAssignedFieldId(int32_t last_assigned_field_id) : last_assigned_field_id_(last_assigned_field_id) {} int32_t last_assigned_field_id() const { return last_assigned_field_id_; } @@ -120,9 +122,9 @@ class ICEBERG_EXPORT AssertCurrentTableLastAssignedFieldId : public TableRequire /// /// This ensures the active schema hasn't changed since the /// metadata was read. -class ICEBERG_EXPORT AssertCurrentTableSchemaID : public TableRequirement { +class ICEBERG_EXPORT AssertCurrentSchemaID : public TableRequirement { public: - explicit AssertCurrentTableSchemaID(int32_t schema_id) : schema_id_(schema_id) {} + explicit AssertCurrentSchemaID(int32_t schema_id) : schema_id_(schema_id) {} int32_t schema_id() const { return schema_id_; } @@ -136,9 +138,9 @@ class ICEBERG_EXPORT AssertCurrentTableSchemaID : public TableRequirement { /// /// This ensures partition specs haven't been modified since the /// metadata was read. -class ICEBERG_EXPORT AssertCurrentTableLastAssignedPartitionId : public TableRequirement { +class ICEBERG_EXPORT AssertLastAssignedPartitionId : public TableRequirement { public: - explicit AssertCurrentTableLastAssignedPartitionId(int32_t last_assigned_partition_id) + explicit AssertLastAssignedPartitionId(int32_t last_assigned_partition_id) : last_assigned_partition_id_(last_assigned_partition_id) {} int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } @@ -153,9 +155,9 @@ class ICEBERG_EXPORT AssertCurrentTableLastAssignedPartitionId : public TableReq /// /// This ensures the default partition spec hasn't changed since /// the metadata was read. -class ICEBERG_EXPORT AssertDefaultTableSpecID : public TableRequirement { +class ICEBERG_EXPORT AssertDefaultSpecID : public TableRequirement { public: - explicit AssertDefaultTableSpecID(int32_t spec_id) : spec_id_(spec_id) {} + explicit AssertDefaultSpecID(int32_t spec_id) : spec_id_(spec_id) {} int32_t spec_id() const { return spec_id_; } @@ -169,9 +171,9 @@ class ICEBERG_EXPORT AssertDefaultTableSpecID : public TableRequirement { /// /// This ensures the default sort order hasn't changed since /// the metadata was read. -class ICEBERG_EXPORT AssertDefaultTableSortOrderID : public TableRequirement { +class ICEBERG_EXPORT AssertDefaultSortOrderID : public TableRequirement { public: - explicit AssertDefaultTableSortOrderID(int32_t sort_order_id) + explicit AssertDefaultSortOrderID(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} int32_t sort_order_id() const { return sort_order_id_; } @@ -182,4 +184,6 @@ class ICEBERG_EXPORT AssertDefaultTableSortOrderID : public TableRequirement { int32_t sort_order_id_; }; +} // namespace table + } // namespace iceberg diff --git a/src/iceberg/table_update.cc b/src/iceberg/table_update.cc index 7be6f9440..563ada6f5 100644 --- a/src/iceberg/table_update.cc +++ b/src/iceberg/table_update.cc @@ -25,178 +25,182 @@ namespace iceberg { -// AssignTableUUID +namespace table { -void AssignTableUUID::ApplyTo(TableMetadataBuilder& builder) const { +// AssignUUID + +void AssignUUID::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status AssignTableUUID::GenerateRequirements(TableUpdateContext& context) const { +Status AssignUUID::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("AssignTableUUID::GenerateRequirements not implemented"); } -// UpgradeTableFormatVersion +// UpgradeFormatVersion -void UpgradeTableFormatVersion::ApplyTo(TableMetadataBuilder& builder) const { +void UpgradeFormatVersion::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status UpgradeTableFormatVersion::GenerateRequirements( +Status UpgradeFormatVersion::GenerateRequirements( TableUpdateContext& context) const { return NotImplemented( "UpgradeTableFormatVersion::GenerateRequirements not implemented"); } -// AddTableSchema +// AddSchema -void AddTableSchema::ApplyTo(TableMetadataBuilder& builder) const { +void AddSchema::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status AddTableSchema::GenerateRequirements(TableUpdateContext& context) const { +Status AddSchema::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("AddTableSchema::GenerateRequirements not implemented"); } -// SetCurrentTableSchema +// SetCurrentSchema -void SetCurrentTableSchema::ApplyTo(TableMetadataBuilder& builder) const { +void SetCurrentSchema::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status SetCurrentTableSchema::GenerateRequirements(TableUpdateContext& context) const { +Status SetCurrentSchema::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("SetCurrentTableSchema::GenerateRequirements not implemented"); } -// AddTablePartitionSpec +// AddPartitionSpec -void AddTablePartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { +void AddPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status AddTablePartitionSpec::GenerateRequirements(TableUpdateContext& context) const { +Status AddPartitionSpec::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("AddTablePartitionSpec::GenerateRequirements not implemented"); } -// SetDefaultTablePartitionSpec +// SetDefaultPartitionSpec -void SetDefaultTablePartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { +void SetDefaultPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status SetDefaultTablePartitionSpec::GenerateRequirements( +Status SetDefaultPartitionSpec::GenerateRequirements( TableUpdateContext& context) const { return NotImplemented( "SetDefaultTablePartitionSpec::GenerateRequirements not implemented"); } -// RemoveTablePartitionSpecs +// RemovePartitionSpecs -void RemoveTablePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const { +void RemovePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status RemoveTablePartitionSpecs::GenerateRequirements( +Status RemovePartitionSpecs::GenerateRequirements( TableUpdateContext& context) const { return NotImplemented( "RemoveTablePartitionSpecs::GenerateRequirements not implemented"); } -// RemoveTableSchemas +// RemoveSchemas -void RemoveTableSchemas::ApplyTo(TableMetadataBuilder& builder) const { +void RemoveSchemas::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status RemoveTableSchemas::GenerateRequirements(TableUpdateContext& context) const { +Status RemoveSchemas::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("RemoveTableSchemas::GenerateRequirements not implemented"); } -// AddTableSortOrder +// AddSortOrder -void AddTableSortOrder::ApplyTo(TableMetadataBuilder& builder) const { +void AddSortOrder::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status AddTableSortOrder::GenerateRequirements(TableUpdateContext& context) const { +Status AddSortOrder::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("AddTableSortOrder::GenerateRequirements not implemented"); } -// SetDefaultTableSortOrder +// SetDefaultSortOrder -void SetDefaultTableSortOrder::ApplyTo(TableMetadataBuilder& builder) const { +void SetDefaultSortOrder::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status SetDefaultTableSortOrder::GenerateRequirements(TableUpdateContext& context) const { +Status SetDefaultSortOrder::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("SetDefaultTableSortOrder::GenerateRequirements not implemented"); } -// AddTableSnapshot +// AddSnapshot -void AddTableSnapshot::ApplyTo(TableMetadataBuilder& builder) const { +void AddSnapshot::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status AddTableSnapshot::GenerateRequirements(TableUpdateContext& context) const { +Status AddSnapshot::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("AddTableSnapshot::GenerateRequirements not implemented"); } -// RemoveTableSnapshots +// RemoveSnapshots -void RemoveTableSnapshots::ApplyTo(TableMetadataBuilder& builder) const {} +void RemoveSnapshots::ApplyTo(TableMetadataBuilder& builder) const {} -Status RemoveTableSnapshots::GenerateRequirements(TableUpdateContext& context) const { +Status RemoveSnapshots::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("RemoveTableSnapshots::GenerateRequirements not implemented"); } -// RemoveTableSnapshotRef +// RemoveSnapshotRef -void RemoveTableSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { +void RemoveSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status RemoveTableSnapshotRef::GenerateRequirements(TableUpdateContext& context) const { +Status RemoveSnapshotRef::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("RemoveTableSnapshotRef::GenerateRequirements not implemented"); } -// SetTableSnapshotRef +// SetSnapshotRef -void SetTableSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { +void SetSnapshotRef::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status SetTableSnapshotRef::GenerateRequirements(TableUpdateContext& context) const { +Status SetSnapshotRef::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("SetTableSnapshotRef::GenerateRequirements not implemented"); } -// SetTableProperties +// SetProperties -void SetTableProperties::ApplyTo(TableMetadataBuilder& builder) const { +void SetProperties::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status SetTableProperties::GenerateRequirements(TableUpdateContext& context) const { +Status SetProperties::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("SetTableProperties::GenerateRequirements not implemented"); } -// RemoveTableProperties +// RemoveProperties -void RemoveTableProperties::ApplyTo(TableMetadataBuilder& builder) const { +void RemoveProperties::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status RemoveTableProperties::GenerateRequirements(TableUpdateContext& context) const { +Status RemoveProperties::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("RemoveTableProperties::GenerateRequirements not implemented"); } -// SetTableLocation +// SetLocation -void SetTableLocation::ApplyTo(TableMetadataBuilder& builder) const { +void SetLocation::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status SetTableLocation::GenerateRequirements(TableUpdateContext& context) const { +Status SetLocation::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("SetTableLocation::GenerateRequirements not implemented"); } +} // namespace table + } // namespace iceberg diff --git a/src/iceberg/table_update.h b/src/iceberg/table_update.h index 9b747728a..d9d189606 100644 --- a/src/iceberg/table_update.h +++ b/src/iceberg/table_update.h @@ -62,10 +62,12 @@ class ICEBERG_EXPORT TableUpdate { virtual Status GenerateRequirements(TableUpdateContext& context) const = 0; }; +namespace table { + /// \brief Represents an assignment of a UUID to the table -class ICEBERG_EXPORT AssignTableUUID : public TableUpdate { +class ICEBERG_EXPORT AssignUUID : public TableUpdate { public: - explicit AssignTableUUID(std::string uuid) : uuid_(std::move(uuid)) {} + explicit AssignUUID(std::string uuid) : uuid_(std::move(uuid)) {} const std::string& uuid() const { return uuid_; } @@ -78,9 +80,9 @@ class ICEBERG_EXPORT AssignTableUUID : public TableUpdate { }; /// \brief Represents an upgrade of the table format version -class ICEBERG_EXPORT UpgradeTableFormatVersion : public TableUpdate { +class ICEBERG_EXPORT UpgradeFormatVersion : public TableUpdate { public: - explicit UpgradeTableFormatVersion(int8_t format_version) + explicit UpgradeFormatVersion(int8_t format_version) : format_version_(format_version) {} int8_t format_version() const { return format_version_; } @@ -94,9 +96,9 @@ class ICEBERG_EXPORT UpgradeTableFormatVersion : public TableUpdate { }; /// \brief Represents adding a new schema to the table -class ICEBERG_EXPORT AddTableSchema : public TableUpdate { +class ICEBERG_EXPORT AddSchema : public TableUpdate { public: - explicit AddTableSchema(std::shared_ptr schema, int32_t last_column_id) + explicit AddSchema(std::shared_ptr schema, int32_t last_column_id) : schema_(std::move(schema)), last_column_id_(last_column_id) {} const std::shared_ptr& schema() const { return schema_; } @@ -113,9 +115,9 @@ class ICEBERG_EXPORT AddTableSchema : public TableUpdate { }; /// \brief Represents setting the current schema -class ICEBERG_EXPORT SetCurrentTableSchema : public TableUpdate { +class ICEBERG_EXPORT SetCurrentSchema : public TableUpdate { public: - explicit SetCurrentTableSchema(int32_t schema_id) : schema_id_(schema_id) {} + explicit SetCurrentSchema(int32_t schema_id) : schema_id_(schema_id) {} int32_t schema_id() const { return schema_id_; } @@ -128,9 +130,9 @@ class ICEBERG_EXPORT SetCurrentTableSchema : public TableUpdate { }; /// \brief Represents adding a new partition spec to the table -class ICEBERG_EXPORT AddTablePartitionSpec : public TableUpdate { +class ICEBERG_EXPORT AddPartitionSpec : public TableUpdate { public: - explicit AddTablePartitionSpec(std::shared_ptr spec) + explicit AddPartitionSpec(std::shared_ptr spec) : spec_(std::move(spec)) {} const std::shared_ptr& spec() const { return spec_; } @@ -144,9 +146,9 @@ class ICEBERG_EXPORT AddTablePartitionSpec : public TableUpdate { }; /// \brief Represents setting the default partition spec -class ICEBERG_EXPORT SetDefaultTablePartitionSpec : public TableUpdate { +class ICEBERG_EXPORT SetDefaultPartitionSpec : public TableUpdate { public: - explicit SetDefaultTablePartitionSpec(int32_t spec_id) : spec_id_(spec_id) {} + explicit SetDefaultPartitionSpec(int32_t spec_id) : spec_id_(spec_id) {} int32_t spec_id() const { return spec_id_; } @@ -159,9 +161,9 @@ class ICEBERG_EXPORT SetDefaultTablePartitionSpec : public TableUpdate { }; /// \brief Represents removing partition specs from the table -class ICEBERG_EXPORT RemoveTablePartitionSpecs : public TableUpdate { +class ICEBERG_EXPORT RemovePartitionSpecs : public TableUpdate { public: - explicit RemoveTablePartitionSpecs(std::vector spec_ids) + explicit RemovePartitionSpecs(std::vector spec_ids) : spec_ids_(std::move(spec_ids)) {} const std::vector& spec_ids() const { return spec_ids_; } @@ -175,9 +177,9 @@ class ICEBERG_EXPORT RemoveTablePartitionSpecs : public TableUpdate { }; /// \brief Represents removing schemas from the table -class ICEBERG_EXPORT RemoveTableSchemas : public TableUpdate { +class ICEBERG_EXPORT RemoveSchemas : public TableUpdate { public: - explicit RemoveTableSchemas(std::vector schema_ids) + explicit RemoveSchemas(std::vector schema_ids) : schema_ids_(std::move(schema_ids)) {} const std::vector& schema_ids() const { return schema_ids_; } @@ -191,9 +193,9 @@ class ICEBERG_EXPORT RemoveTableSchemas : public TableUpdate { }; /// \brief Represents adding a new sort order to the table -class ICEBERG_EXPORT AddTableSortOrder : public TableUpdate { +class ICEBERG_EXPORT AddSortOrder : public TableUpdate { public: - explicit AddTableSortOrder(std::shared_ptr sort_order) + explicit AddSortOrder(std::shared_ptr sort_order) : sort_order_(std::move(sort_order)) {} const std::shared_ptr& sort_order() const { return sort_order_; } @@ -207,9 +209,9 @@ class ICEBERG_EXPORT AddTableSortOrder : public TableUpdate { }; /// \brief Represents setting the default sort order -class ICEBERG_EXPORT SetDefaultTableSortOrder : public TableUpdate { +class ICEBERG_EXPORT SetDefaultSortOrder : public TableUpdate { public: - explicit SetDefaultTableSortOrder(int32_t sort_order_id) + explicit SetDefaultSortOrder(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} int32_t sort_order_id() const { return sort_order_id_; } @@ -223,9 +225,9 @@ class ICEBERG_EXPORT SetDefaultTableSortOrder : public TableUpdate { }; /// \brief Represents adding a snapshot to the table -class ICEBERG_EXPORT AddTableSnapshot : public TableUpdate { +class ICEBERG_EXPORT AddSnapshot : public TableUpdate { public: - explicit AddTableSnapshot(std::shared_ptr snapshot) + explicit AddSnapshot(std::shared_ptr snapshot) : snapshot_(std::move(snapshot)) {} const std::shared_ptr& snapshot() const { return snapshot_; } @@ -239,9 +241,9 @@ class ICEBERG_EXPORT AddTableSnapshot : public TableUpdate { }; /// \brief Represents removing snapshots from the table -class ICEBERG_EXPORT RemoveTableSnapshots : public TableUpdate { +class ICEBERG_EXPORT RemoveSnapshots : public TableUpdate { public: - explicit RemoveTableSnapshots(std::vector snapshot_ids) + explicit RemoveSnapshots(std::vector snapshot_ids) : snapshot_ids_(std::move(snapshot_ids)) {} const std::vector& snapshot_ids() const { return snapshot_ids_; } @@ -255,9 +257,9 @@ class ICEBERG_EXPORT RemoveTableSnapshots : public TableUpdate { }; /// \brief Represents removing a snapshot reference -class ICEBERG_EXPORT RemoveTableSnapshotRef : public TableUpdate { +class ICEBERG_EXPORT RemoveSnapshotRef : public TableUpdate { public: - explicit RemoveTableSnapshotRef(std::string ref_name) + explicit RemoveSnapshotRef(std::string ref_name) : ref_name_(std::move(ref_name)) {} const std::string& ref_name() const { return ref_name_; } @@ -271,9 +273,9 @@ class ICEBERG_EXPORT RemoveTableSnapshotRef : public TableUpdate { }; /// \brief Represents setting a snapshot reference -class ICEBERG_EXPORT SetTableSnapshotRef : public TableUpdate { +class ICEBERG_EXPORT SetSnapshotRef : public TableUpdate { public: - SetTableSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, + SetSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, std::optional min_snapshots_to_keep = std::nullopt, std::optional max_snapshot_age_ms = std::nullopt, std::optional max_ref_age_ms = std::nullopt) @@ -309,9 +311,9 @@ class ICEBERG_EXPORT SetTableSnapshotRef : public TableUpdate { }; /// \brief Represents setting table properties -class ICEBERG_EXPORT SetTableProperties : public TableUpdate { +class ICEBERG_EXPORT SetProperties : public TableUpdate { public: - explicit SetTableProperties(std::unordered_map updated) + explicit SetProperties(std::unordered_map updated) : updated_(std::move(updated)) {} const std::unordered_map& updated() const { return updated_; } @@ -325,9 +327,9 @@ class ICEBERG_EXPORT SetTableProperties : public TableUpdate { }; /// \brief Represents removing table properties -class ICEBERG_EXPORT RemoveTableProperties : public TableUpdate { +class ICEBERG_EXPORT RemoveProperties : public TableUpdate { public: - explicit RemoveTableProperties(std::vector removed) + explicit RemoveProperties(std::vector removed) : removed_(std::move(removed)) {} const std::vector& removed() const { return removed_; } @@ -341,9 +343,9 @@ class ICEBERG_EXPORT RemoveTableProperties : public TableUpdate { }; /// \brief Represents setting the table location -class ICEBERG_EXPORT SetTableLocation : public TableUpdate { +class ICEBERG_EXPORT SetLocation : public TableUpdate { public: - explicit SetTableLocation(std::string location) : location_(std::move(location)) {} + explicit SetLocation(std::string location) : location_(std::move(location)) {} const std::string& location() const { return location_; } @@ -355,4 +357,6 @@ class ICEBERG_EXPORT SetTableLocation : public TableUpdate { std::string location_; }; +} // namespace table + } // namespace iceberg From b42f3ff5b7e03681b7e1b165d10e9bf5b9dabde9 Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Fri, 17 Oct 2025 16:16:49 +0800 Subject: [PATCH 13/14] fix build error --- src/iceberg/table_requirement.cc | 3 +-- src/iceberg/table_update.cc | 9 +++------ src/iceberg/table_update.h | 12 +++++------- 3 files changed, 9 insertions(+), 15 deletions(-) diff --git a/src/iceberg/table_requirement.cc b/src/iceberg/table_requirement.cc index 6c90efbe7..fa5918394 100644 --- a/src/iceberg/table_requirement.cc +++ b/src/iceberg/table_requirement.cc @@ -46,8 +46,7 @@ Status AssertCurrentSchemaID::Validate(const TableMetadata* base) const { return NotImplemented("AssertCurrentTableSchemaID::Validate not implemented"); } -Status AssertLastAssignedPartitionId::Validate( - const TableMetadata* base) const { +Status AssertLastAssignedPartitionId::Validate(const TableMetadata* base) const { return NotImplemented( "AssertCurrentTableLastAssignedPartitionId::Validate not implemented"); } diff --git a/src/iceberg/table_update.cc b/src/iceberg/table_update.cc index 563ada6f5..69cd81d4f 100644 --- a/src/iceberg/table_update.cc +++ b/src/iceberg/table_update.cc @@ -43,8 +43,7 @@ void UpgradeFormatVersion::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status UpgradeFormatVersion::GenerateRequirements( - TableUpdateContext& context) const { +Status UpgradeFormatVersion::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented( "UpgradeTableFormatVersion::GenerateRequirements not implemented"); } @@ -85,8 +84,7 @@ void SetDefaultPartitionSpec::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status SetDefaultPartitionSpec::GenerateRequirements( - TableUpdateContext& context) const { +Status SetDefaultPartitionSpec::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented( "SetDefaultTablePartitionSpec::GenerateRequirements not implemented"); } @@ -97,8 +95,7 @@ void RemovePartitionSpecs::ApplyTo(TableMetadataBuilder& builder) const { throw IcebergError(std::format("{} not implemented", __FUNCTION__)); } -Status RemovePartitionSpecs::GenerateRequirements( - TableUpdateContext& context) const { +Status RemovePartitionSpecs::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented( "RemoveTablePartitionSpecs::GenerateRequirements not implemented"); } diff --git a/src/iceberg/table_update.h b/src/iceberg/table_update.h index d9d189606..445295a4d 100644 --- a/src/iceberg/table_update.h +++ b/src/iceberg/table_update.h @@ -211,8 +211,7 @@ class ICEBERG_EXPORT AddSortOrder : public TableUpdate { /// \brief Represents setting the default sort order class ICEBERG_EXPORT SetDefaultSortOrder : public TableUpdate { public: - explicit SetDefaultSortOrder(int32_t sort_order_id) - : sort_order_id_(sort_order_id) {} + explicit SetDefaultSortOrder(int32_t sort_order_id) : sort_order_id_(sort_order_id) {} int32_t sort_order_id() const { return sort_order_id_; } @@ -259,8 +258,7 @@ class ICEBERG_EXPORT RemoveSnapshots : public TableUpdate { /// \brief Represents removing a snapshot reference class ICEBERG_EXPORT RemoveSnapshotRef : public TableUpdate { public: - explicit RemoveSnapshotRef(std::string ref_name) - : ref_name_(std::move(ref_name)) {} + explicit RemoveSnapshotRef(std::string ref_name) : ref_name_(std::move(ref_name)) {} const std::string& ref_name() const { return ref_name_; } @@ -276,9 +274,9 @@ class ICEBERG_EXPORT RemoveSnapshotRef : public TableUpdate { class ICEBERG_EXPORT SetSnapshotRef : public TableUpdate { public: SetSnapshotRef(std::string ref_name, int64_t snapshot_id, SnapshotRefType type, - std::optional min_snapshots_to_keep = std::nullopt, - std::optional max_snapshot_age_ms = std::nullopt, - std::optional max_ref_age_ms = std::nullopt) + std::optional min_snapshots_to_keep = std::nullopt, + std::optional max_snapshot_age_ms = std::nullopt, + std::optional max_ref_age_ms = std::nullopt) : ref_name_(std::move(ref_name)), snapshot_id_(snapshot_id), type_(type), From a7f2a2811226ec621e0bb29267da8ac38afdc6c7 Mon Sep 17 00:00:00 2001 From: Guotao Yu Date: Fri, 17 Oct 2025 16:36:15 +0800 Subject: [PATCH 14/14] fix linter error --- src/iceberg/table_requirement.cc | 8 ++------ src/iceberg/table_update.cc | 8 ++------ 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/src/iceberg/table_requirement.cc b/src/iceberg/table_requirement.cc index fa5918394..4ca4b915f 100644 --- a/src/iceberg/table_requirement.cc +++ b/src/iceberg/table_requirement.cc @@ -21,9 +21,7 @@ #include "iceberg/table_metadata.h" -namespace iceberg { - -namespace table { +namespace iceberg::table { Status AssertDoesNotExist::Validate(const TableMetadata* base) const { return NotImplemented("AssertTableDoesNotExist::Validate not implemented"); @@ -59,6 +57,4 @@ Status AssertDefaultSortOrderID::Validate(const TableMetadata* base) const { return NotImplemented("AssertDefaultTableSortOrderID::Validate not implemented"); } -} // namespace table - -} // namespace iceberg +} // namespace iceberg::table diff --git a/src/iceberg/table_update.cc b/src/iceberg/table_update.cc index 69cd81d4f..7d81dd8f1 100644 --- a/src/iceberg/table_update.cc +++ b/src/iceberg/table_update.cc @@ -23,9 +23,7 @@ #include "iceberg/table_metadata.h" #include "iceberg/table_requirements.h" -namespace iceberg { - -namespace table { +namespace iceberg::table { // AssignUUID @@ -198,6 +196,4 @@ Status SetLocation::GenerateRequirements(TableUpdateContext& context) const { return NotImplemented("SetTableLocation::GenerateRequirements not implemented"); } -} // namespace table - -} // namespace iceberg +} // namespace iceberg::table