diff --git a/src/iceberg/CMakeLists.txt b/src/iceberg/CMakeLists.txt index 519757d26..685c815ba 100644 --- a/src/iceberg/CMakeLists.txt +++ b/src/iceberg/CMakeLists.txt @@ -65,6 +65,7 @@ set(ICEBERG_SOURCES sort_order.cc statistics_file.cc table.cc + table_identifier.cc table_metadata.cc table_properties.cc table_requirement.cc @@ -84,6 +85,7 @@ set(ICEBERG_SOURCES util/decimal.cc util/gzip_internal.cc util/murmurhash3_internal.cc + util/property_util.cc util/snapshot_util.cc util/temporal_util.cc util/timepoint.cc diff --git a/src/iceberg/catalog/memory/in_memory_catalog.cc b/src/iceberg/catalog/memory/in_memory_catalog.cc index b3fd0060a..626de29d5 100644 --- a/src/iceberg/catalog/memory/in_memory_catalog.cc +++ b/src/iceberg/catalog/memory/in_memory_catalog.cc @@ -26,7 +26,9 @@ #include "iceberg/table_identifier.h" #include "iceberg/table_metadata.h" #include "iceberg/table_requirement.h" +#include "iceberg/table_requirements.h" #include "iceberg/table_update.h" +#include "iceberg/transaction.h" #include "iceberg/util/macros.h" namespace iceberg { @@ -318,7 +320,7 @@ Result InMemoryNamespace::GetTableMetadataLocation( ICEBERG_RETURN_UNEXPECTED(ns); const auto it = ns.value()->table_metadata_locations_.find(table_ident.name); if (it == ns.value()->table_metadata_locations_.end()) { - return NotFound("{} does not exist", table_ident.name); + return NotFound("Table does not exist: {}", table_ident); } return it->second; } @@ -405,7 +407,24 @@ Result> InMemoryCatalog::CreateTable( const std::string& location, const std::unordered_map& properties) { std::unique_lock lock(mutex_); - return NotImplemented("create table"); + if (root_namespace_->TableExists(identifier).value_or(false)) { + return AlreadyExists("Table already exists: {}", identifier); + } + + std::string base_location = + location.empty() ? warehouse_location_ + "/" + identifier.ToString() : location; + + ICEBERG_ASSIGN_OR_RAISE(auto table_metadata, TableMetadata::Make(*schema, *spec, *order, + location, properties)); + + ICEBERG_ASSIGN_OR_RAISE( + auto metadata_file_location, + TableMetadataUtil::Write(*file_io_, nullptr, "", *table_metadata)); + ICEBERG_RETURN_UNEXPECTED( + root_namespace_->UpdateTableMetadataLocation(identifier, metadata_file_location)); + return Table::Make(identifier, std::move(table_metadata), + std::move(metadata_file_location), file_io_, + std::static_pointer_cast(shared_from_this())); } Result> InMemoryCatalog::UpdateTable( @@ -413,24 +432,43 @@ Result> InMemoryCatalog::UpdateTable( const std::vector>& requirements, const std::vector>& updates) { std::unique_lock lock(mutex_); - ICEBERG_ASSIGN_OR_RAISE(auto base_metadata_location, - root_namespace_->GetTableMetadataLocation(identifier)); - - ICEBERG_ASSIGN_OR_RAISE(auto base, - TableMetadataUtil::Read(*file_io_, base_metadata_location)); + auto base_metadata_location = root_namespace_->GetTableMetadataLocation(identifier); + std::unique_ptr base; + std::unique_ptr builder; + ICEBERG_ASSIGN_OR_RAISE(auto is_create, TableRequirements::IsCreate(requirements)); + if (is_create) { + if (base_metadata_location.has_value()) { + return AlreadyExists("Table already exists: {}", identifier); + } + int8_t format_version = TableMetadata::kDefaultTableFormatVersion; + for (const auto& update : updates) { + if (update->kind() == TableUpdate::Kind::kUpgradeFormatVersion) { + format_version = + dynamic_cast(*update).format_version(); + } + } + builder = TableMetadataBuilder::BuildFromEmpty(format_version); + } else { + ICEBERG_RETURN_UNEXPECTED(base_metadata_location); + ICEBERG_ASSIGN_OR_RAISE( + base, TableMetadataUtil::Read(*file_io_, base_metadata_location.value())); + builder = TableMetadataBuilder::BuildFrom(base.get()); + } for (const auto& requirement : requirements) { ICEBERG_RETURN_UNEXPECTED(requirement->Validate(base.get())); } - auto builder = TableMetadataBuilder::BuildFrom(base.get()); for (const auto& update : updates) { update->ApplyTo(*builder); } ICEBERG_ASSIGN_OR_RAISE(auto updated, builder->Build()); ICEBERG_ASSIGN_OR_RAISE( auto new_metadata_location, - TableMetadataUtil::Write(*file_io_, base.get(), base_metadata_location, *updated)); + TableMetadataUtil::Write( + *file_io_, base.get(), + base_metadata_location.has_value() ? base_metadata_location.value() : "", + *updated)); ICEBERG_RETURN_UNEXPECTED( root_namespace_->UpdateTableMetadataLocation(identifier, new_metadata_location)); TableMetadataUtil::DeleteRemovedMetadataFiles(*file_io_, base.get(), *updated); @@ -445,7 +483,20 @@ Result> InMemoryCatalog::StageCreateTable( const std::string& location, const std::unordered_map& properties) { std::unique_lock lock(mutex_); - return NotImplemented("stage create table"); + if (root_namespace_->TableExists(identifier).value_or(false)) { + return AlreadyExists("Table already exists: {}", identifier); + } + + std::string base_location = + location.empty() ? warehouse_location_ + "/" + identifier.ToString() : location; + + ICEBERG_ASSIGN_OR_RAISE( + auto table_metadata, + TableMetadata::Make(*schema, *spec, *order, base_location, properties)); + ICEBERG_ASSIGN_OR_RAISE( + auto table, StagedTable::Make(identifier, std::move(table_metadata), "", file_io_, + shared_from_this())); + return Transaction::Make(std::move(table), Transaction::Kind::kCreate, false); } Result InMemoryCatalog::TableExists(const TableIdentifier& identifier) const { @@ -495,7 +546,7 @@ Result> InMemoryCatalog::RegisterTable( std::unique_lock lock(mutex_); if (!root_namespace_->NamespaceExists(identifier.ns)) { - return NoSuchNamespace("table namespace does not exist."); + return NoSuchNamespace("Table namespace does not exist: {}", identifier.ns); } if (!root_namespace_->RegisterTable(identifier, metadata_file_location)) { return UnknownError("The registry failed."); diff --git a/src/iceberg/meson.build b/src/iceberg/meson.build index 8327ca2e0..4f0637de3 100644 --- a/src/iceberg/meson.build +++ b/src/iceberg/meson.build @@ -87,6 +87,7 @@ iceberg_sources = files( 'sort_order.cc', 'statistics_file.cc', 'table.cc', + 'table_identifier.cc', 'table_metadata.cc', 'table_properties.cc', 'table_requirement.cc', @@ -106,6 +107,7 @@ iceberg_sources = files( 'util/decimal.cc', 'util/gzip_internal.cc', 'util/murmurhash3_internal.cc', + 'util/property_util.cc', 'util/snapshot_util.cc', 'util/temporal_util.cc', 'util/timepoint.cc', diff --git a/src/iceberg/schema.cc b/src/iceberg/schema.cc index 0b7336b6e..afe6be1e7 100644 --- a/src/iceberg/schema.cc +++ b/src/iceberg/schema.cc @@ -235,7 +235,7 @@ Result> Schema::IdentifierFieldNames() const { for (auto id : identifier_field_ids_) { ICEBERG_ASSIGN_OR_RAISE(auto name, FindColumnNameById(id)); if (!name.has_value()) { - return InvalidSchema("Cannot find the field of the specified field id: {}", id); + return InvalidSchema("Cannot find identifier field id: {}", id); } names.emplace_back(name.value()); } diff --git a/src/iceberg/table_identifier.cc b/src/iceberg/table_identifier.cc new file mode 100644 index 000000000..4fd6ce45a --- /dev/null +++ b/src/iceberg/table_identifier.cc @@ -0,0 +1,32 @@ +/* + * 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_identifier.h" + +#include "iceberg/util/formatter_internal.h" + +namespace iceberg { + +std::string Namespace::ToString() const { return FormatRange(levels, ".", "", ""); } + +std::string TableIdentifier::ToString() const { + return std::format("{}.{}", ns.ToString(), name); +} + +} // namespace iceberg diff --git a/src/iceberg/table_identifier.h b/src/iceberg/table_identifier.h index bef9b81dd..77a2ab9cf 100644 --- a/src/iceberg/table_identifier.h +++ b/src/iceberg/table_identifier.h @@ -27,6 +27,7 @@ #include "iceberg/iceberg_export.h" #include "iceberg/result.h" +#include "iceberg/util/formatter.h" // IWYU pragma: keep namespace iceberg { @@ -35,8 +36,12 @@ struct ICEBERG_EXPORT Namespace { std::vector levels; bool operator==(const Namespace& other) const { return levels == other.levels; } + + std::string ToString() const; }; +ICEBERG_EXPORT inline std::string ToString(const Namespace& ns) { return ns.ToString(); } + /// \brief Identifies a table in iceberg catalog. struct ICEBERG_EXPORT TableIdentifier { Namespace ns; @@ -53,6 +58,12 @@ struct ICEBERG_EXPORT TableIdentifier { } return {}; } + + std::string ToString() const; }; +ICEBERG_EXPORT inline std::string ToString(const TableIdentifier& ident) { + return ident.ToString(); +} + } // namespace iceberg diff --git a/src/iceberg/table_metadata.cc b/src/iceberg/table_metadata.cc index b6cb5f63a..6a5105b4e 100644 --- a/src/iceberg/table_metadata.cc +++ b/src/iceberg/table_metadata.cc @@ -37,6 +37,7 @@ #include "iceberg/exception.h" #include "iceberg/file_io.h" #include "iceberg/json_internal.h" +#include "iceberg/metrics_config.h" #include "iceberg/partition_field.h" #include "iceberg/partition_spec.h" #include "iceberg/result.h" @@ -50,12 +51,73 @@ #include "iceberg/util/gzip_internal.h" #include "iceberg/util/location_util.h" #include "iceberg/util/macros.h" +#include "iceberg/util/property_util.h" +#include "iceberg/util/type_util.h" #include "iceberg/util/uuid.h" + namespace iceberg { namespace { const TimePointMs kInvalidLastUpdatedMs = TimePointMs::min(); constexpr int32_t kLastAdded = -1; constexpr std::string_view kMetadataFolderName = "metadata"; + +// TableMetadata private static methods +Result> FreshPartitionSpec(int32_t spec_id, + const PartitionSpec& spec, + const Schema& base_schema, + const Schema& fresh_schema) { + std::vector partition_fields; + partition_fields.reserve(spec.fields().size()); + int32_t last_partition_field_id = PartitionSpec::kInvalidPartitionFieldId; + for (auto& field : spec.fields()) { + ICEBERG_ASSIGN_OR_RAISE(auto source_name, + base_schema.FindColumnNameById(field.source_id())); + if (!source_name.has_value()) [[unlikely]] { + return InvalidSchema( + "Cannot find source partition field with ID {} in the old schema", + field.source_id()); + } + ICEBERG_ASSIGN_OR_RAISE(auto fresh_field, + fresh_schema.FindFieldByName(source_name.value())); + if (!fresh_field.has_value()) [[unlikely]] { + return InvalidSchema("Partition field {} does not exist in the schema", + source_name.value()); + } + partition_fields.emplace_back(fresh_field.value().get().field_id(), + ++last_partition_field_id, std::string(field.name()), + field.transform()); + } + return PartitionSpec::Make(fresh_schema, spec_id, std::move(partition_fields), false); +} + +Result> FreshSortOrder(int32_t order_id, + const SortOrder& order, + const Schema& base_schema, + const Schema& fresh_schema) { + if (order.is_unsorted()) { + return SortOrder::Unsorted(); + } + + std::vector sort_fields; + sort_fields.reserve(order.fields().size()); + for (const auto& field : order.fields()) { + ICEBERG_ASSIGN_OR_RAISE(auto source_name, + base_schema.FindColumnNameById(field.source_id())); + if (!source_name.has_value()) { + return InvalidSchema("Cannot find source sort field with ID {} in the old schema", + field.source_id()); + } + ICEBERG_ASSIGN_OR_RAISE(auto fresh_field, + fresh_schema.FindFieldByName(source_name.value())); + if (!fresh_field.has_value()) { + return InvalidSchema("Cannot find field '{}' in the new schema", + source_name.value()); + } + sort_fields.emplace_back(fresh_field.value().get().field_id(), field.transform(), + field.direction(), field.null_order()); + } + return SortOrder::Make(order_id, std::move(sort_fields)); +} } // namespace std::string ToString(const SnapshotLogEntry& entry) { @@ -68,6 +130,100 @@ std::string ToString(const MetadataLogEntry& entry) { entry.metadata_file); } +Result> TableMetadata::Make( + const class Schema& schema, const class PartitionSpec& spec, + const class SortOrder& sort_order, const std::string& location, + const std::unordered_map& properties, int format_version) { + for (const auto& [key, _] : properties) { + if (TableProperties::reserved_properties().contains(key)) { + return InvalidArgument( + "Table properties should not contain reserved properties, but got {}", key); + } + } + + // Reassign all column ids to ensure consistency + int32_t last_column_id = 0; + auto next_id = [&last_column_id]() -> int32_t { return ++last_column_id; }; + ICEBERG_ASSIGN_OR_RAISE(auto fresh_schema, + AssignFreshIds(Schema::kInitialSchemaId, schema, next_id)); + + // Rebuild the partition spec using the new column ids + ICEBERG_ASSIGN_OR_RAISE( + auto fresh_spec, + FreshPartitionSpec(PartitionSpec::kInitialSpecId, spec, schema, *fresh_schema)); + + // rebuild the sort order using the new column ids + ICEBERG_ASSIGN_OR_RAISE( + auto fresh_order, + FreshSortOrder(SortOrder::kInitialSortOrderId, sort_order, schema, *fresh_schema)) + + // Validata the metrics configuration. + ICEBERG_RETURN_UNEXPECTED( + MetricsConfig::VerifyReferencedColumns(properties, *fresh_schema)); + + PropertyUtil::ValidateCommitProperties(properties); + + return TableMetadataBuilder::BuildFromEmpty(format_version) + ->SetLocation(location) + .SetCurrentSchema(std::move(fresh_schema), last_column_id) + .SetDefaultPartitionSpec(std::move(fresh_spec)) + .SetDefaultSortOrder(std::move(fresh_order)) + .SetProperties(properties) + .Build(); +} + +std::vector> TableMetadata::ChangesForCreate() const { + std::vector> changes; + + // Add UUID assignment + changes.push_back(std::make_unique(table_uuid)); + + // Add format version upgrade + changes.push_back(std::make_unique(format_version)); + + // Add schema + if (auto current_schema_result = Schema(); current_schema_result.has_value()) { + auto current_schema = current_schema_result.value(); + changes.push_back(std::make_unique(current_schema, last_column_id)); + changes.push_back(std::make_unique(kLastAdded)); + } + + // Add partition spec + if (auto partition_spec_result = PartitionSpec(); partition_spec_result.has_value()) { + auto spec = partition_spec_result.value(); + if (spec && spec->spec_id() != PartitionSpec::kInitialSpecId) { + changes.push_back(std::make_unique(spec)); + } else { + changes.push_back( + std::make_unique(PartitionSpec::Unpartitioned())); + } + changes.push_back(std::make_unique(kLastAdded)); + } + + // Add sort order + if (auto sort_order_result = SortOrder(); sort_order_result.has_value()) { + auto order = sort_order_result.value(); + if (order && order->is_sorted()) { + changes.push_back(std::make_unique(order)); + } else { + changes.push_back(std::make_unique(SortOrder::Unsorted())); + } + changes.push_back(std::make_unique(kLastAdded)); + } + + // Set location if not empty + if (!location.empty()) { + changes.push_back(std::make_unique(location)); + } + + // Set properties if not empty + if (!properties.configs().empty()) { + changes.push_back(std::make_unique(properties.configs())); + } + + return changes; +} + Result> TableMetadata::Schema() const { return SchemaById(current_schema_id); } @@ -435,6 +591,7 @@ class TableMetadataBuilder::Impl { Status SetCurrentSchema(int32_t schema_id); Status RemoveSchemas(const std::unordered_set& schema_ids); Result AddSchema(const Schema& schema, int32_t new_last_column_id); + void SetLocation(std::string_view location); Result> Build(); @@ -819,6 +976,14 @@ Result TableMetadataBuilder::Impl::AddSchema(const Schema& schema, return new_schema_id; } +void TableMetadataBuilder::Impl::SetLocation(std::string_view location) { + if (location == metadata_.location) { + return; + } + metadata_.location = std::string(location); + changes_.push_back(std::make_unique(std::string(location))); +} + Result> TableMetadataBuilder::Impl::Build() { // 1. Validate metadata consistency through TableMetadata#Validate @@ -934,7 +1099,15 @@ std::unique_ptr TableMetadataBuilder::BuildFromEmpty( } std::unique_ptr TableMetadataBuilder::BuildFrom( - const TableMetadata* base) { + const TableMetadata* base, bool is_create) { + if (is_create) { + ICEBERG_DCHECK(base != nullptr, "base should not be nullptr if is_create is true"); + auto builder = BuildFromEmpty(); + for (auto& change : base->ChangesForCreate()) { + change->ApplyTo(*builder); + } + return builder; + } return std::unique_ptr(new TableMetadataBuilder(base)); // NOLINT } @@ -1099,7 +1272,8 @@ TableMetadataBuilder& TableMetadataBuilder::RemoveProperties( } TableMetadataBuilder& TableMetadataBuilder::SetLocation(std::string_view location) { - throw IcebergError(std::format("{} not implemented", __FUNCTION__)); + impl_->SetLocation(location); + return *this; } TableMetadataBuilder& TableMetadataBuilder::AddEncryptionKey( diff --git a/src/iceberg/table_metadata.h b/src/iceberg/table_metadata.h index 3885a4582..5ead53db7 100644 --- a/src/iceberg/table_metadata.h +++ b/src/iceberg/table_metadata.h @@ -93,11 +93,11 @@ struct ICEBERG_EXPORT TableMetadata { /// The highest assigned column ID for the table int32_t last_column_id; /// A list of schemas - std::vector> schemas; + std::vector> schemas; /// ID of the table's current schema std::optional current_schema_id; /// A list of partition specs - std::vector> partition_specs; + std::vector> partition_specs; /// ID of the current partition spec that writers should use by default int32_t default_spec_id; /// The highest assigned partition field ID across all partition specs for the table @@ -107,7 +107,7 @@ struct ICEBERG_EXPORT TableMetadata { /// ID of the current table snapshot int64_t current_snapshot_id; /// A list of valid snapshots - std::vector> snapshots; + std::vector> snapshots; /// A list of timestamp and snapshot ID pairs that encodes changes to the current /// snapshot for the table std::vector snapshot_log; @@ -115,7 +115,7 @@ struct ICEBERG_EXPORT TableMetadata { /// previous metadata files for the table std::vector metadata_log; /// A list of sort orders - std::vector> sort_orders; + std::vector> sort_orders; /// Default sort order id of the table int32_t default_sort_order_id; /// A map of snapshot references @@ -127,19 +127,28 @@ struct ICEBERG_EXPORT TableMetadata { /// A `long` higher than all assigned row IDs int64_t next_row_id; + static Result> Make( + const class Schema& schema, const class PartitionSpec& spec, + const class SortOrder& sort_order, const std::string& location, + const std::unordered_map& properties, + int format_version = kDefaultTableFormatVersion); + + /// \brief Get the list of changes required to create this table metadata from scratch + std::vector> ChangesForCreate() const; + /// \brief Get the current schema, return NotFoundError if not found - Result> Schema() const; + Result> Schema() const; /// \brief Get the current schema by ID, return NotFoundError if not found - Result> SchemaById( + Result> SchemaById( std::optional schema_id) const; /// \brief Get the current partition spec, return NotFoundError if not found - Result> PartitionSpec() const; + Result> PartitionSpec() const; /// \brief Get the current sort order, return NotFoundError if not found - Result> SortOrder() const; + Result> SortOrder() const; /// \brief Get the current snapshot, return NotFoundError if not found - Result> Snapshot() const; + Result> Snapshot() const; /// \brief Get the snapshot of this table with the given id - Result> SnapshotById(int64_t snapshot_id) const; + Result> SnapshotById(int64_t snapshot_id) const; ICEBERG_EXPORT friend bool operator==(const TableMetadata& lhs, const TableMetadata& rhs); @@ -210,8 +219,13 @@ class ICEBERG_EXPORT TableMetadataBuilder : public ErrorCollector { /// \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 std::unique_ptr BuildFrom(const TableMetadata* base); + /// \param is_create Whether the builder is for creating a new table. It will call + /// `BuildFromEmpty` and set changes to make the tablemetadata but not copy the base + /// metadata directly if true. + /// \return A new TableMetadataBuilder instance initialized + /// with base metadata + static std::unique_ptr BuildFrom(const TableMetadata* base, + bool is_create = false); /// \brief Set the metadata location of the table /// diff --git a/src/iceberg/table_properties.cc b/src/iceberg/table_properties.cc index 96633bc24..db6adedcf 100644 --- a/src/iceberg/table_properties.cc +++ b/src/iceberg/table_properties.cc @@ -31,6 +31,13 @@ const std::unordered_set& TableProperties::reserved_properties() { return kReservedProperties; } +const std::unordered_set& TableProperties::commit_properties() { + static const std::unordered_set kCommitProperties = { + kCommitNumRetries.key(), kCommitMinRetryWaitMs.key(), kCommitMaxRetryWaitMs.key(), + kCommitTotalRetryTimeMs.key()}; + return kCommitProperties; +} + TableProperties TableProperties::default_properties() { return {}; } TableProperties TableProperties::FromMap( diff --git a/src/iceberg/table_properties.h b/src/iceberg/table_properties.h index debe61da2..feb4a2001 100644 --- a/src/iceberg/table_properties.h +++ b/src/iceberg/table_properties.h @@ -286,6 +286,9 @@ class ICEBERG_EXPORT TableProperties : public ConfigBase { /// \return The set of reserved property keys static const std::unordered_set& reserved_properties(); + /// \brief Get the set of commit table property keys. + static const std::unordered_set& commit_properties(); + /// \brief Create a default TableProperties instance. /// /// \return A unique pointer to a TableProperties instance with default values diff --git a/src/iceberg/table_requirement.h b/src/iceberg/table_requirement.h index eb818106c..82779aec9 100644 --- a/src/iceberg/table_requirement.h +++ b/src/iceberg/table_requirement.h @@ -43,14 +43,14 @@ namespace iceberg { class ICEBERG_EXPORT TableRequirement { public: enum class Kind : uint8_t { - AssertDoesNotExist, - AssertUUID, - AssertRefSnapshotID, - AssertLastAssignedFieldId, - AssertCurrentSchemaID, - AssertLastAssignedPartitionId, - AssertDefaultSpecID, - AssertDefaultSortOrderID, + kAssertDoesNotExist, + kAssertUUID, + kAssertRefSnapshotID, + kAssertLastAssignedFieldId, + kAssertCurrentSchemaID, + kAssertLastAssignedPartitionId, + kAssertDefaultSpecID, + kAssertDefaultSortOrderID, }; virtual ~TableRequirement() = default; @@ -75,7 +75,7 @@ class ICEBERG_EXPORT AssertDoesNotExist : public TableRequirement { public: AssertDoesNotExist() = default; - Kind kind() const override { return Kind::AssertDoesNotExist; } + Kind kind() const override { return Kind::kAssertDoesNotExist; } Status Validate(const TableMetadata* base) const override; }; @@ -90,7 +90,7 @@ class ICEBERG_EXPORT AssertUUID : public TableRequirement { const std::string& uuid() const { return uuid_; } - Kind kind() const override { return Kind::AssertUUID; } + Kind kind() const override { return Kind::kAssertUUID; } Status Validate(const TableMetadata* base) const override; @@ -112,7 +112,7 @@ class ICEBERG_EXPORT AssertRefSnapshotID : public TableRequirement { const std::optional& snapshot_id() const { return snapshot_id_; } - Kind kind() const override { return Kind::AssertRefSnapshotID; } + Kind kind() const override { return Kind::kAssertRefSnapshotID; } Status Validate(const TableMetadata* base) const override; @@ -132,7 +132,7 @@ class ICEBERG_EXPORT AssertLastAssignedFieldId : public TableRequirement { int32_t last_assigned_field_id() const { return last_assigned_field_id_; } - Kind kind() const override { return Kind::AssertLastAssignedFieldId; } + Kind kind() const override { return Kind::kAssertLastAssignedFieldId; } Status Validate(const TableMetadata* base) const override; @@ -150,7 +150,7 @@ class ICEBERG_EXPORT AssertCurrentSchemaID : public TableRequirement { int32_t schema_id() const { return schema_id_; } - Kind kind() const override { return Kind::AssertCurrentSchemaID; } + Kind kind() const override { return Kind::kAssertCurrentSchemaID; } Status Validate(const TableMetadata* base) const override; @@ -169,7 +169,7 @@ class ICEBERG_EXPORT AssertLastAssignedPartitionId : public TableRequirement { int32_t last_assigned_partition_id() const { return last_assigned_partition_id_; } - Kind kind() const override { return Kind::AssertLastAssignedPartitionId; } + Kind kind() const override { return Kind::kAssertLastAssignedPartitionId; } Status Validate(const TableMetadata* base) const override; @@ -187,7 +187,7 @@ class ICEBERG_EXPORT AssertDefaultSpecID : public TableRequirement { int32_t spec_id() const { return spec_id_; } - Kind kind() const override { return Kind::AssertDefaultSpecID; } + Kind kind() const override { return Kind::kAssertDefaultSpecID; } Status Validate(const TableMetadata* base) const override; @@ -206,7 +206,7 @@ class ICEBERG_EXPORT AssertDefaultSortOrderID : public TableRequirement { int32_t sort_order_id() const { return sort_order_id_; } - Kind kind() const override { return Kind::AssertDefaultSortOrderID; } + Kind kind() const override { return Kind::kAssertDefaultSortOrderID; } Status Validate(const TableMetadata* base) const override; diff --git a/src/iceberg/table_requirements.cc b/src/iceberg/table_requirements.cc index 6de6c59e6..8222ded31 100644 --- a/src/iceberg/table_requirements.cc +++ b/src/iceberg/table_requirements.cc @@ -19,7 +19,9 @@ #include "iceberg/table_requirements.h" +#include #include +#include #include "iceberg/snapshot.h" #include "iceberg/table_metadata.h" @@ -134,4 +136,19 @@ Result>> TableRequirements::ForUpd return context.Build(); } +Result TableRequirements::IsCreate( + const std::vector>& requirements) { + bool is_create = std::ranges::any_of(requirements, [](const auto& req) { + return req->kind() == TableRequirement::Kind::kAssertDoesNotExist; + }); + + if (is_create) { + ICEBERG_PRECHECK( + requirements.size() == 1, + "Cannot have other requirements than AssertDoesNotExist in a table creation"); + } + + return is_create; +} + } // namespace iceberg diff --git a/src/iceberg/table_requirements.h b/src/iceberg/table_requirements.h index f79f0bead..6deefdf3a 100644 --- a/src/iceberg/table_requirements.h +++ b/src/iceberg/table_requirements.h @@ -144,6 +144,10 @@ class ICEBERG_EXPORT TableRequirements { static Result>> ForUpdateTable( const TableMetadata& base, const std::vector>& table_updates); + + /// \brief Check if the requirements are for table creation + static Result IsCreate( + const std::vector>& requirements); }; } // namespace iceberg diff --git a/src/iceberg/table_update.cc b/src/iceberg/table_update.cc index 9d92a16ea..91578977c 100644 --- a/src/iceberg/table_update.cc +++ b/src/iceberg/table_update.cc @@ -201,7 +201,7 @@ void RemoveProperties::GenerateRequirements(TableUpdateContext& context) const { // SetLocation void SetLocation::ApplyTo(TableMetadataBuilder& builder) const { - throw IcebergError(std::format("{} not implemented", __FUNCTION__)); + builder.SetLocation(location_); } void SetLocation::GenerateRequirements(TableUpdateContext& context) const { diff --git a/src/iceberg/test/formatter_test.cc b/src/iceberg/test/formatter_test.cc index 767020b2e..7c85b3744 100644 --- a/src/iceberg/test/formatter_test.cc +++ b/src/iceberg/test/formatter_test.cc @@ -29,6 +29,7 @@ #include #include "iceberg/statistics_file.h" +#include "iceberg/table_identifier.h" #include "iceberg/util/formatter_internal.h" namespace iceberg { @@ -160,4 +161,19 @@ TEST(FormatterTest, StatisticsFileFormat) { EXPECT_EQ(expected, std::format("{}", statistics_file)); } +// For Types that has a ToString function +TEST(FormatterTest, TableIdentifierFormat) { + TableIdentifier empty_ns_table{ + .ns = Namespace({}), + .name = "table_name", + }; + EXPECT_EQ(".table_name", std::format("{}", empty_ns_table)); + + TableIdentifier table{ + .ns = Namespace({"ns1", "ns2"}), + .name = "table_name", + }; + EXPECT_EQ("ns1.ns2.table_name", std::format("{}", table)); +} + } // namespace iceberg diff --git a/src/iceberg/test/in_memory_catalog_test.cc b/src/iceberg/test/in_memory_catalog_test.cc index 194d6da51..fdfe2b2de 100644 --- a/src/iceberg/test/in_memory_catalog_test.cc +++ b/src/iceberg/test/in_memory_catalog_test.cc @@ -28,7 +28,9 @@ #include #include "iceberg/arrow/arrow_fs_file_io_internal.h" +#include "iceberg/partition_spec.h" #include "iceberg/schema.h" +#include "iceberg/sort_order.h" #include "iceberg/table.h" #include "iceberg/table_identifier.h" #include "iceberg/table_metadata.h" @@ -38,6 +40,8 @@ #include "iceberg/test/mock_catalog.h" #include "iceberg/test/mock_io.h" #include "iceberg/test/test_resource.h" +#include "iceberg/transaction.h" +#include "iceberg/update/update_properties.h" #include "iceberg/util/uuid.h" namespace iceberg { @@ -106,6 +110,26 @@ TEST_F(InMemoryCatalogTest, TableExists) { EXPECT_THAT(result, HasValue(::testing::Eq(false))); } +TEST_F(InMemoryCatalogTest, CreateTable) { + TableIdentifier table_ident{.ns = {}, .name = "t1"}; + auto schema = std::make_shared( + std::vector{SchemaField::MakeRequired(1, "x", int64())}, + /*schema_id=*/1); + auto spec = PartitionSpec::Unpartitioned(); + auto sort_order = SortOrder::Unsorted(); + auto metadata_location = GenerateTestTableLocation(table_ident.name); + + // Create table successfully + auto table = catalog_->CreateTable(table_ident, schema, spec, sort_order, + metadata_location, {{"property1", "value1"}}); + EXPECT_THAT(table, IsOk()); + + // Create table already exists + auto table2 = catalog_->CreateTable(table_ident, schema, spec, sort_order, + metadata_location, {{"property1", "value1"}}); + EXPECT_THAT(table2, IsError(ErrorKind::kAlreadyExists)); +} + TEST_F(InMemoryCatalogTest, RegisterTable) { TableIdentifier tableIdent{.ns = {}, .name = "t1"}; @@ -231,6 +255,50 @@ TEST_F(InMemoryCatalogTest, UpdateTable) { testing::HasSubstr("metadata/00002-")); } +TEST_F(InMemoryCatalogTest, StageCreateTable) { + TableIdentifier table_ident{.ns = {}, .name = "t1"}; + auto schema = std::make_shared( + std::vector{SchemaField::MakeRequired(1, "x", int64())}, + /*schema_id=*/1); + auto spec = PartitionSpec::Unpartitioned(); + auto sort_order = SortOrder::Unsorted(); + + // Stage table + ICEBERG_UNWRAP_OR_FAIL( + auto staged_table, + catalog_->StageCreateTable(table_ident, schema, spec, sort_order, + GenerateTestTableLocation(table_ident.name), {})); + + // Perform the update + ICEBERG_UNWRAP_OR_FAIL(auto update_properties, staged_table->NewUpdateProperties()); + update_properties->Set("property1", "value1").Commit(); + auto res1 = staged_table->Commit(); + EXPECT_THAT(res1, IsOk()); + auto created_table = res1.value(); + EXPECT_EQ("t1", created_table->name().name); + EXPECT_EQ("value1", created_table->metadata()->properties.Get( + TableProperties::Entry("property1", ""))); + + // Staged already exist table + auto res = catalog_->StageCreateTable(table_ident, schema, spec, sort_order, + GenerateTestTableLocation(table_ident.name), {}); + EXPECT_THAT(res, IsError(ErrorKind::kAlreadyExists)); + + // Stage create ok but commit already exist + TableIdentifier table_ident2{.ns = {}, .name = "t2"}; + ICEBERG_UNWRAP_OR_FAIL( + auto staged_table2, + catalog_->StageCreateTable(table_ident2, schema, spec, sort_order, + GenerateTestTableLocation(table_ident2.name), {})); + ICEBERG_UNWRAP_OR_FAIL( + auto created_table2, + catalog_->CreateTable(table_ident2, schema, spec, sort_order, + GenerateTestTableLocation(table_ident2.name), {})); + + auto commit_res = staged_table2->Commit(); + EXPECT_THAT(commit_res, IsError(ErrorKind::kAlreadyExists)); +} + TEST_F(InMemoryCatalogTest, DropTable) { TableIdentifier tableIdent{.ns = {}, .name = "t1"}; auto result = catalog_->DropTable(tableIdent, false); diff --git a/src/iceberg/test/table_metadata_builder_test.cc b/src/iceberg/test/table_metadata_builder_test.cc index a912a08f5..41fec375f 100644 --- a/src/iceberg/test/table_metadata_builder_test.cc +++ b/src/iceberg/test/table_metadata_builder_test.cc @@ -49,6 +49,24 @@ std::shared_ptr CreateTestSchema() { return std::make_shared(std::vector{field1, field2, field3}, 0); } +// Helper function to create a simple schema with invalid identifier fields +std::shared_ptr CreateInvalidSchema() { + auto field1 = SchemaField::MakeRequired(2, "id", int32()); + auto field2 = SchemaField::MakeRequired(5, "part_col", string()); + auto field3 = SchemaField::MakeRequired(8, "sort_col", timestamp()); + return std::make_shared(std::vector{field1, field2, field3}, + std::make_optional(1), std::vector{1}); +} + +// Helper function to create a simple schema with disordered field_ids +std::shared_ptr CreateDisorderedSchema() { + auto field1 = SchemaField::MakeRequired(2, "id", int32()); + auto field2 = SchemaField::MakeRequired(5, "part_col", string()); + auto field3 = SchemaField::MakeRequired(8, "sort_col", timestamp()); + return std::make_shared(std::vector{field1, field2, field3}, + std::make_optional(1), std::vector{2}); +} + // Helper function to create base metadata for tests std::unique_ptr CreateBaseMetadata() { auto metadata = std::make_unique(); @@ -73,6 +91,98 @@ std::unique_ptr CreateBaseMetadata() { } // namespace +// test for TableMetadata +TEST(TableMetadataTest, Make) { + auto Schema = CreateDisorderedSchema(); + ICEBERG_UNWRAP_OR_FAIL( + auto spec, PartitionSpec::Make(1, std::vector{PartitionField( + 5, 1, "part_name", Transform::Identity())})); + ICEBERG_UNWRAP_OR_FAIL( + auto order, SortOrder::Make(1, std::vector{SortField( + 8, Transform::Identity(), + SortDirection::kAscending, NullOrder::kLast)})); + + ICEBERG_UNWRAP_OR_FAIL( + auto metadata, TableMetadata::Make(*Schema, *spec, *order, "s3://bucket/test", {})); + // Check schema fields + ASSERT_EQ(1, metadata->schemas.size()); + auto fields = metadata->schemas[0]->fields() | std::ranges::to(); + ASSERT_EQ(3, fields.size()); + EXPECT_EQ(1, fields[0].field_id()); + EXPECT_EQ("id", fields[0].name()); + EXPECT_EQ(2, fields[1].field_id()); + EXPECT_EQ("part_col", fields[1].name()); + EXPECT_EQ(3, fields[2].field_id()); + EXPECT_EQ("sort_col", fields[2].name()); + const auto& identifier_field_ids = metadata->schemas[0]->IdentifierFieldIds(); + ASSERT_EQ(1, identifier_field_ids.size()); + EXPECT_EQ(1, identifier_field_ids[0]); + + // Check partition spec + ASSERT_EQ(1, metadata->partition_specs.size()); + EXPECT_EQ(PartitionSpec::kInitialSpecId, metadata->partition_specs[0]->spec_id()); + auto spec_fields = + metadata->partition_specs[0]->fields() | std::ranges::to(); + ASSERT_EQ(1, spec_fields.size()); + EXPECT_EQ(PartitionSpec::kInvalidPartitionFieldId + 1, spec_fields[0].field_id()); + EXPECT_EQ(2, spec_fields[0].source_id()); + EXPECT_EQ("part_name", spec_fields[0].name()); + + // Check sort order + ASSERT_EQ(1, metadata->sort_orders.size()); + EXPECT_EQ(SortOrder::kInitialSortOrderId, metadata->sort_orders[0]->order_id()); + auto order_fields = metadata->sort_orders[0]->fields() | std::ranges::to(); + ASSERT_EQ(1, order_fields.size()); + EXPECT_EQ(3, order_fields[0].source_id()); + EXPECT_EQ(SortDirection::kAscending, order_fields[0].direction()); + EXPECT_EQ(NullOrder::kLast, order_fields[0].null_order()); +} + +TEST(TableMetadataTest, MakeWithInvalidSchema) { + auto schema = CreateInvalidSchema(); + ICEBERG_UNWRAP_OR_FAIL( + auto spec, PartitionSpec::Make(1, std::vector{PartitionField( + 5, 1, "part_name", Transform::Identity())})); + ICEBERG_UNWRAP_OR_FAIL( + auto order, SortOrder::Make(1, std::vector{SortField( + 5, Transform::Identity(), + SortDirection::kAscending, NullOrder::kLast)})); + + auto res = TableMetadata::Make(*schema, *spec, *order, "s3://bucket/test", {}); + EXPECT_THAT(res, IsError(ErrorKind::kInvalidSchema)); + EXPECT_THAT(res, HasErrorMessage("Cannot find identifier field id")); +} + +TEST(TableMetadataTest, MakeWithInvalidPartitionSpec) { + auto schema = CreateDisorderedSchema(); + ICEBERG_UNWRAP_OR_FAIL( + auto spec, PartitionSpec::Make(1, std::vector{PartitionField( + 6, 1, "part_name", Transform::Identity())})); + ICEBERG_UNWRAP_OR_FAIL( + auto order, SortOrder::Make(1, std::vector{SortField( + 8, Transform::Identity(), + SortDirection::kAscending, NullOrder::kLast)})); + + auto res = TableMetadata::Make(*schema, *spec, *order, "s3://bucket/test", {}); + EXPECT_THAT(res, IsError(ErrorKind::kInvalidSchema)); + EXPECT_THAT(res, HasErrorMessage("Cannot find source partition field")); +} + +TEST(TableMetadataTest, MakeWithInvalidSortOrder) { + auto schema = CreateDisorderedSchema(); + ICEBERG_UNWRAP_OR_FAIL( + auto spec, PartitionSpec::Make(1, std::vector{PartitionField( + 5, 1, "part_name", Transform::Identity())})); + ICEBERG_UNWRAP_OR_FAIL( + auto order, SortOrder::Make(1, std::vector{SortField( + 9, Transform::Identity(), + SortDirection::kAscending, NullOrder::kLast)})); + + auto res = TableMetadata::Make(*schema, *spec, *order, "s3://bucket/test", {}); + EXPECT_THAT(res, IsError(ErrorKind::kInvalidSchema)); + EXPECT_THAT(res, HasErrorMessage("Cannot find source sort field")); +} + // test construction of TableMetadataBuilder TEST(TableMetadataBuilderTest, BuildFromEmpty) { auto builder = TableMetadataBuilder::BuildFromEmpty(2); diff --git a/src/iceberg/test/table_requirement_test.cc b/src/iceberg/test/table_requirement_test.cc index 8b67561f8..07055f18c 100644 --- a/src/iceberg/test/table_requirement_test.cc +++ b/src/iceberg/test/table_requirement_test.cc @@ -36,7 +36,7 @@ TEST(TableRequirementTest, AssertUUID) { // Success - UUID matches table::AssertUUID requirement("test-uuid-1234"); - EXPECT_EQ(TableRequirement::Kind::AssertUUID, requirement.kind()); + EXPECT_EQ(TableRequirement::Kind::kAssertUUID, requirement.kind()); ASSERT_THAT(requirement.Validate(base.get()), IsOk()); // UUID mismatch @@ -63,7 +63,7 @@ TEST(TableRequirementTest, AssertCurrentSchemaID) { // Success - schema ID matches table::AssertCurrentSchemaID requirement(5); - EXPECT_EQ(TableRequirement::Kind::AssertCurrentSchemaID, requirement.kind()); + EXPECT_EQ(TableRequirement::Kind::kAssertCurrentSchemaID, requirement.kind()); ASSERT_THAT(requirement.Validate(base.get()), IsOk()); // Schema ID mismatch @@ -89,7 +89,7 @@ TEST(TableRequirementTest, AssertCurrentSchemaID) { TEST(TableRequirementTest, AssertDoesNotExist) { // Success - table does not exist (null metadata) table::AssertDoesNotExist requirement; - EXPECT_EQ(TableRequirement::Kind::AssertDoesNotExist, requirement.kind()); + EXPECT_EQ(TableRequirement::Kind::kAssertDoesNotExist, requirement.kind()); ASSERT_THAT(requirement.Validate(nullptr), IsOk()); // Table already exists @@ -108,7 +108,7 @@ TEST(TableRequirementTest, AssertRefSnapshotID) { // Success - ref snapshot ID matches table::AssertRefSnapshotID requirement("main", 100); - EXPECT_EQ(TableRequirement::Kind::AssertRefSnapshotID, requirement.kind()); + EXPECT_EQ(TableRequirement::Kind::kAssertRefSnapshotID, requirement.kind()); ASSERT_THAT(requirement.Validate(base.get()), IsOk()); // Snapshot ID mismatch @@ -140,7 +140,7 @@ TEST(TableRequirementTest, AssertLastAssignedFieldId) { // Success - field ID matches table::AssertLastAssignedFieldId requirement(10); - EXPECT_EQ(TableRequirement::Kind::AssertLastAssignedFieldId, requirement.kind()); + EXPECT_EQ(TableRequirement::Kind::kAssertLastAssignedFieldId, requirement.kind()); ASSERT_THAT(requirement.Validate(base.get()), IsOk()); // Field ID mismatch @@ -160,7 +160,7 @@ TEST(TableRequirementTest, AssertLastAssignedPartitionId) { // Success - partition ID matches table::AssertLastAssignedPartitionId requirement(5); - EXPECT_EQ(TableRequirement::Kind::AssertLastAssignedPartitionId, requirement.kind()); + EXPECT_EQ(TableRequirement::Kind::kAssertLastAssignedPartitionId, requirement.kind()); ASSERT_THAT(requirement.Validate(base.get()), IsOk()); // Partition ID mismatch @@ -182,7 +182,7 @@ TEST(TableRequirementTest, AssertDefaultSpecID) { // Success - spec ID matches table::AssertDefaultSpecID requirement(3); - EXPECT_EQ(TableRequirement::Kind::AssertDefaultSpecID, requirement.kind()); + EXPECT_EQ(TableRequirement::Kind::kAssertDefaultSpecID, requirement.kind()); ASSERT_THAT(requirement.Validate(base.get()), IsOk()); // Spec ID mismatch @@ -198,7 +198,7 @@ TEST(TableRequirementTest, AssertDefaultSortOrderID) { // Success - sort order ID matches table::AssertDefaultSortOrderID requirement(2); - EXPECT_EQ(TableRequirement::Kind::AssertDefaultSortOrderID, requirement.kind()); + EXPECT_EQ(TableRequirement::Kind::kAssertDefaultSortOrderID, requirement.kind()); ASSERT_THAT(requirement.Validate(base.get()), IsOk()); // Sort order ID mismatch diff --git a/src/iceberg/test/table_requirements_test.cc b/src/iceberg/test/table_requirements_test.cc index bbbcc681e..c05e505f7 100644 --- a/src/iceberg/test/table_requirements_test.cc +++ b/src/iceberg/test/table_requirements_test.cc @@ -102,6 +102,20 @@ TEST(TableRequirementsTest, EmptyUpdatesForCreateTable) { EXPECT_NE(assert_does_not_exist, nullptr); } +TEST(TableRequirementsTest, IsCreate) { + // Should have only AssertDoesNotExist requirement + std::vector> requirements; + requirements.push_back(std::make_unique()); + EXPECT_TRUE(TableRequirements::IsCreate(requirements)); + + // Not only have AssertDoesNotExist requirement + requirements.push_back(std::make_unique(0)); + auto res = TableRequirements::IsCreate(requirements); + EXPECT_THAT(res, IsError(ErrorKind::kInvalidArgument)); + EXPECT_THAT(res, + HasErrorMessage("Cannot have other requirements than AssertDoesNotExist")); +} + TEST(TableRequirementsTest, EmptyUpdatesForUpdateTable) { auto metadata = CreateBaseMetadata(); std::vector> updates; diff --git a/src/iceberg/transaction.cc b/src/iceberg/transaction.cc index 9404fe2e1..a40d99712 100644 --- a/src/iceberg/transaction.cc +++ b/src/iceberg/transaction.cc @@ -40,7 +40,8 @@ Transaction::Transaction(std::shared_ptr table, Kind kind, bool auto_comm : table_(std::move(table)), kind_(kind), auto_commit_(auto_commit), - metadata_builder_(TableMetadataBuilder::BuildFrom(table_->metadata().get())) {} + metadata_builder_(TableMetadataBuilder::BuildFrom(table_->metadata().get(), + kind == Kind::kCreate)) {} Transaction::~Transaction() = default; diff --git a/src/iceberg/util/meson.build b/src/iceberg/util/meson.build index 9f3277533..188981b73 100644 --- a/src/iceberg/util/meson.build +++ b/src/iceberg/util/meson.build @@ -31,6 +31,7 @@ install_headers( 'location_util.h', 'macros.h', 'partition_value_util.h', + 'property_util.h', 'string_util.h', 'temporal_util.h', 'timepoint.h', diff --git a/src/iceberg/util/property_util.cc b/src/iceberg/util/property_util.cc new file mode 100644 index 000000000..636083fdd --- /dev/null +++ b/src/iceberg/util/property_util.cc @@ -0,0 +1,52 @@ +/* + * 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/util/property_util.h" + +#include + +#include "iceberg/table_properties.h" + +namespace iceberg { + +Status PropertyUtil::ValidateCommitProperties( + const std::unordered_map& properties) { + for (const auto& property : TableProperties::commit_properties()) { + if (auto it = properties.find(property); it != properties.end()) { + int32_t parsed; + auto [ptr, ec] = std::from_chars(it->second.data(), + it->second.data() + it->second.size(), parsed); + if (ec == std::errc::invalid_argument) { + return ValidationFailed("Table property {} must have integer value, but got {}", + property, it->second); + } else if (ec == std::errc::result_out_of_range) { + return ValidationFailed("Table property {} value out of range {}", property, + it->second); + } + if (parsed < 0) { + return ValidationFailed( + "Table property {} must have non negative integer value, but got {}", + property, parsed); + } + } + } + return {}; +} + +} // namespace iceberg diff --git a/src/iceberg/util/property_util.h b/src/iceberg/util/property_util.h new file mode 100644 index 000000000..4e3e9b125 --- /dev/null +++ b/src/iceberg/util/property_util.h @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#pragma once + +#include +#include + +#include "iceberg/iceberg_export.h" +#include "iceberg/result.h" + +namespace iceberg { + +class ICEBERG_EXPORT PropertyUtil { + public: + static Status ValidateCommitProperties( + const std::unordered_map& properties); +}; + +} // namespace iceberg