diff --git a/ci/docker/integration/runner/Dockerfile b/ci/docker/integration/runner/Dockerfile index 6b2c679699cb..5923adc6cda5 100644 --- a/ci/docker/integration/runner/Dockerfile +++ b/ci/docker/integration/runner/Dockerfile @@ -80,6 +80,8 @@ org.apache.hudi:hudi-spark3.5-bundle_2.12:1.0.1,\ org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.4.3,\ org.apache.hadoop:hadoop-aws:3.3.4,\ com.amazonaws:aws-java-sdk-bundle:1.12.262,\ +org.apache.hadoop:hadoop-azure:3.3.4,\ +com.microsoft.azure:azure-storage:8.6.6,\ org.apache.spark:spark-avro_2.12:3.5.1"\ && /spark-3.5.5-bin-hadoop3/bin/spark-shell --packages "$packages" \ && find /root/.ivy2/ -name '*.jar' -exec ln -sf {} /spark-3.5.5-bin-hadoop3/jars/ \; diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index c0c27b384429..6da79527d35d 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -649,6 +649,88 @@ GRANT ALTER TABLE ON my_iceberg_table TO my_user; - The catalog's own authorization (REST catalog auth, AWS Glue IAM, etc.) is enforced independently when ClickHouse updates the metadata ::: +### Remove Orphan Files {#iceberg-remove-orphan-files} + +Orphan files are files on storage that are not referenced by any snapshot in the Iceberg table metadata. They accumulate from failed writes, partial cleanup after compaction, and interrupted operations, causing unbounded storage growth. The `remove_orphan_files` command identifies and removes these orphan files. + +**Syntax:** + +```sql +-- Positional form: single unnamed older_than argument +ALTER TABLE iceberg_table EXECUTE remove_orphan_files('timestamp') + +-- Named form +ALTER TABLE iceberg_table EXECUTE remove_orphan_files( + older_than = 'timestamp', + location = 'path', + dry_run = 0|1 +) + +-- No arguments: use all defaults (older_than = 3 days ago) +ALTER TABLE iceberg_table EXECUTE remove_orphan_files() +``` + +**Parameters:** + +| Parameter | Type | Default | Description | +|---|---|---|---| +| `older_than` | `String` (timestamp) | 3 days ago (configurable via `iceberg_orphan_files_older_than_seconds`) | Only consider files with a last-modified time older than this timestamp as orphan candidates. Safety guard against deleting files from in-progress writes. | +| `location` | `String` | Table location | Restrict the scan to a specific subdirectory under the table location (e.g., `'data/'` or `'metadata/'`). | +| `dry_run` | `UInt64` | `0` | When `1`, identify orphan files and return the result summary without actually deleting anything. | + +**Examples:** + +```sql +-- Remove orphan files older than a specific timestamp +ALTER TABLE iceberg_table EXECUTE remove_orphan_files('2026-03-01 00:00:00'); + +-- Dry run: preview which files would be deleted +ALTER TABLE iceberg_table EXECUTE remove_orphan_files(dry_run = 1); + +-- Scan only the data directory +ALTER TABLE iceberg_table EXECUTE remove_orphan_files( + older_than = '2026-03-01 00:00:00', + location = 'data/' +); + +-- Combine positional older_than with named arguments +ALTER TABLE iceberg_table EXECUTE remove_orphan_files( + '2026-03-01 00:00:00', + dry_run = 1 +); +``` + +**Output:** + +The command returns a table with `metric_name` and `metric_value` columns showing the count of deleted (or would-be-deleted in dry_run mode) files by category. File categories are classified using best-effort heuristics based on file naming conventions; files that do not match any specific pattern default to `deleted_data_files_count`: + +| metric_name | metric_value | +|---|---| +| deleted_data_files_count | 5 | +| deleted_position_delete_files_count | 2 | +| deleted_equality_delete_files_count | 0 | +| deleted_manifest_files_count | 3 | +| deleted_manifest_lists_count | 1 | +| deleted_metadata_files_count | 0 | +| deleted_statistics_files_count | 0 | +| skipped_missing_metadata_count | 0 | +| failed_deletions_count | 0 | + +**Settings:** + +| Setting | Type | Default | Description | +|---|---|---|---| +| `allow_iceberg_remove_orphan_files` | `Bool` | `false` | Gate setting to enable the feature (experimental). | +| `iceberg_orphan_files_older_than_seconds` | `UInt64` | `259200` (3 days) | Default `older_than` threshold in seconds when the argument is omitted. | + +:::note +- **Requires Iceberg format version 2 (or higher).** Version 1 tables are rejected because they lack `manifest-list` pointers in snapshots, which are needed to safely determine the reachable file set. Running the command on a v1 table returns a `BAD_ARGUMENTS` error. +- Requires both `allow_insert_into_iceberg` and `allow_iceberg_remove_orphan_files` settings to be enabled +- It is recommended to run `expire_snapshots` before `remove_orphan_files` so that files uniquely referenced by expired snapshots are cleaned up first +- Use `dry_run = 1` to preview orphan files before deletion +- The `older_than` threshold protects against deleting files from in-progress writes — the default 3-day threshold provides a generous safety margin +::: + ## Altinity Antalya branch ### Specify storage type in arguments diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index c9994afb205e..9ed1b0d93304 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -7754,6 +7754,12 @@ Allow to execute `insert` queries into iceberg. )", BETA, allow_experimental_insert_into_iceberg) \ DECLARE(Bool, allow_experimental_iceberg_compaction, false, R"( Allow to explicitly use 'OPTIMIZE' for iceberg tables. +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_iceberg_remove_orphan_files, false, R"( +Allow to use 'ALTER TABLE ... EXECUTE remove_orphan_files()' for iceberg tables. +)", EXPERIMENTAL) \ + DECLARE(UInt64, iceberg_orphan_files_older_than_seconds, 259200, R"( +Default age threshold in seconds for orphan file removal in Iceberg tables. Files newer than this are not considered orphans. Used when the older_than argument is omitted from the remove_orphan_files() procedure call. Default is 259200 (3 days). )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_expire_snapshots, false, R"( Allow to execute experimental Iceberg command `ALTER TABLE ... EXECUTE expire_snapshots`. diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index d785ceaf7f68..e4947bbfb2d3 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -42,6 +42,30 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() addSettingsChanges(settings_changes_history, "26.3.1.20001.altinityantalya", { {"object_storage_cluster_join_mode", "allow", "allow", "New setting"}, + {"allow_iceberg_remove_orphan_files", false, false, "New setting to gate Iceberg orphan file removal"}, + {"iceberg_orphan_files_older_than_seconds", 259200, 259200, "New setting for default orphan file age threshold"}, + {"output_format_arrow_unsupported_types_as_binary", false, true, "New setting to convert unsupported CH types to arrow binary instead of UNKNOWN_TYPE exception."}, + {"output_format_parquet_unsupported_types_as_binary", false, false, "New setting to convert unsupported CH types to parquet (arrow) binary instead of UNKNOWN_TYPE exception."}, + {"asterisk_include_virtual_columns", false, false, "New setting"}, + {"max_wkb_geometry_elements", 1'000'000, 1'000'000, "New setting to limit element counts in WKB geometry parsing, preventing excessive memory allocation on malformed data."}, + {"max_rand_distribution_trials", 1'000'000'000, 1'000'000'000, "New setting to limit trial counts in random distribution functions, preventing hangs with extreme inputs."}, + {"max_rand_distribution_parameter", 1e6, 1e6, "New setting to limit shape parameters in random distribution functions, preventing hangs with extreme inputs."}, + {"optimize_truncate_order_by_after_group_by_keys", false, true, "Remove trailing ORDER BY elements once all GROUP BY keys are covered in the ORDER BY prefix."}, + {"use_statistics_for_part_pruning", false, true, "New setting to use statistics for part pruning during query execution."}, + {"distributed_index_analysis_only_on_coordinator", false, false, "New setting."}, + {"query_plan_optimize_join_order_randomize", 0, 0, "New setting to randomize join order statistics for testing."}, + {"enable_materialized_cte", false, false, "New setting"}, + {"use_strict_insert_block_limits", false, false, "New setting to use strict min and max insert bounds on inserts. When min < max, max limits take precedence."}, + {"finalize_projection_parts_synchronously", false, false, "New setting to finalize projection parts synchronously during INSERT to reduce peak memory usage."}, + {"read_in_order_use_virtual_row_per_block", false, false, "Emit virtual row after each block during read-in-order to allow more frequent source reprioritization in MergingSortedTransform."}, + {"distributed_plan_prefer_replicas_over_workers", false, false, "New setting to serialize distributed plan for replicas"}, + {"use_text_index_like_evaluation_by_dictionary_scan", true, true, "New setting"}, + {"text_index_like_min_pattern_length", 4, 4, "New setting"}, + {"text_index_like_max_postings_to_read", 50, 50, "New setting"}, + {"analyzer_inline_views", false, false, "New setting"}, + {"highlight_max_matches_per_row", 10000, 10000, "New setting to limit the number of highlight matches per row to protect against excessive memory usage."}, + {"materialize_statistics_on_insert", true, false, "Disable building statistics on INSERT by default, rely on merges instead"}, + {"enable_join_transitive_predicates", false, false, "New setting to infer transitive equi-join predicates for join order optimization."}, }); addSettingsChanges(settings_changes_history, "26.3", { diff --git a/src/Databases/DataLake/GlueCatalog.cpp b/src/Databases/DataLake/GlueCatalog.cpp index 4d8bccf19fe8..047184a1ede3 100644 --- a/src/Databases/DataLake/GlueCatalog.cpp +++ b/src/Databases/DataLake/GlueCatalog.cpp @@ -38,19 +38,20 @@ #include -#include -#include -#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include #include #include +#include +#include +#include +#include +#include namespace DB::ErrorCodes { @@ -554,14 +555,7 @@ String GlueCatalog::resolveMetadataPathFromTableLocation(const String & table_lo try { auto [metadata_version, metadata_path, compression_method] = DB::Iceberg::getLatestOrExplicitMetadataFileAndVersion( - object_storage, - table_path, - *storage_settings, - nullptr, - getContext(), - log.get(), - std::nullopt - ); + object_storage, table_path, *storage_settings, nullptr, getContext(), log.get(), std::nullopt, DB::CompressionMethod::None); LOG_TRACE(log, "Resolved metadata path '{}' (version {}) for table location '{}'", metadata_path, metadata_version, table_location); diff --git a/src/Databases/enableAllExperimentalSettings.cpp b/src/Databases/enableAllExperimentalSettings.cpp index d1321689ead1..3804a27c37e8 100644 --- a/src/Databases/enableAllExperimentalSettings.cpp +++ b/src/Databases/enableAllExperimentalSettings.cpp @@ -59,6 +59,7 @@ void enableAllExperimentalSettings(ContextMutablePtr context) context->setSetting("allow_experimental_lightweight_update", 1); context->setSetting("allow_insert_into_iceberg", 1); context->setSetting("allow_experimental_iceberg_compaction", 1); + context->setSetting("allow_iceberg_remove_orphan_files", 1); context->setSetting("allow_experimental_expire_snapshots", 1); context->setSetting("allow_experimental_delta_lake_writes", 1); context->setSetting("allow_dynamic_type_in_join_keys", 1); diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 70088e13bcd3..7d0982e26a3f 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -380,7 +380,7 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran ObjectMetadata ReadBufferFromAzureBlobStorage::getObjectMetadataFromTheLastRequest() const { - if (last_object_metadata.get()->has_value()) + if (!last_object_metadata.get()->has_value()) throw Exception(ErrorCodes::NOT_INITIALIZED, "No Azure object metadata available because there were no successful requests"); return last_object_metadata.get()->value(); diff --git a/src/Interpreters/IcebergMetadataLog.cpp b/src/Interpreters/IcebergMetadataLog.cpp index c0552c14e0c2..2eece43962ed 100644 --- a/src/Interpreters/IcebergMetadataLog.cpp +++ b/src/Interpreters/IcebergMetadataLog.cpp @@ -84,7 +84,7 @@ void insertRowToLogTable( std::function get_row, IcebergMetadataLogLevel row_log_level, const String & table_path, - const String & file_path, + const Iceberg::IcebergPathFromMetadata & file_path, std::optional row_in_file, std::optional pruning_status) { @@ -108,7 +108,7 @@ void insertRowToLogTable( .query_id = local_context->getCurrentQueryId(), .content_type = row_log_level, .table_path = table_path, - .file_path = file_path, + .file_path = file_path.serialize(), .metadata_content = get_row(), .row_in_file = row_in_file, .pruning_status = pruning_status}); diff --git a/src/Interpreters/IcebergMetadataLog.h b/src/Interpreters/IcebergMetadataLog.h index be1114c4a847..cdad76443d52 100644 --- a/src/Interpreters/IcebergMetadataLog.h +++ b/src/Interpreters/IcebergMetadataLog.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -33,7 +34,7 @@ void insertRowToLogTable( std::function get_row, IcebergMetadataLogLevel row_log_level, const String & table_path, - const String & file_path, + const Iceberg::IcebergPathFromMetadata & file_path, std::optional row_in_file, std::optional pruning_status); diff --git a/src/Storages/ObjectStorage/DataLakes/Common/AvroForIcebergDeserializer.cpp b/src/Storages/ObjectStorage/DataLakes/Common/AvroForIcebergDeserializer.cpp index c27bb2ac7117..e3ca40b47599 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common/AvroForIcebergDeserializer.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Common/AvroForIcebergDeserializer.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -35,7 +36,9 @@ namespace DB::Iceberg using namespace DB; AvroForIcebergDeserializer::AvroForIcebergDeserializer( - std::unique_ptr buffer_, const std::string & manifest_file_path_, const DB::FormatSettings & format_settings) + std::unique_ptr buffer_, + const IcebergPathFromMetadata & manifest_file_path_, + const DB::FormatSettings & format_settings) try : buffer(std::move(buffer_)) , manifest_file_path(manifest_file_path_) @@ -156,7 +159,8 @@ ParsedManifestFileEntryPtr AvroForIcebergDeserializer::createParsedManifestFileE } - const auto file_path_key = getValueFromRowByName(row_index, c_data_file_file_path, TypeIndex::String).safeGet(); + const auto file_path_key = IcebergPathFromMetadata::deserialize( + getValueFromRowByName(row_index, c_data_file_file_path, TypeIndex::String).safeGet()); /// NOTE: This is weird, because in manifest file partition looks like this: /// { /// ... @@ -257,16 +261,18 @@ ParsedManifestFileEntryPtr AvroForIcebergDeserializer::createParsedManifestFileE } case FileContentType::POSITION_DELETE: { /// reference_file_path can be absent in schema for some reason, though it is present in specification: https://iceberg.apache.org/spec/#manifests - std::optional lower_reference_data_file_path = std::nullopt; - std::optional upper_reference_data_file_path = std::nullopt; + std::optional lower_reference_data_file_path; + std::optional upper_reference_data_file_path; bool bounds_set_by_referenced_data_file = false; if (hasPath(c_data_file_referenced_data_file)) { Field reference_file_path_field = getValueFromRowByName(row_index, c_data_file_referenced_data_file); if (!reference_file_path_field.isNull()) { - lower_reference_data_file_path = reference_file_path_field.safeGet(); - upper_reference_data_file_path = reference_file_path_field.safeGet(); + lower_reference_data_file_path.emplace( + Iceberg::IcebergPathFromMetadata::deserialize(reference_file_path_field.safeGet())); + upper_reference_data_file_path.emplace( + Iceberg::IcebergPathFromMetadata::deserialize(reference_file_path_field.safeGet())); bounds_set_by_referenced_data_file = true; } } @@ -277,9 +283,9 @@ ParsedManifestFileEntryPtr AvroForIcebergDeserializer::createParsedManifestFileE { auto & [lower, upper] = it->second; if (!lower.isNull()) - lower_reference_data_file_path = lower.safeGet(); + lower_reference_data_file_path.emplace(Iceberg::IcebergPathFromMetadata::deserialize(lower.safeGet())); if (!upper.isNull()) - upper_reference_data_file_path = upper.safeGet(); + upper_reference_data_file_path.emplace(Iceberg::IcebergPathFromMetadata::deserialize(upper.safeGet())); } } return std::make_shared( diff --git a/src/Storages/ObjectStorage/DataLakes/Common/AvroForIcebergDeserializer.h b/src/Storages/ObjectStorage/DataLakes/Common/AvroForIcebergDeserializer.h index 31719fb7ca05..9b1e080cd890 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common/AvroForIcebergDeserializer.h +++ b/src/Storages/ObjectStorage/DataLakes/Common/AvroForIcebergDeserializer.h @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -36,7 +37,7 @@ class AvroForIcebergDeserializer { private: std::unique_ptr buffer; - std::string manifest_file_path; + Iceberg::IcebergPathFromMetadata manifest_file_path; DB::ColumnPtr parsed_column; std::shared_ptr parsed_column_data_type; mutable std::optional cache_parsed_columns TSA_GUARDED_BY(cache_mutex); @@ -61,7 +62,7 @@ class AvroForIcebergDeserializer public: AvroForIcebergDeserializer( std::unique_ptr buffer_, - const std::string & manifest_file_path_, + const Iceberg::IcebergPathFromMetadata & manifest_file_path_, const DB::FormatSettings & format_settings); size_t rows() const; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Compaction.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/Compaction.cpp index a41006d83afa..4e0c2380b66d 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Compaction.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Compaction.cpp @@ -44,11 +44,11 @@ struct ManifestFilePlan { } - String path; - std::vector manifest_lists_path; + Iceberg::IcebergPathFromMetadata path; + std::vector manifest_lists_path; DataFileStatistics statistics; - FileNamesGenerator::Result patched_path; + Iceberg::IcebergPathFromMetadata patched_path; }; struct DataFilePlan @@ -56,7 +56,7 @@ struct DataFilePlan IcebergDataObjectInfoPtr data_object_info; std::shared_ptr manifest_list; - FileNamesGenerator::Result patched_path; + Iceberg::IcebergPathFromMetadata patched_path; UInt64 new_records_count = 0; }; @@ -68,10 +68,10 @@ struct Plan using PartitionPlan = std::vector>; std::vector partitions; IcebergHistory history; - std::unordered_map manifest_file_to_first_snapshot; - std::unordered_map> manifest_list_to_manifest_files; + std::unordered_map manifest_file_to_first_snapshot; + std::unordered_map> manifest_list_to_manifest_files; std::unordered_map>> snapshot_id_to_data_files; - std::unordered_map> path_to_data_file; + std::unordered_map> path_to_data_file; FileNamesGenerator generator; Poco::JSON::Object::Ptr initial_metadata_object; @@ -120,8 +120,7 @@ Plan getPlan( LoggerPtr log = getLogger("IcebergCompaction::getPlan"); Plan plan; - plan.generator = FileNamesGenerator( - persistent_table_components.table_path, persistent_table_components.table_path, false, compression_method, write_format); + plan.generator = FileNamesGenerator(persistent_table_components.path_resolver.getTableLocation(), false, compression_method, write_format); const auto [metadata_version, metadata_file_path, _] = getLatestOrExplicitMetadataFileAndVersion( object_storage, @@ -130,7 +129,8 @@ Plan getPlan( persistent_table_components.metadata_cache, context, log.get(), - persistent_table_components.table_uuid); + persistent_table_components.table_uuid, + persistent_table_components.metadata_compression_method); Poco::JSON::Object::Ptr initial_metadata_object = getMetadataJSONObject(metadata_file_path, object_storage, persistent_table_components.metadata_cache, context, log, compression_method, persistent_table_components.table_uuid); @@ -152,7 +152,7 @@ Plan getPlan( plan.initial_metadata_object = initial_metadata_object; std::vector all_positional_delete_files; - std::unordered_map> manifest_files; + std::unordered_map> manifest_files; for (const auto & snapshot : snapshots_info) { auto manifest_list = getManifestList(object_storage, persistent_table_components, context, snapshot.manifest_list_path, log); @@ -179,7 +179,8 @@ Plan getPlan( if (plan.partitions.size() <= partition_index) plan.partitions.push_back({}); - IcebergDataObjectInfoPtr data_object_info = std::make_shared(data_file, 0); + IcebergDataObjectInfoPtr data_object_info = std::make_shared( + data_file, persistent_table_components.path_resolver.resolve(data_file->parsed_entry->file_path_key), 0); std::shared_ptr data_file_ptr; if (!plan.path_to_data_file.contains(manifest_file.manifest_file_path)) { @@ -209,7 +210,8 @@ Plan getPlan( for (auto & data_file : plan.partitions[partition_index]) { if (data_file->data_object_info->info.sequence_number <= delete_file->sequence_number) - data_file->data_object_info->addPositionDeleteObject(delete_file); + data_file->data_object_info->addPositionDeleteObject( + delete_file, persistent_table_components.path_resolver.resolve(delete_file->parsed_entry->file_path_key)); } } plan.history = std::move(snapshots_info); @@ -221,6 +223,7 @@ static void writeDataFiles( Plan & initial_plan, SharedHeader sample_block, ObjectStoragePtr object_storage, + const IcebergPathResolver & path_resolver, const std::optional & format_settings, ContextPtr context, const String & write_format, @@ -259,7 +262,7 @@ static void writeDataFiles( false); auto write_buffer = object_storage->writeObject( - StoredObject(data_file->patched_path.path_in_storage), + StoredObject(path_resolver.resolve(data_file->patched_path)), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, @@ -293,7 +296,7 @@ static void writeDataFiles( } void writeMetadataFiles( - Plan & plan, ObjectStoragePtr object_storage, ContextPtr context, SharedHeader sample_block_, String write_format, String table_path) + Plan & plan, const IcebergPathResolver & path_resolver, ObjectStoragePtr object_storage, ContextPtr context, SharedHeader sample_block_, String write_format, String table_path) { auto log = getLogger("IcebergCompaction"); @@ -314,7 +317,7 @@ void writeMetadataFiles( MetadataGenerator metadata_generator(metadata_object); std::vector new_snapshots; - auto generated_metadata_name = plan.generator.generateMetadataName(); + auto generated_metadata_info = plan.generator.generateMetadataPathWithInfo(); std::unordered_map snapshot_id_to_snapshot; std::unordered_map snapshot_id_to_records_count; @@ -332,7 +335,7 @@ void writeMetadataFiles( auto new_snapshot = metadata_generator.generateNextMetadata( plan.generator, - generated_metadata_name.path_in_metadata, + generated_metadata_info.path, history_record.parent_id, history_record.added_files, total_records_count, @@ -348,11 +351,11 @@ void writeMetadataFiles( } Poco::JSON::Object::Ptr initial_metadata_object = plan.initial_metadata_object; - std::unordered_map manifest_file_renamings; - std::unordered_map manifest_file_sizes; + std::unordered_map manifest_file_renamings; + std::unordered_map manifest_file_sizes; { - std::unordered_map, std::unordered_set> grouped_by_manifest_files_result; + std::unordered_map, std::unordered_set> grouped_by_manifest_files_result; std::unordered_map, size_t> grouped_by_manifest_files_partitions; std::unordered_map, size_t> partition_values; @@ -362,7 +365,7 @@ void writeMetadataFiles( for (const auto & data_file : partition) { grouped_by_manifest_files_partitions[data_file->manifest_list] = i; - grouped_by_manifest_files_result[data_file->manifest_list].insert(data_file->patched_path.path_in_metadata); + grouped_by_manifest_files_result[data_file->manifest_list].insert(data_file->patched_path); partition_values[data_file->manifest_list] = i; } } @@ -391,9 +394,9 @@ void writeMetadataFiles( for (auto & [manifest_entry, data_filenames] : grouped_by_manifest_files_result) { manifest_entry->patched_path = plan.generator.generateManifestEntryName(); - manifest_file_renamings[manifest_entry->path] = manifest_entry->patched_path.path_in_metadata; + manifest_file_renamings[manifest_entry->path] = manifest_entry->patched_path; auto buffer_manifest_entry = object_storage->writeObject( - StoredObject(manifest_entry->patched_path.path_in_storage), + StoredObject(path_resolver.resolve(manifest_entry->patched_path)), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, @@ -419,18 +422,25 @@ void writeMetadataFiles( *buffer_manifest_entry, Iceberg::FileContentType::DATA); - manifest_file_sizes[manifest_entry->patched_path.path_in_metadata] += buffer_manifest_entry->count(); buffer_manifest_entry->finalize(); + auto manifest_bytes = buffer_manifest_entry->count(); + if (manifest_bytes == 0) + { + auto file_metadata = object_storage->getObjectMetadata( + path_resolver.resolve(manifest_entry->patched_path), /*with_tags=*/ false); + manifest_bytes = file_metadata.size_bytes; + } + manifest_file_sizes[manifest_entry->patched_path] += manifest_bytes; } } - std::unordered_map manifest_list_renamings; + std::unordered_map manifest_list_renamings; for (size_t i = 0; i < plan.history.size(); ++i) { if (plan.history[i].added_files == 0) continue; - manifest_list_renamings[plan.history[i].manifest_list_path] = new_snapshots[i].metadata_path; + manifest_list_renamings[plan.history[i].manifest_list_path] = new_snapshots[i].manifest_list_path; } for (size_t i = 0; i < plan.history.size(); ++i) @@ -441,27 +451,32 @@ void writeMetadataFiles( auto initial_manifest_list_name = plan.history[i].manifest_list_path; auto initial_manifest_entries = plan.manifest_list_to_manifest_files[initial_manifest_list_name]; auto renamed_manifest_list = manifest_list_renamings[initial_manifest_list_name]; - std::vector renamed_manifest_entries; - Int32 total_manifest_file_sizes = 0; + std::vector renamed_manifest_entries; for (const auto & initial_manifest_entry : initial_manifest_entries) { auto renamed_manifest_entry = manifest_file_renamings[initial_manifest_entry]; if (!renamed_manifest_entry.empty()) { renamed_manifest_entries.push_back(renamed_manifest_entry); - total_manifest_file_sizes += manifest_file_sizes[renamed_manifest_entry]; } } + std::vector per_manifest_sizes; + for (const auto & entry : renamed_manifest_entries) + per_manifest_sizes.push_back(manifest_file_sizes[entry]); auto buffer_manifest_list = object_storage->writeObject( - StoredObject(renamed_manifest_list), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + StoredObject(path_resolver.resolve(renamed_manifest_list)), + WriteMode::Rewrite, + std::nullopt, + DBMS_DEFAULT_BUFFER_SIZE, + context->getWriteSettings()); generateManifestList( - plan.generator, + path_resolver, metadata_object, object_storage, context, renamed_manifest_entries, new_snapshots[i].snapshot, - total_manifest_file_sizes, + per_manifest_sizes, *buffer_manifest_list, Iceberg::FileContentType::DATA, false); @@ -474,7 +489,7 @@ void writeMetadataFiles( std::string json_representation = removeEscapedSlashes(oss.str()); auto buffer_metadata = object_storage->writeObject( - StoredObject(generated_metadata_name.path_in_storage), + StoredObject(path_resolver.resolve(generated_metadata_info.path)), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, @@ -529,11 +544,12 @@ void compactIcebergTable( plan, sample_block_, object_storage_, + persistent_table_components.path_resolver, format_settings_, context_, write_format, persistent_table_components.metadata_compression_method); - writeMetadataFiles(plan, object_storage_, context_, sample_block_, write_format, persistent_table_components.table_path); + writeMetadataFiles(plan, persistent_table_components.path_resolver, object_storage_, context_, sample_block_, write_format, persistent_table_components.table_path); clearOldFiles(object_storage_, old_files); } } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Constant.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Constant.h index 6f1e455e351a..469eab2ea214 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Constant.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Constant.h @@ -127,6 +127,8 @@ DEFINE_ICEBERG_FIELD_ALIAS(field_id, field-id); DEFINE_ICEBERG_FIELD_ALIAS(last_sequence_number, last-sequence-number); DEFINE_ICEBERG_FIELD_ALIAS(metadata_file, metadata-file); DEFINE_ICEBERG_FIELD_ALIAS(metadata_log, metadata-log); +DEFINE_ICEBERG_FIELD_ALIAS(partition_statistics, partition-statistics); +DEFINE_ICEBERG_FIELD_ALIAS(statistics_path, statistics-path); DEFINE_ICEBERG_FIELD_ALIAS(metadata_sequence_number, sequence-number); DEFINE_ICEBERG_FIELD_ALIAS(min_snapshots_to_keep, history.expire.min-snapshots-to-keep); DEFINE_ICEBERG_FIELD_ALIAS(max_snapshot_age_ms, history.expire.max-snapshot-age-ms); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteCommandArgs.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteCommandArgs.h new file mode 100644 index 000000000000..201848ba52d1 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteCommandArgs.h @@ -0,0 +1,244 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +/// Declarative argument parser for `ALTER TABLE ... EXECUTE command(args...)`. +/// +/// Supports: +/// - Named arguments: `command(key = 'value', flag = 1)` +/// - Positional arguments: `command('2026-01-01')` mapped by index to a name +/// - Mixed: `command('2026-01-01', dry_run = 1)` +/// - Defaults: filled in after parsing when an argument was not supplied +/// - Type validation: optional per-argument Field::Types::Which constraint +/// +/// Usage: +/// ExecuteCommandArgs schema("remove_orphan_files"); +/// schema.addPositional("older_than", Field::Types::String); +/// schema.addNamed("location", Field::Types::String); +/// schema.addNamed("dry_run", Field::Types::UInt64); +/// schema.addDefault("dry_run", Field(UInt64(0))); +/// +/// auto result = schema.parse(args_ast); +/// // result.getAs("older_than"), result.has("location"), etc. +/// +/// Args registered without a type accept any Field type (useful for +/// complex types like Array or values needing custom conversion): +/// schema.addNamed("snapshot_ids"); // no type check +/// +class ExecuteCommandArgs +{ +public: + /// Parsed result — a thin wrapper over the resolved name->value map. + class Result + { + public: + bool has(const String & name) const { return values.contains(name); } + + const Field & get(const String & name) const + { + auto it = values.find(name); + if (it == values.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument '{}' was not provided and has no default", name); + return it->second; + } + + template + T getAs(const String & name) const { return get(name).safeGet(); } + + void set(const String & name, Field value) { values[name] = std::move(value); } + + const std::unordered_map & all() const { return values; } + + private: + friend class ExecuteCommandArgs; + std::unordered_map values; + }; + + explicit ExecuteCommandArgs(String command_name_) : command_name(std::move(command_name_)) {} + + /// Register a positional argument with type validation. + void addPositional(const String & name, Field::Types::Which expected_type) + { + positional_names.push_back(name); + known_names.insert(name); + expected_types[name] = expected_type; + } + + /// Register a positional argument that accepts any Field type. + void addPositional(const String & name) + { + positional_names.push_back(name); + known_names.insert(name); + } + + /// Register a named-only argument with type validation. + void addNamed(const String & name, Field::Types::Which expected_type) + { + known_names.insert(name); + expected_types[name] = expected_type; + } + + /// Register a named-only argument that accepts any Field type. + void addNamed(const String & name) + { + known_names.insert(name); + } + + /// Register a default value for an argument (positional or named). + void addDefault(const String & name, Field value) + { + defaults[name] = std::move(value); + } + + /// Register a post-parse callback that runs after all arguments are resolved. + /// Use for derived defaults (e.g. computing a timestamp from a setting). + void addPostParse(std::function callback) + { + post_parse_callbacks.push_back(std::move(callback)); + } + + /// Parse the AST argument list. `args` may be nullptr (no arguments). + Result parse(const ASTPtr & args) const + { + Result result; + bool seen_named = false; + size_t positional_index = 0; + + if (args) + { + for (size_t i = 0; i < args->children.size(); ++i) + { + auto named = tryExtractNamedArg(args->children[i]); + if (named.has_value()) + { + seen_named = true; + const auto & [name, value] = *named; + setNamed(result, name, value); + } + else + { + if (seen_named) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Positional arguments must precede named arguments in {}()", command_name); + + const auto * literal = args->children[i]->as(); + if (!literal) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument {} to {}() must be a literal value", i + 1, command_name); + + setPositional(result, positional_index, literal->value); + ++positional_index; + } + } + } + + for (const auto & [name, default_value] : defaults) + { + if (!result.has(name)) + result.values[name] = default_value; + } + + for (const auto & callback : post_parse_callbacks) + callback(result); + + return result; + } + +private: + String command_name; + std::vector positional_names; + std::unordered_set known_names; + std::unordered_map expected_types; + std::unordered_map defaults; + std::vector> post_parse_callbacks; + + static std::optional> tryExtractNamedArg(const ASTPtr & node) + { + const auto * func = node->as(); + if (!func || func->name != "equals" || !func->arguments || func->arguments->children.size() != 2) + return std::nullopt; + + const auto * ident = func->arguments->children[0]->as(); + const auto * lit = func->arguments->children[1]->as(); + if (!ident || !lit) + return std::nullopt; + + return std::make_pair(ident->name(), lit->value); + } + + void setNamed(Result & result, const String & name, const Field & value) const + { + if (!known_names.contains(name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown parameter '{}' for {}", name, command_name); + + if (result.has(name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Duplicate parameter '{}' for {}()", name, command_name); + + auto it = expected_types.find(name); + if (it != expected_types.end()) + validateType(name, value, it->second); + + result.values[name] = value; + } + + void setPositional(Result & result, size_t index, const Field & value) const + { + if (index >= positional_names.size()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{} accepts at most {} positional argument(s)", command_name, positional_names.size()); + + const auto & name = positional_names[index]; + + if (result.has(name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Duplicate parameter '{}' for {}(): specified both as positional and named argument", name, command_name); + + auto it = expected_types.find(name); + if (it != expected_types.end()) + validateType(name, value, it->second); + + result.values[name] = value; + } + + void validateType(const String & name, const Field & value, Field::Types::Which expected) const + { + if (value.getType() != expected) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Parameter '{}' for {} has wrong type (got {}, expected {})", + name, command_name, value.getTypeName(), fieldTypeToString(expected)); + } + + static const char * fieldTypeToString(Field::Types::Which type) + { + switch (type) + { + case Field::Types::Null: return "Null"; + case Field::Types::UInt64: return "UInt64"; + case Field::Types::Int64: return "Int64"; + case Field::Types::Float64: return "Float64"; + case Field::Types::String: return "String"; + case Field::Types::Array: return "Array"; + default: return "Unknown"; + } + } +}; + +} diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteOptionsParser.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteOptionsParser.cpp deleted file mode 100644 index 50c7a46cd1f9..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteOptionsParser.cpp +++ /dev/null @@ -1,102 +0,0 @@ -#include "config.h" -#if USE_AVRO - -#include - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - -namespace Iceberg -{ - -ExpireSnapshotsOptions parseExpireSnapshotsOptions(const ASTPtr & args, ContextPtr context) -{ - static constexpr std::string_view cmd = "expire_snapshots"; - ExpireSnapshotsOptions options; - - if (!args) - return options; - - ASTs all_args = args->children; - auto first_kv_it = getFirstKeyValueArgument(all_args); - size_t pos_count = static_cast(std::distance(all_args.begin(), first_kv_it)); - - if (pos_count > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots expects at most 1 positional argument, got {}", pos_count); - - if (pos_count == 1) - { - auto * lit = all_args[0]->as(); - String timestamp = lit ? lit->value.safeGet() : all_args[0]->getColumnName(); - ReadBufferFromString buf(timestamp); - time_t expire_time; - readDateTimeText(expire_time, buf); - options.expire_before_ms = static_cast(expire_time) * 1000; - } - - ASTs kv_args(first_kv_it, all_args.end()); - auto parsed_kv = parseKeyValueArguments(kv_args, context); - - for (const auto & [key, value] : parsed_kv) - { - if (key == "expire_before") - { - if (options.expire_before_ms.has_value()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "expire_snapshots: 'expire_before' specified both as positional argument and named argument"); - if (value.getType() != Field::Types::String) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots expects 'expire_before' to be a datetime string"); - String timestamp = value.safeGet(); - ReadBufferFromString buf(timestamp); - time_t expire_time; - readDateTimeText(expire_time, buf); - options.expire_before_ms = static_cast(expire_time) * 1000; - } - else if (key == "retention_period") - options.retention_period_ms = fieldToPeriodMs(value, cmd, key); - else if (key == "retain_last") - { - Int64 retain_last = fieldToInt64(value, cmd, key); - if (retain_last <= 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots expects 'retain_last' to be positive"); - if (retain_last > std::numeric_limits::max()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots 'retain_last' is too large: {}", retain_last); - options.retain_last = static_cast(retain_last); - } - else if (key == "snapshot_ids") - options.snapshot_ids = fieldToInt64Array(value, cmd, key); - else if (key == "dry_run") - options.dry_run = fieldToBool(value, cmd, key); - else - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Unknown expire_snapshots argument '{}'. Supported: expire_before, retention_period, retain_last, snapshot_ids, dry_run", - key); - } - - if (options.snapshot_ids && (options.retention_period_ms || options.retain_last)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "expire_snapshots argument 'snapshot_ids' cannot be combined with 'retention_period' or 'retain_last'"); - - return options; -} - -} -} - -#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteOptionsParser.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteOptionsParser.h deleted file mode 100644 index d47f3bed128f..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ExecuteOptionsParser.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AVRO - -#include -#include -#include - -namespace DB::Iceberg -{ - -ExpireSnapshotsOptions parseExpireSnapshotsOptions(const ASTPtr & args, ContextPtr context); - -} - -#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ExpireSnapshotsExecute.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ExpireSnapshotsExecute.cpp new file mode 100644 index 000000000000..7c8fea5f6379 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ExpireSnapshotsExecute.cpp @@ -0,0 +1,855 @@ +#include "config.h" +#if USE_AVRO + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +extern const int LIMIT_EXCEEDED; +} + +namespace DataLakeStorageSetting +{ +extern const DataLakeStorageSettingsBool iceberg_use_version_hint; +} + +namespace Setting +{ +extern const SettingsInt64 iceberg_expire_default_min_snapshots_to_keep; +extern const SettingsInt64 iceberg_expire_default_max_snapshot_age_ms; +extern const SettingsInt64 iceberg_expire_default_max_ref_age_ms; +} + +namespace Iceberg +{ + +namespace +{ + +constexpr auto MAX_TRANSACTION_RETRIES = 1000; + +// --------------------------------------------------------------------------- +// Argument parsing +// --------------------------------------------------------------------------- + +ExecuteCommandArgs makeSchema() +{ + ExecuteCommandArgs schema("expire_snapshots"); + schema.addPositional("expire_before", Field::Types::String); + schema.addNamed("retention_period"); + schema.addNamed("retain_last"); + schema.addNamed("snapshot_ids"); + schema.addNamed("dry_run"); + schema.addDefault("dry_run", Field(UInt64(0))); + return schema; +} + +ExpireSnapshotsOptions buildOptions(const ExecuteCommandArgs::Result & parsed) +{ + ExpireSnapshotsOptions options; + static constexpr std::string_view cmd = "expire_snapshots"; + + if (parsed.has("expire_before")) + { + String ts = parsed.getAs("expire_before"); + ReadBufferFromString buf(ts); + time_t expire_time; + readDateTimeText(expire_time, buf); + options.expire_before_ms = static_cast(expire_time) * 1000; + } + + if (parsed.has("retention_period")) + options.retention_period_ms = fieldToPeriodMs(parsed.get("retention_period"), cmd, "retention_period"); + + if (parsed.has("retain_last")) + { + Int64 retain_last = fieldToInt64(parsed.get("retain_last"), cmd, "retain_last"); + if (retain_last <= 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots expects 'retain_last' to be positive"); + if (retain_last > std::numeric_limits::max()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots 'retain_last' is too large: {}", retain_last); + options.retain_last = static_cast(retain_last); + } + + if (parsed.has("snapshot_ids")) + options.snapshot_ids = fieldToInt64Array(parsed.get("snapshot_ids"), cmd, "snapshot_ids"); + + if (parsed.has("dry_run")) + options.dry_run = fieldToBool(parsed.get("dry_run"), cmd, "dry_run"); + + if (options.snapshot_ids && (options.retention_period_ms || options.retain_last)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "expire_snapshots argument 'snapshot_ids' cannot be combined with 'retention_period' or 'retain_last'"); + + return options; +} + +Pipe resultToPipe(const ExpireSnapshotsResult & result) +{ + Block header{ + ColumnWithTypeAndName(std::make_shared(), "metric_name"), + ColumnWithTypeAndName(std::make_shared(), "metric_value"), + }; + + MutableColumns columns = header.cloneEmptyColumns(); + + auto add = [&](const char * name, Int64 value) + { + columns[0]->insert(String(name)); + columns[1]->insert(value); + }; + + add("deleted_data_files_count", result.deleted_data_files_count); + add("deleted_position_delete_files_count", result.deleted_position_delete_files_count); + add("deleted_equality_delete_files_count", result.deleted_equality_delete_files_count); + add("deleted_manifest_files_count", result.deleted_manifest_files_count); + add("deleted_manifest_lists_count", result.deleted_manifest_lists_count); + add("deleted_statistics_files_count", result.deleted_statistics_files_count); + add("dry_run", result.dry_run ? 1 : 0); + + const size_t rows = columns[0]->size(); + Chunk chunk(std::move(columns), rows); + return Pipe(std::make_shared(std::make_shared(std::move(header)), std::move(chunk))); +} + +// --------------------------------------------------------------------------- +// Retention policy +// --------------------------------------------------------------------------- + +struct RetentionPolicy +{ + Int32 min_snapshots_to_keep = Iceberg::default_min_snapshots_to_keep; + Int64 max_snapshot_age_ms = Iceberg::default_max_snapshot_age_ms; + Int64 max_ref_age_ms = Iceberg::default_max_ref_age_ms; +}; + +RetentionPolicy readRetentionPolicy(const Poco::JSON::Object::Ptr & metadata, ContextPtr context, const ExpireSnapshotsOptions & options) +{ + RetentionPolicy policy; + const auto & settings = context->getSettingsRef(); + Int64 min_keep_from_settings = settings[Setting::iceberg_expire_default_min_snapshots_to_keep].value; + Int64 max_snapshot_age_from_settings = settings[Setting::iceberg_expire_default_max_snapshot_age_ms].value; + Int64 max_ref_age_from_settings = settings[Setting::iceberg_expire_default_max_ref_age_ms].value; + + if (min_keep_from_settings <= 0 || min_keep_from_settings > std::numeric_limits::max()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "iceberg_expire_default_min_snapshots_to_keep must be in range [1, {}], got {}", + std::numeric_limits::max(), + min_keep_from_settings); + if (max_snapshot_age_from_settings < 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "iceberg_expire_default_max_snapshot_age_ms must be non-negative, got {}", + max_snapshot_age_from_settings); + if (max_ref_age_from_settings < 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "iceberg_expire_default_max_ref_age_ms must be non-negative, got {}", + max_ref_age_from_settings); + + policy.min_snapshots_to_keep = static_cast(min_keep_from_settings); + policy.max_snapshot_age_ms = max_snapshot_age_from_settings; + policy.max_ref_age_ms = max_ref_age_from_settings; + + if (metadata->has(Iceberg::f_properties)) + { + auto props = metadata->getObject(Iceberg::f_properties); + if (props->has(Iceberg::f_min_snapshots_to_keep)) + policy.min_snapshots_to_keep = std::stoi(props->getValue(Iceberg::f_min_snapshots_to_keep)); + if (props->has(Iceberg::f_max_snapshot_age_ms)) + policy.max_snapshot_age_ms = std::stoll(props->getValue(Iceberg::f_max_snapshot_age_ms)); + if (props->has(Iceberg::f_max_ref_age_ms)) + policy.max_ref_age_ms = std::stoll(props->getValue(Iceberg::f_max_ref_age_ms)); + } + + if (options.retain_last.has_value()) + policy.min_snapshots_to_keep = *options.retain_last; + if (options.retention_period_ms.has_value()) + policy.max_snapshot_age_ms = *options.retention_period_ms; + + return policy; +} + +// --------------------------------------------------------------------------- +// Snapshot graph + retention policy application +// --------------------------------------------------------------------------- + +class SnapshotGraph +{ +public: + explicit SnapshotGraph(const Poco::JSON::Array::Ptr & snapshots) + { + for (UInt32 i = 0; i < snapshots->size(); ++i) + { + auto snapshot = snapshots->getObject(i); + Int64 snap_id = snapshot->getValue(Iceberg::f_metadata_snapshot_id); + timestamps[snap_id] = snapshot->getValue(Iceberg::f_timestamp_ms); + if (snapshot->has(Iceberg::f_parent_snapshot_id) && !snapshot->isNull(Iceberg::f_parent_snapshot_id)) + parent_chain[snap_id] = snapshot->getValue(Iceberg::f_parent_snapshot_id); + } + } + + bool hasSnapshot(Int64 snap_id) const { return timestamps.contains(snap_id); } + + Int64 getTimestamp(Int64 snap_id) const { return timestamps.at(snap_id); } + + std::optional getParent(Int64 snap_id) const + { + auto it = parent_chain.find(snap_id); + return it != parent_chain.end() ? std::optional(it->second) : std::nullopt; + } + + void walkBranchAncestors(Int64 now_ms, Int64 head_id, Int32 min_keep, Int64 max_age_ms, std::set & retained) const + { + Int64 walk_id = head_id; + Int32 count = 0; + while (hasSnapshot(walk_id)) + { + bool within_min_keep = (count < min_keep); + bool within_max_age = (now_ms - getTimestamp(walk_id) <= max_age_ms); + if (!within_min_keep && !within_max_age) + break; + retained.insert(walk_id); + ++count; + auto parent = getParent(walk_id); + if (!parent) + break; + walk_id = *parent; + } + } + +private: + std::unordered_map parent_chain; + std::unordered_map timestamps; +}; + +std::pair, Strings> applyRetentionPolicy( + const Poco::JSON::Object::Ptr & metadata, + Int64 current_snapshot_id, + const SnapshotGraph & graph, + const RetentionPolicy & policy, + Int64 now_ms) +{ + std::set retained; + Strings expired_ref_names; + bool main_branch_walked = false; + if (metadata->has(Iceberg::f_refs)) + { + auto refs = metadata->getObject(Iceberg::f_refs); + for (const auto & ref_name : refs->getNames()) + { + auto ref_obj = refs->getObject(ref_name); + Int64 ref_snap_id = ref_obj->getValue(Iceberg::f_metadata_snapshot_id); + String ref_type = ref_obj->getValue(Iceberg::f_type); + + Int64 ref_max_ref_age = ref_obj->has(Iceberg::f_ref_max_ref_age_ms) + ? ref_obj->getValue(Iceberg::f_ref_max_ref_age_ms) + : policy.max_ref_age_ms; + + bool is_main = (ref_name == Iceberg::f_main); + + if (!graph.hasSnapshot(ref_snap_id)) + { + if (!is_main) + { + LOG_WARNING(getLogger("IcebergExpireSnapshots"), + "Removing invalid ref {}: snapshot {} does not exist", ref_name, ref_snap_id); + expired_ref_names.push_back(ref_name); + } + else + { + LOG_WARNING(getLogger("IcebergExpireSnapshots"), + "Main ref points to missing snapshot {}; falling back to current_snapshot_id walk", ref_snap_id); + } + continue; + } + + bool ref_expired = !is_main && (now_ms - graph.getTimestamp(ref_snap_id)) > ref_max_ref_age; + + if (ref_expired) + { + expired_ref_names.push_back(ref_name); + continue; + } + + if (ref_type == Iceberg::f_branch) + { + Int32 min_keep = ref_obj->has(Iceberg::f_ref_min_snapshots_to_keep) + ? ref_obj->getValue(Iceberg::f_ref_min_snapshots_to_keep) + : policy.min_snapshots_to_keep; + Int64 max_age = ref_obj->has(Iceberg::f_ref_max_snapshot_age_ms) + ? ref_obj->getValue(Iceberg::f_ref_max_snapshot_age_ms) + : policy.max_snapshot_age_ms; + graph.walkBranchAncestors(now_ms, ref_snap_id, min_keep, max_age, retained); + if (is_main) + main_branch_walked = true; + } + else if (ref_type == Iceberg::f_tag) + { + retained.insert(ref_snap_id); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "expire_snapshots: unexpected ref type '{}' for ref '{}'", ref_type, ref_name); + } + } + } + + if (!main_branch_walked) + graph.walkBranchAncestors(now_ms, current_snapshot_id, policy.min_snapshots_to_keep, policy.max_snapshot_age_ms, retained); + + return {retained, expired_ref_names}; +} + +// --------------------------------------------------------------------------- +// File collection helpers +// --------------------------------------------------------------------------- + +void collectAllFilePaths( + const Iceberg::ManifestFileIterator::ManifestFileEntriesHandle & entries_handle, + std::set & out) +{ + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::DATA)) + out.insert(entry->parsed_entry->file_path_key); + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::POSITION_DELETE)) + out.insert(entry->parsed_entry->file_path_key); + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::EQUALITY_DELETE)) + out.insert(entry->parsed_entry->file_path_key); +} + +void collectRetainedFiles( + const Poco::JSON::Array::Ptr & retained_snapshots, + ObjectStoragePtr object_storage, + const PersistentTableComponents & persistent_table_components, + ContextPtr context, + LoggerPtr log, + Int32 current_schema_id, + std::set & retained_manifest_paths, + std::set & retained_data_file_paths, + std::set & retained_manifest_list_paths) +{ + for (UInt32 i = 0; i < retained_snapshots->size(); ++i) + { + auto snapshot = retained_snapshots->getObject(i); + if (!snapshot->has(Iceberg::f_manifest_list)) + continue; + + auto manifest_list_path = IcebergPathFromMetadata::deserialize(snapshot->getValue(Iceberg::f_manifest_list)); + retained_manifest_list_paths.insert(manifest_list_path); + + auto manifest_keys = getManifestList(object_storage, persistent_table_components, context, manifest_list_path, log); + + for (const auto & manifest_entry : manifest_keys) + { + retained_manifest_paths.insert(manifest_entry.manifest_file_path); + auto entries_handle = getManifestFileEntriesHandle( + object_storage, persistent_table_components, context, log, + manifest_entry, current_schema_id); + collectAllFilePaths(entries_handle, retained_data_file_paths); + } + } +} + +struct ExpiredFiles +{ + std::vector all_paths; + Int64 data_files = 0; + Int64 position_delete_files = 0; + Int64 equality_delete_files = 0; + Int64 manifest_files = 0; + Int64 manifest_lists = 0; +}; + +ExpiredFiles collectExpiredFiles( + const std::vector & expired_manifest_list_paths, + const std::set & retained_manifest_list_paths, + const std::set & retained_manifest_paths, + const std::set & retained_data_file_paths, + ObjectStoragePtr object_storage, + const PersistentTableComponents & persistent_table_components, + ContextPtr context, + LoggerPtr log, + Int32 current_schema_id) +{ + ExpiredFiles result; + std::set seen_expired_manifest_list_paths; + std::set seen_expired_manifest_paths; + for (const auto & manifest_list_path : expired_manifest_list_paths) + { + if (retained_manifest_list_paths.contains(manifest_list_path)) + continue; + + if (seen_expired_manifest_list_paths.contains(manifest_list_path)) + continue; + + ManifestFileCacheKeys manifest_keys; + try + { + manifest_keys = getManifestList(object_storage, persistent_table_components, context, manifest_list_path, log); + } + catch (...) + { + LOG_WARNING(log, "Failed to read manifest list {}, skipping", manifest_list_path); + continue; + } + + for (const auto & manifest_entry : manifest_keys) + { + if (retained_manifest_paths.contains(manifest_entry.manifest_file_path)) + continue; + + if (seen_expired_manifest_paths.contains(manifest_entry.manifest_file_path)) + continue; + + try + { + auto entries_handle = getManifestFileEntriesHandle( + object_storage, persistent_table_components, context, log, + manifest_entry, current_schema_id); + + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::DATA)) + if (!retained_data_file_paths.contains(entry->parsed_entry->file_path_key)) + { + result.all_paths.push_back(entry->parsed_entry->file_path_key); + ++result.data_files; + } + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::POSITION_DELETE)) + if (!retained_data_file_paths.contains(entry->parsed_entry->file_path_key)) + { + result.all_paths.push_back(entry->parsed_entry->file_path_key); + ++result.position_delete_files; + } + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::EQUALITY_DELETE)) + if (!retained_data_file_paths.contains(entry->parsed_entry->file_path_key)) + { + result.all_paths.push_back(entry->parsed_entry->file_path_key); + ++result.equality_delete_files; + } + } + catch (...) + { + LOG_WARNING(log, "Failed to read manifest file {}, skipping", manifest_entry.manifest_file_path); + continue; + } + + seen_expired_manifest_paths.insert(manifest_entry.manifest_file_path); + result.all_paths.push_back(manifest_entry.manifest_file_path); + ++result.manifest_files; + } + + seen_expired_manifest_list_paths.insert(manifest_list_path); + result.all_paths.push_back(manifest_list_path); + ++result.manifest_lists; + } + return result; +} + +// --------------------------------------------------------------------------- +// Snapshot partitioning +// --------------------------------------------------------------------------- + +void trimSnapshotLog( + Poco::JSON::Object::Ptr metadata, + const std::set & expired_snapshot_ids) +{ + if (!metadata->has(Iceberg::f_snapshot_log)) + return; + + auto snapshot_log = metadata->get(Iceberg::f_snapshot_log).extract(); + Int32 suffix_start = static_cast(snapshot_log->size()); + for (Int32 j = static_cast(snapshot_log->size()) - 1; j >= 0; --j) + { + auto entry = snapshot_log->getObject(static_cast(j)); + Int64 snap_id = entry->getValue(Iceberg::f_metadata_snapshot_id); + if (expired_snapshot_ids.contains(snap_id)) + break; + suffix_start = j; + } + Poco::JSON::Array::Ptr retained_log = new Poco::JSON::Array; + for (UInt32 j = static_cast(suffix_start); j < snapshot_log->size(); ++j) + retained_log->add(snapshot_log->getObject(j)); + metadata->set(Iceberg::f_snapshot_log, retained_log); +} + +struct SnapshotPartition +{ + Poco::JSON::Array::Ptr retained_snapshots = new Poco::JSON::Array; + std::set expired_snapshot_ids; + std::vector expired_manifest_list_paths; +}; + +SnapshotPartition partitionSnapshots( + const Poco::JSON::Array::Ptr & snapshots, + const std::set & retention_retained_ids, + std::optional expire_before_ms) +{ + SnapshotPartition result; + for (UInt32 i = 0; i < snapshots->size(); ++i) + { + auto snapshot = snapshots->getObject(i); + Int64 snap_id = snapshot->getValue(Iceberg::f_metadata_snapshot_id); + Int64 snap_ts = snapshot->getValue(Iceberg::f_timestamp_ms); + + bool is_retained_by_policy = retention_retained_ids.contains(snap_id); + bool is_protected_by_fuse = expire_before_ms.has_value() && (snap_ts >= *expire_before_ms); + + if (is_retained_by_policy || is_protected_by_fuse) + { + result.retained_snapshots->add(snapshot); + } + else + { + result.expired_snapshot_ids.insert(snap_id); + if (snapshot->has(Iceberg::f_manifest_list)) + result.expired_manifest_list_paths.push_back( + Iceberg::IcebergPathFromMetadata::deserialize(snapshot->getValue(Iceberg::f_manifest_list))); + } + } + return result; +} + +SnapshotPartition partitionSnapshotsByIds( + const Poco::JSON::Object::Ptr & metadata, + const Poco::JSON::Array::Ptr & snapshots, + const std::vector & snapshot_ids, + Int64 current_snapshot_id, + std::optional expire_before_ms) +{ + std::unordered_set requested_ids(snapshot_ids.begin(), snapshot_ids.end()); + std::unordered_set existing_ids; + std::unordered_set ref_protected_ids; + SnapshotPartition result; + + if (metadata->has(Iceberg::f_refs)) + { + auto refs = metadata->getObject(Iceberg::f_refs); + for (const auto & ref_name : refs->getNames()) + { + auto ref = refs->getObject(ref_name); + if (ref->has(Iceberg::f_metadata_snapshot_id)) + ref_protected_ids.insert(ref->getValue(Iceberg::f_metadata_snapshot_id)); + } + } + + ref_protected_ids.insert(current_snapshot_id); + + for (UInt32 i = 0; i < snapshots->size(); ++i) + { + auto snapshot = snapshots->getObject(i); + Int64 snap_id = snapshot->getValue(Iceberg::f_metadata_snapshot_id); + Int64 snap_ts = snapshot->getValue(Iceberg::f_timestamp_ms); + + existing_ids.insert(snap_id); + bool requested = requested_ids.contains(snap_id); + bool is_protected_by_fuse = expire_before_ms.has_value() && (snap_ts >= *expire_before_ms); + + if (requested && ref_protected_ids.contains(snap_id)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "expire_snapshots cannot expire snapshot {} because it is referenced by current snapshot, branch, or tag", + snap_id); + + if (requested && !is_protected_by_fuse) + { + result.expired_snapshot_ids.insert(snap_id); + if (snapshot->has(Iceberg::f_manifest_list)) + result.expired_manifest_list_paths.push_back(Iceberg::IcebergPathFromMetadata::deserialize(snapshot->getValue(Iceberg::f_manifest_list))); + } + else + { + result.retained_snapshots->add(snapshot); + } + } + + for (Int64 requested_id : requested_ids) + { + if (!existing_ids.contains(requested_id)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots snapshot id {} does not exist", requested_id); + } + + return result; +} + +// --------------------------------------------------------------------------- +// Metadata mutation + file deletion +// --------------------------------------------------------------------------- + +void updateMetadataForExpiration( + Poco::JSON::Object::Ptr metadata, + const Strings & expired_ref_names, + const Poco::JSON::Array::Ptr & retained_snapshots, + const std::set & expired_snapshot_ids) +{ + for (const auto & ref_name : expired_ref_names) + metadata->getObject(Iceberg::f_refs)->remove(ref_name); + + metadata->set(Iceberg::f_snapshots, retained_snapshots); + trimSnapshotLog(metadata, expired_snapshot_ids); + + auto now = std::chrono::system_clock::now(); + auto ms = duration_cast(now.time_since_epoch()); + metadata->set(Iceberg::f_last_updated_ms, ms.count()); +} + +void deleteExpiredFiles( + const std::vector & files_to_delete, + const Iceberg::IcebergPathResolver & path_resolver, + ObjectStoragePtr object_storage, + LoggerPtr log) +{ + for (const auto & file_path : files_to_delete) + { + try + { + object_storage->removeObjectIfExists(StoredObject(path_resolver.resolve(file_path))); + LOG_DEBUG(log, "Deleted expired file {}", file_path); + } + catch (...) + { + LOG_WARNING(log, "Failed to delete file {}: {}", file_path, getCurrentExceptionMessage(false)); + } + } +} + +} + + +// --------------------------------------------------------------------------- +// Public: expireSnapshots orchestration +// --------------------------------------------------------------------------- + +ExpireSnapshotsResult expireSnapshots( + const ExpireSnapshotsOptions & options, + ContextPtr context, + ObjectStoragePtr object_storage, + const DataLakeStorageSettings & data_lake_settings, + const PersistentTableComponents & persistent_table_components, + const String & write_format, + std::shared_ptr catalog, + const String & table_name) +{ + auto common_path = persistent_table_components.table_path; + if (!common_path.starts_with('/')) + common_path = "/" + common_path; + + int max_retries = MAX_TRANSACTION_RETRIES; + while (--max_retries > 0) + { + FileNamesGenerator filename_generator(persistent_table_components.path_resolver.getTableLocation(), false, CompressionMethod::None, write_format); + auto log = getLogger("IcebergExpireSnapshots"); + auto [last_version, metadata_path, compression_method] = getLatestOrExplicitMetadataFileAndVersion( + object_storage, + persistent_table_components.table_path, + data_lake_settings, + persistent_table_components.metadata_cache, + context, + log.get(), + persistent_table_components.table_uuid, + persistent_table_components.metadata_compression_method, + /* force_fetch_latest_metadata */ true, + /* ignore_explicit_metadata_file_path */ true); + + filename_generator.setVersion(last_version + 1); + filename_generator.setCompressionMethod(compression_method); + + auto metadata = getMetadataJSONObject( + metadata_path, + object_storage, + persistent_table_components.metadata_cache, + context, + log, + compression_method, + persistent_table_components.table_uuid); + + if (metadata->getValue(f_format_version) < 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots is supported only for the second version of iceberg format"); + + if (!metadata->has(Iceberg::f_current_snapshot_id)) + { + LOG_INFO(log, "No snapshots to expire (table has no current snapshot)"); + return {.dry_run = options.dry_run}; + } + + Int64 current_snapshot_id = metadata->getValue(Iceberg::f_current_snapshot_id); + if (current_snapshot_id < 0) + { + LOG_INFO(log, "No snapshots to expire (table has no current snapshot)"); + return {.dry_run = options.dry_run}; + } + + auto snapshots = metadata->get(Iceberg::f_snapshots).extract(); + auto now_ms = duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + + Strings expired_ref_names; + SnapshotPartition partition; + if (options.snapshot_ids.has_value()) + { + partition = partitionSnapshotsByIds(metadata, snapshots, *options.snapshot_ids, current_snapshot_id, options.expire_before_ms); + } + else + { + auto policy = readRetentionPolicy(metadata, context, options); + SnapshotGraph graph(snapshots); + auto [retention_retained_ids, retention_expired_ref_names] = applyRetentionPolicy(metadata, current_snapshot_id, graph, policy, now_ms); + expired_ref_names = std::move(retention_expired_ref_names); + partition = partitionSnapshots(snapshots, retention_retained_ids, options.expire_before_ms); + } + + if (partition.expired_snapshot_ids.empty()) + { + LOG_INFO(log, "No snapshots to expire"); + return {.dry_run = options.dry_run}; + } + LOG_INFO(log, "Expiring {} snapshots", partition.expired_snapshot_ids.size()); + + Int32 current_schema_id = metadata->getValue(Iceberg::f_current_schema_id); + + std::set retained_manifest_paths; + std::set retained_data_file_paths; + std::set retained_manifest_list_paths; + collectRetainedFiles( + partition.retained_snapshots, object_storage, persistent_table_components, context, log, + current_schema_id, retained_manifest_paths, retained_data_file_paths, retained_manifest_list_paths); + auto expired_files = collectExpiredFiles( + partition.expired_manifest_list_paths, retained_manifest_list_paths, retained_manifest_paths, retained_data_file_paths, + object_storage, persistent_table_components, context, log, current_schema_id); + + if (options.dry_run) + { + LOG_INFO(log, "Dry-run mode: skip metadata commit and file deletion"); + return ExpireSnapshotsResult{ + .deleted_data_files_count = expired_files.data_files, + .deleted_position_delete_files_count = expired_files.position_delete_files, + .deleted_equality_delete_files_count = expired_files.equality_delete_files, + .deleted_manifest_files_count = expired_files.manifest_files, + .deleted_manifest_lists_count = expired_files.manifest_lists, + .dry_run = true, + }; + } + + updateMetadataForExpiration(metadata, expired_ref_names, partition.retained_snapshots, partition.expired_snapshot_ids); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + Poco::JSON::Stringifier::stringify(metadata, oss, 4); + std::string json_representation = removeEscapedSlashes(oss.str()); + auto metadata_info = filename_generator.generateMetadataPathWithInfo(); + auto hint_path = filename_generator.generateVersionHint(); + if (!writeMetadataFileAndVersionHint( + persistent_table_components.path_resolver, + metadata_info, + json_representation, + hint_path, + object_storage, + context, + data_lake_settings[DataLakeStorageSetting::iceberg_use_version_hint])) + { + LOG_WARNING(log, "Metadata commit conflict during expire_snapshots, retrying ({} retries left)", max_retries); + continue; + } + + if (catalog) + { + auto catalog_filename = persistent_table_components.path_resolver.resolveForCatalog(metadata_info.path); + const auto & [namespace_name, parsed_table_name] = DataLake::parseTableName(table_name); + if (!catalog->updateMetadata(namespace_name, parsed_table_name, catalog_filename, nullptr)) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Failed to update catalog metadata after writing new metadata file. " + "The table metadata may be in an inconsistent state"); + } + } + + LOG_INFO(log, "Deleting {} expired files for {} expired snapshots", expired_files.all_paths.size(), partition.expired_snapshot_ids.size()); + deleteExpiredFiles(expired_files.all_paths, persistent_table_components.path_resolver, object_storage, log); + LOG_INFO(log, "Expired {} snapshots, deleted {} files", partition.expired_snapshot_ids.size(), expired_files.all_paths.size()); + + return ExpireSnapshotsResult{ + .deleted_data_files_count = expired_files.data_files, + .deleted_position_delete_files_count = expired_files.position_delete_files, + .deleted_equality_delete_files_count = expired_files.equality_delete_files, + .deleted_manifest_files_count = expired_files.manifest_files, + .deleted_manifest_lists_count = expired_files.manifest_lists, + .dry_run = false, + }; + } + + if (max_retries == 0) + throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Too many unsuccessful retries to expire iceberg snapshots"); + + UNREACHABLE(); +} + + +// --------------------------------------------------------------------------- +// Public: executeExpireSnapshots (entry point from ALTER TABLE ... EXECUTE) +// --------------------------------------------------------------------------- + +Pipe executeExpireSnapshots( + const ASTPtr & args, + ContextPtr context, + ObjectStoragePtr object_storage, + const DataLakeStorageSettings & data_lake_settings, + const PersistentTableComponents & persistent_components, + const String & write_format, + std::shared_ptr catalog, + const String & table_name) +{ + auto parsed = makeSchema().parse(args); + auto options = buildOptions(parsed); + + auto result = expireSnapshots( + options, + context, + object_storage, + data_lake_settings, + persistent_components, + write_format, + catalog, + table_name); + + return resultToPipe(result); +} + +} +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ExpireSnapshotsExecute.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ExpireSnapshotsExecute.h new file mode 100644 index 000000000000..e3a682c6c56b --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ExpireSnapshotsExecute.h @@ -0,0 +1,39 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO + +#include +#include +#include +#include +#include +#include + +namespace DB::Iceberg +{ + +ExpireSnapshotsResult expireSnapshots( + const ExpireSnapshotsOptions & options, + ContextPtr context, + ObjectStoragePtr object_storage, + const DataLakeStorageSettings & data_lake_settings, + const PersistentTableComponents & persistent_table_components, + const String & write_format, + std::shared_ptr catalog, + const String & table_name); + +Pipe executeExpireSnapshots( + const ASTPtr & args, + ContextPtr context, + ObjectStoragePtr object_storage, + const DataLakeStorageSettings & data_lake_settings, + const PersistentTableComponents & persistent_components, + const String & write_format, + std::shared_ptr catalog, + const String & table_name); + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.cpp index 9c51d95cf805..b0b1c3e5ef68 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.cpp @@ -1,45 +1,32 @@ #include #include +#include #if USE_AVRO -namespace DB::ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - namespace DB { FileNamesGenerator::FileNamesGenerator( - const String & table_dir_, - const String & storage_dir_, + const String & table_location_, bool use_uuid_in_metadata_, CompressionMethod compression_method_, const String & format_name_) - : table_dir(table_dir_) - , storage_dir(storage_dir_) - , data_dir(table_dir + "data/") - , metadata_dir(table_dir + "metadata/") - , storage_data_dir(storage_dir + "data/") - , storage_metadata_dir(storage_dir + "metadata/") + : table_location(table_location_) , use_uuid_in_metadata(use_uuid_in_metadata_) , compression_method(compression_method_) , format_name(boost::to_lower_copy(format_name_)) { + /// Normalize: ensure table_location ends with '/' + if (!table_location.empty() && table_location.back() != '/') + table_location += '/'; } FileNamesGenerator::FileNamesGenerator(const FileNamesGenerator & other) { - data_dir = other.data_dir; - metadata_dir = other.metadata_dir; - storage_data_dir = other.storage_data_dir; - storage_metadata_dir = other.storage_metadata_dir; initial_version = other.initial_version; - - table_dir = other.table_dir; - storage_dir = other.storage_dir; + table_location = other.table_location; use_uuid_in_metadata = other.use_uuid_in_metadata; compression_method = other.compression_method; format_name = other.format_name; @@ -50,14 +37,8 @@ FileNamesGenerator & FileNamesGenerator::operator=(const FileNamesGenerator & ot if (this == &other) return *this; - data_dir = other.data_dir; - metadata_dir = other.metadata_dir; - storage_data_dir = other.storage_data_dir; - storage_metadata_dir = other.storage_metadata_dir; initial_version = other.initial_version; - - table_dir = other.table_dir; - storage_dir = other.storage_dir; + table_location = other.table_location; use_uuid_in_metadata = other.use_uuid_in_metadata; compression_method = other.compression_method; format_name = other.format_name; @@ -65,85 +46,58 @@ FileNamesGenerator & FileNamesGenerator::operator=(const FileNamesGenerator & ot return *this; } -FileNamesGenerator::Result FileNamesGenerator::generateDataFileName() +Iceberg::IcebergPathFromMetadata FileNamesGenerator::generateDataFileName() { auto uuid_str = uuid_generator.createRandom().toString(); - - return Result{ - .path_in_metadata = fmt::format("{}data-{}.{}", data_dir, uuid_str, format_name), - .path_in_storage = fmt::format("{}data-{}.{}", storage_data_dir, uuid_str, format_name) - }; + return Iceberg::IcebergPathFromMetadata(fmt::format("{}data/data-{}.{}", table_location, uuid_str, format_name)); } -FileNamesGenerator::Result FileNamesGenerator::generateManifestEntryName() +Iceberg::IcebergPathFromMetadata FileNamesGenerator::generateManifestEntryName() { auto uuid_str = uuid_generator.createRandom().toString(); - - return Result{ - .path_in_metadata = fmt::format("{}{}.avro", metadata_dir, uuid_str), - .path_in_storage = fmt::format("{}{}.avro", storage_metadata_dir, uuid_str), - }; + return Iceberg::IcebergPathFromMetadata(fmt::format("{}metadata/{}.avro", table_location, uuid_str)); } -FileNamesGenerator::Result FileNamesGenerator::generateManifestListName(Int64 snapshot_id, Int32 format_version) +Iceberg::IcebergPathFromMetadata FileNamesGenerator::generateManifestListName(Int64 snapshot_id, Int32 format_version) { auto uuid_str = uuid_generator.createRandom().toString(); - - return Result{ - .path_in_metadata = fmt::format("{}snap-{}-{}-{}.avro", metadata_dir, snapshot_id, format_version, uuid_str), - .path_in_storage = fmt::format("{}snap-{}-{}-{}.avro", storage_metadata_dir, snapshot_id, format_version, uuid_str), - }; + return Iceberg::IcebergPathFromMetadata(fmt::format("{}metadata/snap-{}-{}-{}.avro", table_location, snapshot_id, format_version, uuid_str)); } -FileNamesGenerator::Result FileNamesGenerator::generateMetadataName() +GeneratedMetadataFileWithInfo FileNamesGenerator::generateMetadataPathWithInfo() { auto compression_suffix = toContentEncodingName(compression_method); if (!compression_suffix.empty()) compression_suffix = "." + compression_suffix; + auto used_version = initial_version++; if (!use_uuid_in_metadata) { - auto res = Result{ - .path_in_metadata = fmt::format("{}v{}{}.metadata.json", metadata_dir, initial_version, compression_suffix), - .path_in_storage = fmt::format("{}v{}{}.metadata.json", storage_metadata_dir, initial_version, compression_suffix), - }; - initial_version++; - return res; + return GeneratedMetadataFileWithInfo{ + .path = Iceberg::IcebergPathFromMetadata( + fmt::format("{}metadata/v{}{}.metadata.json", table_location, used_version, compression_suffix)), + .version = used_version, + .compression_method = compression_method}; } else { auto uuid_str = uuid_generator.createRandom().toString(); - auto res = Result{ - .path_in_metadata = fmt::format("{}v{}-{}{}.metadata.json", metadata_dir, initial_version, uuid_str, compression_suffix), - .path_in_storage = fmt::format("{}v{}-{}{}.metadata.json", storage_metadata_dir, initial_version, uuid_str, compression_suffix), - }; - initial_version++; - return res; + return GeneratedMetadataFileWithInfo{ + .path = Iceberg::IcebergPathFromMetadata( + fmt::format("{}metadata/v{}-{}{}.metadata.json", table_location, used_version, uuid_str, compression_suffix)), + .version = used_version, + .compression_method = compression_method}; } } -FileNamesGenerator::Result FileNamesGenerator::generateVersionHint() +Iceberg::IcebergPathFromMetadata FileNamesGenerator::generateVersionHint() { - return Result{ - .path_in_metadata = fmt::format("{}version-hint.text", metadata_dir), - .path_in_storage = fmt::format("{}version-hint.text", storage_metadata_dir), - }; + return Iceberg::IcebergPathFromMetadata(fmt::format("{}metadata/version-hint.text", table_location)); } -FileNamesGenerator::Result FileNamesGenerator::generatePositionDeleteFile() +Iceberg::IcebergPathFromMetadata FileNamesGenerator::generatePositionDeleteFile() { auto uuid_str = uuid_generator.createRandom().toString(); - - return Result{ - .path_in_metadata = fmt::format("{}{}-deletes.{}", data_dir, uuid_str, format_name), - .path_in_storage = fmt::format("{}{}-deletes.{}", storage_data_dir, uuid_str, format_name) - }; -} - -String FileNamesGenerator::convertMetadataPathToStoragePath(const String & metadata_path) const -{ - if (!metadata_path.starts_with(table_dir)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Paths in Iceberg must use a consistent format — either /your/path or s3://your/path. Use the write_full_path_in_iceberg_metadata setting to control this behavior {} {}", metadata_path, table_dir); - return storage_dir + metadata_path.substr(table_dir.size()); + return Iceberg::IcebergPathFromMetadata(fmt::format("{}data/{}-deletes.{}", table_location, uuid_str, format_name)); } } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.h index 07dbe97cd6f9..b26a5c86ffd4 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.h @@ -1,7 +1,9 @@ #pragma once +#include "config.h" + #include -#include +#include #include @@ -10,24 +12,25 @@ namespace DB #if USE_AVRO +struct GeneratedMetadataFileWithInfo +{ + Iceberg::IcebergPathFromMetadata path; + Int32 version; + CompressionMethod compression_method; +}; + +/// Generates Iceberg metadata paths (IcebergPathFromMetadata) for new files. +/// +/// All generated paths use table_location as prefix, ensuring they are +/// always in the correct format for writing into Iceberg metadata files. +/// To get the actual storage path for I/O, pass the result through +/// IcebergPathResolver::resolve(). class FileNamesGenerator { public: - struct Result - { - /// Path recorded in the Iceberg metadata files. - /// If `write_full_path_in_iceberg_metadata` is disabled, it will be a simple relative path (e.g., /a/b/c.avro). - /// Otherwise, it will include a prefix indicating the file system type (e.g., s3://a/b/c.avro). - String path_in_metadata; - - /// Actual path to the object in the storage (e.g., /a/b/c.avro). - String path_in_storage; - }; - FileNamesGenerator() = default; explicit FileNamesGenerator( - const String & table_dir_, - const String & storage_dir_, + const String & table_location_, bool use_uuid_in_metadata_, CompressionMethod compression_method_, const String & format_name_); @@ -35,29 +38,24 @@ class FileNamesGenerator FileNamesGenerator(const FileNamesGenerator & other); FileNamesGenerator & operator=(const FileNamesGenerator & other); - Result generateDataFileName(); - Result generateManifestEntryName(); - Result generateManifestListName(Int64 snapshot_id, Int32 format_version); - Result generateMetadataName(); - Result generateVersionHint(); - Result generatePositionDeleteFile(); - - String convertMetadataPathToStoragePath(const String & metadata_path) const; + /// All generate* methods return IcebergPathFromMetadata. + /// These paths are ready to be written into Iceberg metadata files. + /// To get a storage path for actual I/O, use IcebergPathResolver::resolve(). + Iceberg::IcebergPathFromMetadata generateDataFileName(); + Iceberg::IcebergPathFromMetadata generateManifestEntryName(); + Iceberg::IcebergPathFromMetadata generateManifestListName(Int64 snapshot_id, Int32 format_version); + GeneratedMetadataFileWithInfo generateMetadataPathWithInfo(); + Iceberg::IcebergPathFromMetadata generateVersionHint(); + Iceberg::IcebergPathFromMetadata generatePositionDeleteFile(); void setVersion(Int32 initial_version_) { initial_version = initial_version_; } void setCompressionMethod(CompressionMethod compression_method_) { compression_method = compression_method_; } private: Poco::UUIDGenerator uuid_generator; - String table_dir; - String storage_dir; - - String data_dir; - String metadata_dir; - String storage_data_dir; - String storage_metadata_dir; - bool use_uuid_in_metadata; - CompressionMethod compression_method; + String table_location; + bool use_uuid_in_metadata = false; + CompressionMethod compression_method = CompressionMethod::None; String format_name; Int32 initial_version = 0; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataObjectInfo.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataObjectInfo.cpp index 9a963feb3d3f..e0d7d9197fb6 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataObjectInfo.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataObjectInfo.cpp @@ -34,8 +34,8 @@ extern const SettingsBool use_roaring_bitmap_iceberg_positional_deletes; #if USE_AVRO IcebergDataObjectInfo::IcebergDataObjectInfo( - Iceberg::ProcessedManifestFileEntryPtr data_manifest_file_entry_, Int32 schema_id_relevant_to_iterator_) - : ObjectInfo(RelativePathWithMetadata(data_manifest_file_entry_->file_path)) + Iceberg::ProcessedManifestFileEntryPtr data_manifest_file_entry_, const String & resolved_storage_path_, Int32 schema_id_relevant_to_iterator_) + : ObjectInfo(RelativePathWithMetadata(resolved_storage_path_)) , info{ data_manifest_file_entry_->parsed_entry->file_path_key, data_manifest_file_entry_->resolved_schema_id, @@ -66,7 +66,7 @@ std::shared_ptr IcebergDataObjectInfo::getPositionDeleteTransf return std::make_shared(header, self, object_storage, format_settings, parser_shared_resources, context_); } -void IcebergDataObjectInfo::addPositionDeleteObject(Iceberg::ProcessedManifestFileEntryPtr position_delete_object) +void IcebergDataObjectInfo::addPositionDeleteObject(Iceberg::ProcessedManifestFileEntryPtr position_delete_object, const String & resolved_storage_path) { if (Poco::toUpper(info.file_format) != "PARQUET") { @@ -76,13 +76,13 @@ void IcebergDataObjectInfo::addPositionDeleteObject(Iceberg::ProcessedManifestFi info.file_format); } info.position_deletes_objects.emplace_back( - position_delete_object->file_path, position_delete_object->parsed_entry->file_format, std::nullopt); + resolved_storage_path, position_delete_object->parsed_entry->file_format, std::nullopt); } -void IcebergDataObjectInfo::addEqualityDeleteObject(const Iceberg::ProcessedManifestFileEntryPtr & equality_delete_object) +void IcebergDataObjectInfo::addEqualityDeleteObject(const Iceberg::ProcessedManifestFileEntryPtr & equality_delete_object, const String & resolved_storage_path) { info.equality_deletes_objects.emplace_back( - equality_delete_object->file_path, + resolved_storage_path, equality_delete_object->parsed_entry->file_format, equality_delete_object->parsed_entry->equality_ids, equality_delete_object->resolved_schema_id); @@ -93,7 +93,7 @@ void IcebergDataObjectInfo::addEqualityDeleteObject(const Iceberg::ProcessedMani void IcebergObjectSerializableInfo::serializeForClusterFunctionProtocol(WriteBuffer & out, size_t protocol_version) const { checkVersion(protocol_version); - writeStringBinary(data_object_file_path_key, out); + writeStringBinary(data_object_file_path_key.serialize(), out); writeVarInt(underlying_format_read_schema_id, out); writeVarInt(schema_id_relevant_to_iterator, out); writeVarInt(sequence_number, out); @@ -142,7 +142,11 @@ void IcebergObjectSerializableInfo::serializeForClusterFunctionProtocol(WriteBuf void IcebergObjectSerializableInfo::deserializeForClusterFunctionProtocol(ReadBuffer & in, size_t protocol_version) { checkVersion(protocol_version); - readStringBinary(data_object_file_path_key, in); + { + String raw_path; + readStringBinary(raw_path, in); + data_object_file_path_key = IcebergPathFromMetadata::deserialize(std::move(raw_path)); + } readVarInt(underlying_format_read_schema_id, in); readVarInt(schema_id_relevant_to_iterator, in); readVarInt(sequence_number, in); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataObjectInfo.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataObjectInfo.h index ce8e551d6e32..8931f5d702df 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataObjectInfo.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataObjectInfo.h @@ -9,13 +9,14 @@ #include #include +#include namespace DB::Iceberg { struct IcebergObjectSerializableInfo { - String data_object_file_path_key; + IcebergPathFromMetadata data_object_file_path_key; Int32 underlying_format_read_schema_id; Int32 schema_id_relevant_to_iterator; Int64 sequence_number; @@ -47,7 +48,7 @@ struct IcebergDataObjectInfo : public ObjectInfo, std::enable_shared_from_this getFileFormat() const override { return info.file_format; } - void addPositionDeleteObject(Iceberg::ProcessedManifestFileEntryPtr position_delete_object); + void addPositionDeleteObject(Iceberg::ProcessedManifestFileEntryPtr position_delete_object, const String & resolved_storage_path); - void addEqualityDeleteObject(const Iceberg::ProcessedManifestFileEntryPtr & equality_delete_object); + void addEqualityDeleteObject(const Iceberg::ProcessedManifestFileEntryPtr & equality_delete_object, const String & resolved_storage_path); Iceberg::IcebergObjectSerializableInfo info; }; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergIterator.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergIterator.cpp index 97de3771264e..21b02a6a0ba4 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergIterator.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergIterator.cpp @@ -131,7 +131,7 @@ std::span defineDeletesSpan( "delete object info is {}", std::distance(beg_it, end_it), is_equality_delete ? "equality" : "position", - data_object_->file_path, + data_object_->parsed_entry->file_path_key, data_object_->dumpDeletesMatchingInfo(), (*beg_it)->dumpDeletesMatchingInfo(), (*previous_it)->dumpDeletesMatchingInfo()); @@ -142,7 +142,7 @@ std::span defineDeletesSpan( logger, "No {} delete elements for data file {}, taken data file object info: {}", is_equality_delete ? "equality" : "position", - data_object_->file_path, + data_object_->parsed_entry->file_path_key, data_object_->dumpDeletesMatchingInfo()); } return {beg_it, end_it}; @@ -173,8 +173,8 @@ std::optional SingleThreadIcebergKeysIterator::ne /// Find the next manifest file with matching content type. while (manifest_file_index < data_snapshot->manifest_list_entries.size()) { - const auto & mle = data_snapshot->manifest_list_entries[manifest_file_index++]; - if (mle.content_type != manifest_file_content_type) + const auto & manifest_list_entry = data_snapshot->manifest_list_entries[manifest_file_index++]; + if (manifest_list_entry.content_type != manifest_file_content_type) continue; auto manifest_file_cacheable_part = Iceberg::getManifestFile( @@ -182,20 +182,18 @@ std::optional SingleThreadIcebergKeysIterator::ne persistent_components, local_context, log, - mle.manifest_file_path, - mle.manifest_file_byte_size); + manifest_list_entry.manifest_file_path, + manifest_list_entry.manifest_file_byte_size); current_manifest_file_iterator = Iceberg::ManifestFileIterator::create( manifest_file_cacheable_part.deserializer, - mle.manifest_file_path, + manifest_list_entry.manifest_file_path, persistent_components.format_version, - persistent_components.table_path, + persistent_components.path_resolver, *persistent_components.schema_processor, - mle.added_sequence_number, - mle.added_snapshot_id, - persistent_components.table_location, + manifest_list_entry.added_sequence_number, + manifest_list_entry.added_snapshot_id, local_context, - mle.manifest_file_path, filter_dag, table_snapshot->schema_id); break; @@ -332,7 +330,10 @@ ObjectInfoPtr IcebergIterator::next(size_t) if (blocking_queue.pop(manifest_file_entry)) { IcebergDataObjectInfoPtr object_info - = std::make_shared(manifest_file_entry, table_state_snapshot->schema_id); + = std::make_shared( + manifest_file_entry, + persistent_components.path_resolver.resolve(manifest_file_entry->parsed_entry->file_path_key), + table_state_snapshot->schema_id); for (const auto & position_delete : defineDeletesSpan(manifest_file_entry, position_deletes_files, /* is_equality_delete */ false, logger)) { @@ -340,7 +341,8 @@ ObjectInfoPtr IcebergIterator::next(size_t) const auto & lower = position_delete->parsed_entry->lower_reference_data_file_path; const auto & upper = position_delete->parsed_entry->upper_reference_data_file_path; bool can_contain_data_file_deletes - = (!lower.has_value() || lower.value() <= data_file_path) && (!upper.has_value() || upper.value() >= data_file_path); + = (!lower.has_value() || *lower <= data_file_path) + && (!upper.has_value() || *upper >= data_file_path); /// Skip position deletes that do not match the data file path. if (!can_contain_data_file_deletes) { @@ -350,10 +352,10 @@ ObjectInfoPtr IcebergIterator::next(size_t) "Skipping position delete file `{}` for data file `{}` because position delete has out of bounds reference data file " "bounds: " "(lower bound: `{}`, upper bound: `{}`)", - position_delete->file_path, + position_delete->parsed_entry->file_path_key, data_file_path, - lower.value_or("[no lower bound]"), - upper.value_or("[no upper bound]")); + lower.has_value() ? lower->serialize() : "[no lower bound]", + upper.has_value() ? upper->serialize() : "[no upper bound]"); } else { @@ -362,11 +364,12 @@ ObjectInfoPtr IcebergIterator::next(size_t) logger, "Processing position delete file `{}` for data file `{}` with reference data file bounds: " "(lower bound: `{}`, upper bound: `{}`)", - position_delete->file_path, + position_delete->parsed_entry->file_path_key, data_file_path, - lower.value_or("[no lower bound]"), - upper.value_or("[no upper bound]")); - object_info->addPositionDeleteObject(position_delete); + lower.has_value() ? lower->serialize() : "[no lower bound]", + upper.has_value() ? upper->serialize() : "[no upper bound]"); + object_info->addPositionDeleteObject( + position_delete, persistent_components.path_resolver.resolve(position_delete->parsed_entry->file_path_key)); } } @@ -382,7 +385,8 @@ ObjectInfoPtr IcebergIterator::next(size_t) for (const auto & equality_delete : defineDeletesSpan(manifest_file_entry, equality_deletes_files, /* is_equality_delete */ true, logger)) { - object_info->addEqualityDeleteObject(equality_delete); + object_info->addEqualityDeleteObject( + equality_delete, persistent_components.path_resolver.resolve(equality_delete->parsed_entry->file_path_key)); } if (!object_info->info.equality_deletes_objects.empty()) diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index f6c7d4e86999..3a6c63001b05 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -4,6 +4,7 @@ #if USE_AVRO #include +#include #include #include #include @@ -32,14 +33,14 @@ #include #include -#include -#include #include #include #include #include +#include #include #include +#include #include #include #include @@ -48,6 +49,7 @@ #include #include #include +#include #include #include @@ -58,12 +60,12 @@ #include #include -#include #include #include #include #include -#include +#include +#include #include #include #include @@ -127,6 +129,7 @@ extern const SettingsBool use_roaring_bitmap_iceberg_positional_deletes; extern const SettingsString iceberg_metadata_compression_method; extern const SettingsBool allow_insert_into_iceberg; extern const SettingsBool allow_experimental_iceberg_compaction; +extern const SettingsBool allow_iceberg_remove_orphan_files; extern const SettingsBool allow_experimental_expire_snapshots; extern const SettingsBool iceberg_delete_data_on_drop; } @@ -206,7 +209,7 @@ Iceberg::PersistentTableComponents IcebergMetadata::initializePersistentTableCom StorageObjectStorageConfigurationPtr configuration, IcebergMetadataFilesCachePtr cache_ptr, ContextPtr context_) { const auto [metadata_version, metadata_file_path, compression_method] - = getLatestOrExplicitMetadataFileAndVersion(object_storage, configuration->getPathForRead().path, configuration->getDataLakeSettings(), cache_ptr, context_, log.get(), std::nullopt, true); + = getLatestOrExplicitMetadataFileAndVersion(object_storage, configuration->getPathForRead().path, configuration->getDataLakeSettings(), cache_ptr, context_, log.get(), std::nullopt, CompressionMethod::None, true); LOG_DEBUG(log, "Latest metadata file path is {}, version {}", metadata_file_path, metadata_version); auto metadata_object = getMetadataJSONObject(metadata_file_path, object_storage, cache_ptr, context_, log, compression_method, std::nullopt); @@ -228,15 +231,17 @@ Iceberg::PersistentTableComponents IcebergMetadata::initializePersistentTableCom Iceberg::f_table_uuid); } } + auto table_path = configuration->getPathForRead().path; return PersistentTableComponents{ .schema_processor = std::make_shared(context_), .metadata_cache = cache_ptr, .format_version = format_version, .table_location = table_location, .metadata_compression_method = compression_method, - .table_path = configuration->getPathForRead().path, + .table_path = table_path, .table_uuid = table_uuid, .common_namespace = configuration->getNamespace(), + .path_resolver = IcebergPathResolver(table_location, table_path, configuration->getTypeName(), configuration->getNamespace()), }; } @@ -250,6 +255,7 @@ std::pair IcebergMetadata::getReleva context, log.get(), persistent_components.table_uuid, + persistent_components.metadata_compression_method, force_fetch_latest_metadata); return getState(context, metadata_file_path, metadata_version); } @@ -426,7 +432,7 @@ IcebergDataSnapshotPtr IcebergMetadata::createIcebergDataSnapshotFromSnapshotJSO ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "Snapshot object doesn't contain a manifest list path for snapshot with id `{}`", snapshot_id); - String manifest_list_file_path = snapshot_object->getValue(f_manifest_list); + IcebergPathFromMetadata manifest_list_file_path = IcebergPathFromMetadata::deserialize(snapshot_object->getValue(f_manifest_list)); std::optional total_rows; std::optional total_bytes; std::optional total_position_deletes; @@ -452,13 +458,7 @@ IcebergDataSnapshotPtr IcebergMetadata::createIcebergDataSnapshotFromSnapshotJSO return std::make_shared( - getManifestList( - object_storage, - persistent_components, - local_context, - getProperFilePathFromMetadataInfo( - manifest_list_file_path, persistent_components.table_path, persistent_components.table_location), - log), + getManifestList(object_storage, persistent_components, local_context, manifest_list_file_path, log), snapshot_id, schema_id, total_rows, @@ -586,8 +586,8 @@ IcebergMetadata::getState(const ContextPtr & local_context, const String & metad local_context, dump_metadata, DB::IcebergMetadataLogLevel::Metadata, - persistent_components.table_path, - metadata_path, + persistent_components.path_resolver.getTableRoot(), + Iceberg::IcebergPathFromMetadata::deserialize(metadata_path), std::nullopt, std::nullopt); @@ -627,7 +627,7 @@ std::shared_ptr IcebergMetadata::getSchemaTransformer(ContextP void IcebergMetadata::mutate( const MutationCommands & commands, - StorageObjectStorageConfigurationPtr configuration, + StorageObjectStorageConfigurationPtr /*configuration*/, ContextPtr context, const StorageID & storage_id, StorageMetadataPtr metadata_snapshot, @@ -652,10 +652,7 @@ void IcebergMetadata::mutate( persistent_components, write_format, format_settings, - catalog, - configuration->getTypeName(), - configuration->getNamespace() - ); + catalog); } void IcebergMetadata::checkMutationIsPossible(const MutationCommands & commands) @@ -698,39 +695,11 @@ void IcebergMetadata::alter(const AlterCommands & params, ContextPtr context) Iceberg::alter(params, context, object_storage, data_lake_settings, persistent_components, write_format); } -static Pipe expireSnapshotsResultToPipe(const Iceberg::ExpireSnapshotsResult & result) -{ - Block header{ - ColumnWithTypeAndName(std::make_shared(), "metric_name"), - ColumnWithTypeAndName(std::make_shared(), "metric_value"), - }; - - MutableColumns columns = header.cloneEmptyColumns(); - - auto add = [&](const char * name, Int64 value) - { - columns[0]->insert(String(name)); - columns[1]->insert(value); - }; - - add("deleted_data_files_count", result.deleted_data_files_count); - add("deleted_position_delete_files_count", result.deleted_position_delete_files_count); - add("deleted_equality_delete_files_count", result.deleted_equality_delete_files_count); - add("deleted_manifest_files_count", result.deleted_manifest_files_count); - add("deleted_manifest_lists_count", result.deleted_manifest_lists_count); - add("deleted_statistics_files_count", result.deleted_statistics_files_count); - add("dry_run", result.dry_run ? 1 : 0); - - const size_t rows = columns[0]->size(); - Chunk chunk(std::move(columns), rows); - return Pipe(std::make_shared(std::make_shared(std::move(header)), std::move(chunk))); -} - Pipe IcebergMetadata::executeCommand( const String & command_name, const ASTPtr & args, ObjectStoragePtr object_storage_, - StorageObjectStorageConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr /*configuration*/, std::shared_ptr catalog_, ContextPtr context, const StorageID & storage_id) @@ -753,21 +722,22 @@ Pipe IcebergMetadata::executeCommand( "To allow its usage, enable setting allow_experimental_expire_snapshots"); } - auto options = parseExpireSnapshotsOptions(args, context); - - auto result = Iceberg::expireSnapshots( - options, - context, - object_storage_, - data_lake_settings, - persistent_components, - write_format, - catalog_, - configuration_->getTypeName(), - configuration_->getNamespace(), - storage_id.getTableName()); + return Iceberg::executeExpireSnapshots( + args, context, object_storage_, data_lake_settings, persistent_components, + write_format, catalog_, storage_id.getTableName()); + } + else if (command_name == "remove_orphan_files") + { + if (!context->getSettingsRef()[Setting::allow_iceberg_remove_orphan_files].value) + { + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "remove_orphan_files is experimental. " + "To allow its usage, enable setting allow_iceberg_remove_orphan_files"); + } - return expireSnapshotsResultToPipe(result); + return Iceberg::executeRemoveOrphanFiles( + args, context, object_storage_, data_lake_settings, persistent_components); } else { @@ -809,6 +779,8 @@ void IcebergMetadata::createInitial( } String location_path = configuration_ptr->getRawPath().path; + if (location_path.find("://") == String::npos && !location_path.starts_with('/')) + location_path = "/" + location_path; if (local_context->getSettingsRef()[Setting::write_full_path_in_iceberg_metadata].value) location_path = configuration_ptr->getTypeName() + "://" + configuration_ptr->getNamespace() + "/" + configuration_ptr->getRawPath().path; @@ -841,7 +813,7 @@ void IcebergMetadata::createInitial( if (configuration_ptr->getDataLakeSettings()[DataLakeStorageSetting::iceberg_use_version_hint].value) { auto filename_version_hint = configuration_ptr->getRawPath().path + "metadata/version-hint.text"; - writeMessageToFile(filename, filename_version_hint, object_storage, local_context, "*", ""); + writeMessageToFile("1", filename_version_hint, object_storage, local_context, "*", ""); } if (catalog) @@ -904,7 +876,8 @@ IcebergMetadata::IcebergHistory IcebergMetadata::getHistory(ContextPtr local_con persistent_components.metadata_cache, local_context, log.get(), - persistent_components.table_uuid); + persistent_components.table_uuid, + persistent_components.metadata_compression_method); auto metadata_object = getMetadataJSONObject(metadata_file_path, object_storage, persistent_components.metadata_cache, local_context, log, compression_method, persistent_components.table_uuid); @@ -948,7 +921,7 @@ IcebergMetadata::IcebergHistory IcebergMetadata::getHistory(ContextPtr local_con const auto snapshot = snapshots->getObject(static_cast(i)); history_record.snapshot_id = snapshot->getValue(f_metadata_snapshot_id); - history_record.manifest_list_path = snapshot->getValue(f_manifest_list); + history_record.manifest_list_path = IcebergPathFromMetadata::deserialize(snapshot->getValue(f_manifest_list)); const auto summary = snapshot->getObject(f_summary); if (summary->has(f_added_data_files)) history_record.added_files = summary->getValue(f_added_data_files); @@ -1488,15 +1461,18 @@ bool IcebergMetadata::commitImportPartitionTransactionImpl( } CompressionMethod metadata_compression_method = persistent_components.metadata_compression_method; + const auto & resolver = persistent_components.path_resolver; - auto [metadata_name, storage_metadata_name] = filename_generator.generateMetadataName(); + auto metadata_info = filename_generator.generateMetadataPathWithInfo(); + String storage_metadata_name = resolver.resolve(metadata_info.path); Int64 parent_snapshot = -1; if (metadata->has(Iceberg::f_current_snapshot_id)) parent_snapshot = metadata->getValue(Iceberg::f_current_snapshot_id); - auto [new_snapshot, manifest_list_name, storage_manifest_list_name] = MetadataGenerator(metadata).generateNextMetadata( - filename_generator, metadata_name, parent_snapshot, total_data_files, total_rows, total_chunks_size, total_data_files, /* added_delete_files */0, /* num_deleted_rows */0); + auto [new_snapshot, manifest_list_path] = MetadataGenerator(metadata).generateNextMetadata( + filename_generator, metadata_info.path, parent_snapshot, total_data_files, total_rows, total_chunks_size, total_data_files, /* added_delete_files */0, /* num_deleted_rows */0); + String storage_manifest_list_name = resolver.resolve(manifest_list_path); /// Embed the stable transaction identifier in the snapshot summary so that a retry after crash /// can detect the commit already happened by scanning the live snapshots array, without extra S3 @@ -1504,7 +1480,7 @@ bool IcebergMetadata::commitImportPartitionTransactionImpl( new_snapshot->getObject(Iceberg::f_summary)->set( Iceberg::f_clickhouse_export_partition_transaction_id, transaction_id); - String manifest_entry_name; + Iceberg::IcebergPathFromMetadata manifest_entry_path; String storage_manifest_entry_name; Int64 manifest_lengths = 0; @@ -1553,6 +1529,7 @@ bool IcebergMetadata::commitImportPartitionTransactionImpl( context, getLogger("IcebergWrites").get(), persistent_components.table_uuid, + persistent_components.metadata_compression_method, true); } @@ -1594,13 +1571,15 @@ bool IcebergMetadata::commitImportPartitionTransactionImpl( } }; + std::vector data_file_metadata_paths; + data_file_metadata_paths.reserve(data_file_paths.size()); + for (const auto & path : data_file_paths) + data_file_metadata_paths.push_back(Iceberg::IcebergPathFromMetadata::deserialize(path)); + try { - { - auto result = filename_generator.generateManifestEntryName(); - manifest_entry_name = result.path_in_metadata; - storage_manifest_entry_name = result.path_in_storage; - } + manifest_entry_path = filename_generator.generateManifestEntryName(); + storage_manifest_entry_name = resolver.resolve(manifest_entry_path); auto buffer_manifest_entry = object_storage->writeObject( StoredObject(storage_manifest_entry_name), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); @@ -1617,7 +1596,7 @@ bool IcebergMetadata::commitImportPartitionTransactionImpl( partition_columns, partition_values, partition_types, - data_file_paths, + data_file_metadata_paths, std::nullopt, /// per_file_stats is filled, no need for the generic aggregate sample_block, new_snapshot, @@ -1643,7 +1622,7 @@ bool IcebergMetadata::commitImportPartitionTransactionImpl( try { generateManifestList( - filename_generator, metadata, object_storage, context, {manifest_entry_name}, new_snapshot, manifest_lengths, *buffer_manifest_list, Iceberg::FileContentType::DATA, true); + resolver, metadata, object_storage, context, {manifest_entry_path}, new_snapshot, {manifest_lengths}, *buffer_manifest_list, Iceberg::FileContentType::DATA, true); buffer_manifest_list->finalize(); } catch (...) @@ -1659,15 +1638,14 @@ bool IcebergMetadata::commitImportPartitionTransactionImpl( std::string json_representation = removeEscapedSlashes(oss.str()); LOG_DEBUG(log, "Writing new metadata file {}", storage_metadata_name); - auto hint = filename_generator.generateVersionHint(); + auto hint_path = filename_generator.generateVersionHint(); if (!writeMetadataFileAndVersionHint( - storage_metadata_name, + resolver, + metadata_info, json_representation, - hint.path_in_storage, - storage_metadata_name, + hint_path, object_storage, context, - metadata_compression_method, data_lake_settings[DataLakeStorageSetting::iceberg_use_version_hint])) { LOG_DEBUG(log, "Failed to write metadata {}, retrying", storage_metadata_name); @@ -1679,9 +1657,9 @@ bool IcebergMetadata::commitImportPartitionTransactionImpl( if (catalog) { - String catalog_filename = metadata_name; + String catalog_filename = metadata_info.path.serialize(); if (!catalog_filename.starts_with(blob_storage_type_name)) - catalog_filename = blob_storage_type_name + "://" + blob_storage_namespace_name + "/" + metadata_name; + catalog_filename = blob_storage_type_name + "://" + blob_storage_namespace_name + "/" + catalog_filename; const auto & [namespace_name, table_name] = DataLake::parseTableName(table_id.getTableName()); if (!catalog->updateMetadata(namespace_name, table_name, catalog_filename, new_snapshot)) @@ -1766,6 +1744,7 @@ void IcebergMetadata::commitExportPartitionTransaction( context, getLogger("IcebergMetadata").get(), persistent_components.table_uuid, + persistent_components.metadata_compression_method, true); /// Latest metadata is ALWAYS necessary to commit - but we abort in case schema or partition spec changed @@ -1816,26 +1795,10 @@ void IcebergMetadata::commitExportPartitionTransaction( const auto partition_types = partitioner.getResultTypes(); const auto metadata_compression_method = persistent_components.metadata_compression_method; - auto config_path = persistent_components.table_path; - if (config_path.empty() || config_path.back() != '/') - config_path += "/"; - if (!config_path.starts_with('/')) - config_path = '/' + config_path; - - FileNamesGenerator filename_generator; - if (!context->getSettingsRef()[Setting::write_full_path_in_iceberg_metadata]) - { - filename_generator = FileNamesGenerator( - config_path, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); - } - else - { - auto bucket = metadata->getValue(Iceberg::f_location); - if (bucket.empty() || bucket.back() != '/') - bucket += "/"; - filename_generator = FileNamesGenerator( - bucket, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); - } + + FileNamesGenerator filename_generator( + persistent_components.path_resolver.getTableLocation(), + (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); filename_generator.setVersion(updated_metadata_file_info.version + 1); /// Load per-file sidecar stats, necessary to populate the manifest file stats. diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h index 44309f94e60a..f52e596e2315 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h @@ -55,7 +55,7 @@ using LatestMetadataVersionPtr = std::shared_ptr; /// And we can get `ManifestFileContent` from cache by ManifestFileEntry. struct ManifestFileCacheKey { - String manifest_file_path; + Iceberg::IcebergPathFromMetadata manifest_file_path; size_t manifest_file_byte_size; Int64 added_sequence_number; Int64 added_snapshot_id; @@ -110,7 +110,7 @@ struct IcebergMetadataFilesCacheCell : private boost::noncopyable size_t total_size = 0; for (const auto & entry: manifest_file_cache_keys) { - total_size += sizeof(ManifestFileCacheKey) + entry.manifest_file_path.capacity(); + total_size += sizeof(ManifestFileCacheKey) + entry.manifest_file_path.serialize().capacity(); } return total_size; } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergPath.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergPath.cpp new file mode 100644 index 000000000000..fee289f73389 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergPath.cpp @@ -0,0 +1,95 @@ +#include +#include + +#include + +namespace DB::ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +namespace DB::Iceberg +{ + +// This function is used to get the file path inside the directory which corresponds to Iceberg table from the full blob path which is written in manifest and metadata files. +// For example, if the full blob path is s3://bucket/table_name/data/00000-1-1234567890.avro, the function will return table_name/data/00000-1-1234567890.avro +// Common path should end with "" or "/". +String IcebergPathResolver::resolve(const IcebergPathFromMetadata & metadata_path) const +{ + auto trim_forward_slash = [](std::string_view str) -> std::string_view + { + if (str.starts_with('/')) + { + return str.substr(1); + } + return str; + }; + + auto raw_path = metadata_path.serialize(); + + if (raw_path.starts_with(table_location) && table_location.ends_with(table_root)) + { + auto result = std::filesystem::path{table_root} / trim_forward_slash(raw_path.substr(table_location.size())); + return result; + } + + if (table_root.empty()) + { + throw ::DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, + "IcebergPathResolver::resolve failed first branch. raw_path='{}', table_location='{}', table_root='{}'", + raw_path, table_location, table_root); + } + + + auto pos = raw_path.find(table_root); + /// Valid situation when data and metadata files are stored in different directories. + if (pos == std::string::npos) + { + /// connection://bucket + auto prefix = table_location.substr(0, table_location.size() - table_root.size()); + if (raw_path.size() < prefix.size()) + { + throw ::DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, + "IcebergPathResolver::resolve failed in the second branch. raw_path='{}', table_location='{}', table_root='{}'", + raw_path, + table_location, + table_root); + } + return std::string{raw_path.substr(prefix.size())}; + } + + size_t good_pos = std::string::npos; + while (pos != std::string::npos) + { + auto potential_position = pos + table_root.size(); + if (((potential_position + 6 <= raw_path.size()) && (std::string_view(raw_path.data() + potential_position, 6) == "/data/")) + || ((potential_position + 10 <= raw_path.size()) + && (std::string_view(raw_path.data() + potential_position, 10) == "/metadata/"))) + { + good_pos = pos; + break; + } + size_t new_pos = raw_path.find(table_root, pos + 1); + if (new_pos == std::string::npos) + { + break; + } + pos = new_pos; + } + + + if (good_pos != std::string::npos) + { + return std::string{raw_path.substr(good_pos)}; + } + else if (pos != std::string::npos) + { + return std::string{raw_path.substr(pos)}; + } + else + { + throw ::DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Expected to find '{}' in data path: '{}'", table_root, raw_path); + } +} +} diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergPath.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergPath.h new file mode 100644 index 000000000000..13072990005c --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergPath.h @@ -0,0 +1,119 @@ +#pragma once + +#include +#include + +namespace DB +{ +class FileNamesGenerator; +} + +namespace DB::Iceberg +{ + +/// Strong type for paths stored in Iceberg metadata files (Avro/JSON). +/// These paths may use various URI schemes (wasb://, s3://, abfss://, hdfs://, etc.) +/// or may be absolute paths (/table/data/xxx.parquet) if written from ClickHouse. +/// +/// If we get a path from metadata, we don't modify its format to preserve compatibility +/// with different Iceberg components and versions. Instead, we treat it as opaque and pass through IcebergPathResolver to get the actual storage path for I/O operations. +/// +/// This type is intentionally NOT implicitly convertible to String +/// to prevent accidental use of metadata paths as storage paths. +/// +/// Also pay attention that paths to metadata.json files themselves may not have this form, because usually they are either +/// produced by ClickHouse by table_location traversal or taken from the corresponding Iceberg Catalog. In case metadata.json +/// is generated by ClickHouse it will be expressed by this type. +class IcebergPathFromMetadata +{ +public: + IcebergPathFromMetadata() = default; + + /// Should be used only when deserialization is inevitable (the most common case is reading manifest files, which contain metadata paths). + /// Also needed to get the file which corresponds to a line in the Chunk when used for position-delete algorithms. + static IcebergPathFromMetadata deserialize(String path_) { return IcebergPathFromMetadata(std::move(path_)); } + + /// Extract the raw path string for writing into Iceberg metadata files, + /// serialization, cache keys, virtual column values, etc. + const String & serialize() const { return raw_path; } + + bool empty() const { return raw_path.empty(); } + + auto operator<=>(const IcebergPathFromMetadata & other) const { return raw_path <=> other.raw_path; } + auto operator==(const IcebergPathFromMetadata & other) const { return raw_path == other.raw_path; } + + +private: + friend class DB::FileNamesGenerator; + + explicit IcebergPathFromMetadata(String path_) + : raw_path(std::move(path_)) + { + } + String raw_path; +}; + +/// Converts Iceberg metadata paths to actual object storage paths. +/// +/// This is the ONLY way to go from a metadata path to a storage path. +class IcebergPathResolver +{ +public: + IcebergPathResolver(String table_location_, String table_root_, String blob_storage_type_name_ = {}, String blob_storage_namespace_name_ = {}) + : table_location(std::move(table_location_)) + , table_root(std::move(table_root_)) + , blob_storage_type_name(std::move(blob_storage_type_name_)) + , blob_storage_namespace_name(std::move(blob_storage_namespace_name_)) + { + auto trim_backward_slashes = [](String & str) + { + while (!str.empty() && str.back() == '/') + str.pop_back(); + }; + trim_backward_slashes(table_root); + trim_backward_slashes(table_location); + + /// Normalize: non-URI table_location should start with '/' + if (!table_location.empty() && table_location.find("://") == String::npos && table_location[0] != '/') + table_location = "/" + table_location; + } + + /// Convert a metadata path to an actual storage path for I/O operations. + String resolve(const IcebergPathFromMetadata & metadata_path) const; + + /// Convert a metadata path to a catalog-compatible path. + /// Done this way because backward compatibility reasons. + String resolveForCatalog(const IcebergPathFromMetadata & metadata_path) const + { + String catalog_filename = metadata_path.serialize(); + if (!catalog_filename.starts_with(blob_storage_type_name)) + catalog_filename = blob_storage_type_name + "://" + blob_storage_namespace_name + "/" + catalog_filename; + return catalog_filename; + } + + const String & getTableLocation() const { return table_location; } + const String & getTableRoot() const { return table_root; } + +private: + String table_location; + String table_root; + String blob_storage_type_name; + String blob_storage_namespace_name; +}; + +} + +template <> +struct std::hash +{ + size_t operator()(const DB::Iceberg::IcebergPathFromMetadata & p) const noexcept { return std::hash{}(p.serialize()); } +}; + +template <> +struct fmt::formatter : fmt::formatter +{ + auto format(const DB::Iceberg::IcebergPathFromMetadata & p, fmt::format_context & ctx) const + { + return fmt::formatter::format(p.serialize(), ctx); + } +}; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.cpp index c102e0edd36d..8cd7b343241b 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.cpp @@ -405,7 +405,7 @@ void generateManifestFile( const std::vector & partition_columns, const std::vector & partition_values, const std::vector & partition_types, - const std::vector & data_file_names, + const std::vector & data_file_names, const std::optional & data_file_statistics, SharedHeader sample_block, Poco::JSON::Object::Ptr new_snapshot, @@ -488,7 +488,7 @@ void generateManifestFile( avro::GenericRecord & data_file = manifest.field(Iceberg::f_data_file).value(); if (version > 1) data_file.field(Iceberg::f_content) = avro::GenericDatum(static_cast(content_type)); - data_file.field(Iceberg::f_file_path) = avro::GenericDatum(data_file_name); + data_file.field(Iceberg::f_file_path) = avro::GenericDatum(data_file_name.serialize()); data_file.field(Iceberg::f_file_format) = avro::GenericDatum(format); /// vibe coded - needs extra attention @@ -648,13 +648,13 @@ void generateManifestFile( } void generateManifestList( - const FileNamesGenerator & filename_generator, + const Iceberg::IcebergPathResolver & path_resolver, Poco::JSON::Object::Ptr metadata, ObjectStoragePtr object_storage, ContextPtr context, - const Strings & manifest_entry_names, + const std::vector & manifest_entry_names, Poco::JSON::Object::Ptr new_snapshot, - Int64 manifest_length, + const std::vector & manifest_entry_sizes, WriteBuffer & buf, Iceberg::FileContentType content_type, bool use_previous_snapshots) @@ -673,13 +673,13 @@ void generateManifestList( auto adapter = std::make_unique(buf); avro::DataFileWriter writer(std::move(adapter), schema); - for (const auto & manifest_entry_name : manifest_entry_names) + for (size_t entry_idx = 0; entry_idx < manifest_entry_names.size(); ++entry_idx) { avro::GenericDatum entry_datum(schema.root()); avro::GenericRecord & entry = entry_datum.value(); - entry.field(Iceberg::f_manifest_path) = manifest_entry_name; - entry.field(Iceberg::f_manifest_length) = manifest_length; + entry.field(Iceberg::f_manifest_path) = manifest_entry_names[entry_idx].serialize(); + entry.field(Iceberg::f_manifest_length) = manifest_entry_sizes[entry_idx]; entry.field(Iceberg::f_partition_spec_id) = metadata->getValue(Iceberg::f_default_spec_id); if (version > 1) { @@ -755,9 +755,10 @@ void generateManifestList( { if (snapshots->getObject(static_cast(i))->getValue(Iceberg::f_metadata_snapshot_id) == parent_snapshot_id) { - auto manifest_list = snapshots->getObject(static_cast(i))->getValue(Iceberg::f_manifest_list); + auto manifest_list = Iceberg::IcebergPathFromMetadata::deserialize( + snapshots->getObject(static_cast(i))->getValue(Iceberg::f_manifest_list)); - RelativePathWithMetadata relative_path_with_metadata(filename_generator.convertMetadataPathToStoragePath(manifest_list)); + RelativePathWithMetadata relative_path_with_metadata(path_resolver.resolve(manifest_list)); auto manifest_list_buf = createReadBuffer(relative_path_with_metadata, object_storage, context, getLogger("IcebergWrites")); auto input_stream = std::make_unique(*manifest_list_buf); @@ -850,8 +851,6 @@ IcebergStorageSink::IcebergStorageSink( , persistent_table_components(persistent_table_components_) , data_lake_settings(configuration_->getDataLakeSettings()) , write_format(configuration_->getFormat()) - , blob_storage_type_name(configuration_->getTypeName()) - , blob_storage_namespace_name(configuration_->getNamespace()) { auto [last_version, metadata_path, compression_method] = getLatestOrExplicitMetadataFileAndVersion( object_storage, @@ -861,6 +860,7 @@ IcebergStorageSink::IcebergStorageSink( context_, log.get(), persistent_table_components.table_uuid, + persistent_table_components.metadata_compression_method, true); metadata = getMetadataJSONObject( @@ -872,25 +872,9 @@ IcebergStorageSink::IcebergStorageSink( compression_method, persistent_table_components.table_uuid); metadata_compression_method = compression_method; - auto config_path = persistent_table_components.table_path; - if (config_path.empty() || config_path.back() != '/') - config_path += "/"; - if (!config_path.starts_with('/')) - config_path = '/' + config_path; - - if (!context_->getSettingsRef()[Setting::write_full_path_in_iceberg_metadata]) - { - filename_generator = FileNamesGenerator( - config_path, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); - } - else - { - auto bucket = metadata->getValue(Iceberg::f_location); - if (bucket.empty() || bucket.back() != '/') - bucket += "/"; - filename_generator = FileNamesGenerator( - bucket, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); - } + filename_generator = FileNamesGenerator( + persistent_table_components.path_resolver.getTableLocation(), + (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); filename_generator.setVersion(last_version + 1); @@ -974,6 +958,7 @@ void IcebergStorageSink::consume(Chunk & chunk) context->getSettingsRef()[Setting::iceberg_insert_max_bytes_in_data_file], current_schema->getArray(Iceberg::f_fields), filename_generator, + persistent_table_components.path_resolver, object_storage, context, format_settings, @@ -1091,7 +1076,8 @@ void IcebergStorageSink::cancelBuffers() bool IcebergStorageSink::initializeMetadata() { - auto [metadata_name, storage_metadata_name] = filename_generator.generateMetadataName(); + const auto & resolver = persistent_table_components.path_resolver; + auto metadata_info = filename_generator.generateMetadataPathWithInfo(); Int64 parent_snapshot = -1; if (metadata->has(Iceberg::f_current_snapshot_id)) @@ -1100,13 +1086,22 @@ bool IcebergStorageSink::initializeMetadata() Int32 total_data_files = 0; for (const auto & [_, writer] : writer_per_partition_key) total_data_files += writer.getDataFiles().size(); - auto [new_snapshot, manifest_list_name, storage_manifest_list_name] = MetadataGenerator(metadata).generateNextMetadata( - filename_generator, metadata_name, parent_snapshot, total_data_files, total_rows, total_chunks_size, total_data_files, /* added_delete_files */0, /* num_deleted_rows */0); + auto [new_snapshot, manifest_list_path] = MetadataGenerator(metadata).generateNextMetadata( + filename_generator, + metadata_info.path, + parent_snapshot, + total_data_files, + total_rows, + total_chunks_size, + total_data_files, + /* added_delete_files */ 0, + /* num_deleted_rows */ 0); + auto storage_manifest_list_name = resolver.resolve(manifest_list_path); Strings manifest_entries_in_storage; - Strings manifest_entries; - Int64 manifest_lengths = 0; + std::vector manifest_entries; + std::vector manifest_entry_sizes; auto cleanup = [&] (bool retry_because_of_metadata_conflict) { @@ -1136,6 +1131,7 @@ bool IcebergStorageSink::initializeMetadata() context, getLogger("IcebergWrites").get(), persistent_table_components.table_uuid, + persistent_table_components.metadata_compression_method, true, /* ignore_explicit_metadata_file_path */ true); @@ -1185,12 +1181,12 @@ bool IcebergStorageSink::initializeMetadata() { for (const auto & [partition_key, writer] : writer_per_partition_key) { - auto [manifest_entry_name, storage_manifest_entry_name] = filename_generator.generateManifestEntryName(); - manifest_entries_in_storage.push_back(storage_manifest_entry_name); - manifest_entries.push_back(manifest_entry_name); + auto manifest_entry_path = filename_generator.generateManifestEntryName(); + manifest_entries_in_storage.push_back(resolver.resolve(manifest_entry_path)); + manifest_entries.push_back(manifest_entry_path); auto buffer_manifest_entry = object_storage->writeObject( - StoredObject(storage_manifest_entry_name), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + StoredObject(resolver.resolve(manifest_entry_path)), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); try { generateManifestFile( @@ -1208,7 +1204,12 @@ bool IcebergStorageSink::initializeMetadata() *buffer_manifest_entry, Iceberg::FileContentType::DATA); buffer_manifest_entry->finalize(); - manifest_lengths += buffer_manifest_entry->count(); + auto size = buffer_manifest_entry->count(); + if (size == 0) + { + size = object_storage->getObjectMetadata(resolver.resolve(manifest_entry_path), /*with_tags=*/false).size_bytes; + } + manifest_entry_sizes.push_back(size); } catch (...) { @@ -1223,7 +1224,7 @@ bool IcebergStorageSink::initializeMetadata() try { generateManifestList( - filename_generator, metadata, object_storage, context, manifest_entries, new_snapshot, manifest_lengths, *buffer_manifest_list, Iceberg::FileContentType::DATA); + persistent_table_components.path_resolver, metadata, object_storage, context, manifest_entries, new_snapshot, manifest_entry_sizes, *buffer_manifest_list, Iceberg::FileContentType::DATA); buffer_manifest_list->finalize(); } catch (...) @@ -1243,32 +1244,29 @@ bool IcebergStorageSink::initializeMetadata() throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failpoint for cleanup enabled"); }); - LOG_DEBUG(log, "Writing new metadata file {}", storage_metadata_name); - auto hint = filename_generator.generateVersionHint(); + LOG_DEBUG(log, "Writing new metadata file {}", metadata_info.path); + auto hint_path = filename_generator.generateVersionHint(); if (!writeMetadataFileAndVersionHint( - storage_metadata_name, + persistent_table_components.path_resolver, + metadata_info, json_representation, - hint.path_in_storage, - storage_metadata_name, + hint_path, object_storage, context, - metadata_compression_method, data_lake_settings[DataLakeStorageSetting::iceberg_use_version_hint])) { - LOG_DEBUG(log, "Failed to write metadata {}, retrying", storage_metadata_name); + LOG_DEBUG(log, "Failed to write metadata {}, retrying", metadata_info.path); cleanup(true); return false; } else { - LOG_DEBUG(log, "Metadata file {} written", storage_metadata_name); + LOG_DEBUG(log, "Metadata file {} written", metadata_info.path); } if (catalog) { - String catalog_filename = metadata_name; - if (!catalog_filename.starts_with(blob_storage_type_name)) - catalog_filename = blob_storage_type_name + "://" + blob_storage_namespace_name + "/" + metadata_name; + auto catalog_filename = resolver.resolveForCatalog(metadata_info.path); const auto & [namespace_name, table_name] = DataLake::parseTableName(table_id.getTableName()); if (!catalog->updateMetadata(namespace_name, table_name, catalog_filename, new_snapshot)) @@ -1334,25 +1332,9 @@ IcebergImportSink::IcebergImportSink( const auto metadata_compression_method = persistent_table_components.metadata_compression_method; - auto config_path = persistent_table_components.table_path; - if (config_path.empty() || config_path.back() != '/') - config_path += "/"; - if (!config_path.starts_with('/')) - config_path = '/' + config_path; - - if (!context_->getSettingsRef()[Setting::write_full_path_in_iceberg_metadata]) - { - filename_generator = FileNamesGenerator( - config_path, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); - } - else - { - auto bucket = metadata_json->getValue(Iceberg::f_location); - if (bucket.empty() || bucket.back() != '/') - bucket += "/"; - filename_generator = FileNamesGenerator( - bucket, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); - } + filename_generator = FileNamesGenerator( + persistent_table_components.path_resolver.getTableLocation(), + (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); const auto [last_version, unused_meta_path, unused_compression] = getLatestOrExplicitMetadataFileAndVersion( object_storage, @@ -1361,7 +1343,8 @@ IcebergImportSink::IcebergImportSink( persistent_table_components.metadata_cache, context_, getLogger("IcebergWrites").get(), - persistent_table_components.table_uuid); + persistent_table_components.table_uuid, + persistent_table_components.metadata_compression_method); (void)unused_meta_path; (void)unused_compression; @@ -1372,6 +1355,7 @@ IcebergImportSink::IcebergImportSink( context->getSettingsRef()[Setting::iceberg_insert_max_bytes_in_data_file], current_schema->getArray(Iceberg::f_fields), filename_generator, + persistent_table_components.path_resolver, object_storage, context, format_settings, diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.h index 71b8743bd1ce..ffaeb5d10588 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.h @@ -82,7 +82,7 @@ void generateManifestFile( const std::vector & partition_columns, const std::vector & partition_values, const std::vector & partition_types, - const std::vector & data_file_names, + const std::vector & data_file_names, const std::optional & data_file_statistics, SharedHeader sample_block, Poco::JSON::Object::Ptr new_snapshot, @@ -94,13 +94,13 @@ void generateManifestFile( const std::vector & per_file_stats = {}); void generateManifestList( - const FileNamesGenerator & filename_generator, + const Iceberg::IcebergPathResolver & path_resolver, Poco::JSON::Object::Ptr metadata, ObjectStoragePtr object_storage, ContextPtr context, - const Strings & manifest_entry_names, + const std::vector & manifest_entry_names, Poco::JSON::Object::Ptr new_snapshot, - Int64 manifest_length, + const std::vector & manifest_entry_sizes, WriteBuffer & buf, Iceberg::FileContentType content_type, bool use_previous_snapshots = true); @@ -162,8 +162,6 @@ class IcebergStorageSink : public SinkToStorage Iceberg::PersistentTableComponents persistent_table_components; const DataLakeStorageSettings & data_lake_settings; const String write_format; - const String blob_storage_type_name; - const String blob_storage_namespace_name; }; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h index 3a1bccc53411..c56dbe5e7a74 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h @@ -21,6 +21,7 @@ struct ColumnInfo #include #include +#include #include #include @@ -76,8 +77,9 @@ struct ManifestFileCacheableInfo struct ParsedManifestFileEntry : boost::noncopyable { FileContentType content_type; - // It's the original string in the Iceberg metadata - String file_path_key; + /// The original path as stored in the Iceberg metadata. + /// Must be resolved through IcebergPathResolver before use in storage operations. + IcebergPathFromMetadata file_path_key; Int64 row_number; ManifestEntryStatus status; @@ -89,8 +91,8 @@ struct ParsedManifestFileEntry : boost::noncopyable std::unordered_map> value_bounds; String file_format; - std::optional lower_reference_data_file_path; // For position delete files only. - std::optional upper_reference_data_file_path; // For position delete files only. + std::optional lower_reference_data_file_path; // For position delete files only. + std::optional upper_reference_data_file_path; // For position delete files only. std::optional> equality_ids; /// Data file is sorted with this sort_order_id (can be read from metadata.json) @@ -98,7 +100,7 @@ struct ParsedManifestFileEntry : boost::noncopyable ParsedManifestFileEntry( FileContentType content_type_, - String file_path_key_, + IcebergPathFromMetadata file_path_key_, Int64 row_number_, ManifestEntryStatus status_, std::optional written_sequence_number_, @@ -107,8 +109,8 @@ struct ParsedManifestFileEntry : boost::noncopyable std::unordered_map columns_infos_, std::unordered_map> value_bounds_, String file_format_, - std::optional lower_reference_data_file_path_, - std::optional upper_reference_data_file_path_, + std::optional lower_reference_data_file_path_, + std::optional upper_reference_data_file_path_, std::optional> equality_ids_, std::optional sort_order_id_) : content_type(content_type_) @@ -134,9 +136,6 @@ struct ProcessedManifestFileEntry std::shared_ptr parsed_entry; std::shared_ptr common_partition_specification; - /// Computed file path for Object Storage (resolved from parsed_entry->file_path_key) - String file_path; - // Always zero in case of format version 1 Int64 sequence_number; Int32 resolved_schema_id; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileIterator.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileIterator.cpp index d02094697ef2..2c6666dda118 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileIterator.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileIterator.cpp @@ -147,15 +147,13 @@ ManifestFileIterator::~ManifestFileIterator() = default; std::shared_ptr ManifestFileIterator::create( std::shared_ptr manifest_file_deserializer_, - const String & manifest_file_name_, + const IcebergPathFromMetadata & path_to_manifest_file_, Int32 format_version_, - const String & common_path_, + const IcebergPathResolver & path_resolver_, IcebergSchemaProcessor & schema_processor, Int64 inherited_sequence_number_, Int64 inherited_snapshot_id_, - const String & table_location_, DB::ContextPtr context_, - const String & path_to_manifest_file_, std::shared_ptr filter_dag_, Int32 table_snapshot_schema_id_) { @@ -164,7 +162,7 @@ std::shared_ptr ManifestFileIterator::create( context_, dump_metadata, DB::IcebergMetadataLogLevel::ManifestFileMetadata, - common_path_, + path_resolver_.getTableRoot(), path_to_manifest_file_, std::nullopt, std::nullopt); @@ -200,7 +198,7 @@ std::shared_ptr ManifestFileIterator::create( throw Exception( ErrorCodes::BAD_ARGUMENTS, "Cannot read Iceberg table: manifest file '{}' doesn't have field '{}' in its metadata", - manifest_file_name_, + path_to_manifest_file_, f_schema); Poco::Dynamic::Var json = parser.parse(*schema_json_string); @@ -244,10 +242,8 @@ std::shared_ptr ManifestFileIterator::create( return std::shared_ptr(new ManifestFileIterator( std::move(manifest_file_deserializer_), path_to_manifest_file_, - manifest_file_name_, format_version_, - common_path_, - table_location_, + path_resolver_, schema_processor, inherited_sequence_number_, inherited_snapshot_id_, @@ -262,11 +258,9 @@ std::shared_ptr ManifestFileIterator::create( ManifestFileIterator::ManifestFileIterator( std::shared_ptr manifest_file_deserializer_, - const String & path_to_manifest_file_, - const String & manifest_file_name_, + const IcebergPathFromMetadata & path_to_manifest_file_, Int32 format_version_, - const String & common_path_, - const String & table_location_, + const IcebergPathResolver & path_resolver_, IcebergSchemaProcessor & schema_processor, Int64 inherited_sequence_number_, Int64 inherited_snapshot_id_, @@ -279,10 +273,8 @@ ManifestFileIterator::ManifestFileIterator( Int32 table_snapshot_schema_id_) : manifest_file_deserializer(std::move(manifest_file_deserializer_)) , path_to_manifest_file(path_to_manifest_file_) - , manifest_file_name(manifest_file_name_) , format_version(format_version_) - , common_path(common_path_) - , table_location(table_location_) + , path_resolver(path_resolver_) , inherited_sequence_number(inherited_sequence_number_) , inherited_snapshot_id(inherited_snapshot_id_) , context(context_) @@ -317,7 +309,7 @@ ProcessedManifestFileEntryPtr ManifestFileIterator::processRow(size_t row_index) context, dump_metadata, DB::IcebergMetadataLogLevel::ManifestFileEntry, - common_path, + path_resolver.getTableRoot(), path_to_manifest_file, row_index, std::nullopt); @@ -325,7 +317,6 @@ ProcessedManifestFileEntryPtr ManifestFileIterator::processRow(size_t row_index) } /// Compute inherited/resolved fields - const auto file_path = getProperFilePathFromMetadataInfo(parsed_entry->file_path_key, common_path, table_location); Int64 resolved_snapshot_id; if (parsed_entry->parsed_snapshot_id.has_value()) @@ -337,7 +328,7 @@ ProcessedManifestFileEntryPtr ManifestFileIterator::processRow(size_t row_index) throw Exception( ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "Cannot read Iceberg table: manifest file '{}' has entry with snapshot_id 'null' for which write file schema is unknown", - manifest_file_name); + path_to_manifest_file); } else { @@ -354,7 +345,7 @@ ProcessedManifestFileEntryPtr ManifestFileIterator::processRow(size_t row_index) throw Exception( ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "Cannot read Iceberg table: manifest file '{}' has entry with snapshot_id '{}' for which write file schema is unknown", - manifest_file_name, + path_to_manifest_file, resolved_snapshot_id); } catch (const Exception &) @@ -384,7 +375,7 @@ ProcessedManifestFileEntryPtr ManifestFileIterator::processRow(size_t row_index) } auto entry = std::make_shared( - parsed_entry, common_partition_specification, file_path, resolved_sequence_number, resolved_schema_id); + parsed_entry, common_partition_specification, resolved_sequence_number, resolved_schema_id); PruningReturnStatus pruning_status = PruningReturnStatus::NOT_PRUNED; @@ -432,7 +423,7 @@ ProcessedManifestFileEntryPtr ManifestFileIterator::processRow(size_t row_index) context, dump_metadata, DB::IcebergMetadataLogLevel::ManifestFileEntry, - common_path, + path_resolver.getTableRoot(), path_to_manifest_file, row_index, pruning_status); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileIterator.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileIterator.h index 26b630e81bc7..74a2f1e794b8 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileIterator.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileIterator.h @@ -5,6 +5,7 @@ #if USE_AVRO +#include #include #include #include @@ -76,15 +77,13 @@ class ManifestFileIterator : public boost::noncopyable static std::shared_ptr create( std::shared_ptr manifest_file_deserializer, - const String & manifest_file_name, + const IcebergPathFromMetadata & path_to_manifest_file, Int32 format_version_, - const String & common_path, + const IcebergPathResolver & path_resolver, IcebergSchemaProcessor & schema_processor, Int64 inherited_sequence_number, Int64 inherited_snapshot_id, - const std::string & table_location, DB::ContextPtr context, - const String & path_to_manifest_file_, std::shared_ptr filter_dag_, Int32 table_snapshot_schema_id_); @@ -97,8 +96,6 @@ class ManifestFileIterator : public boost::noncopyable std::optional getRowsCountInAllFilesExcludingDeleted(FileContentType content) const; std::optional getBytesCountInAllDataFilesExcludingDeleted() const; - const String & getPathToManifestFile() const { return path_to_manifest_file; } - bool areAllDataFilesSortedBySortOrderID(Int32 sort_order_id) const; /// Returns true if all manifest file entries have been processed @@ -115,11 +112,9 @@ class ManifestFileIterator : public boost::noncopyable private: ManifestFileIterator( std::shared_ptr manifest_file_deserializer, - const String & path_to_manifest_file, - const String & manifest_file_name, + const IcebergPathFromMetadata & path_to_manifest_file, Int32 format_version, - const String & common_path, - const String & table_location, + const IcebergPathResolver & path_resolver, IcebergSchemaProcessor & schema_processor, Int64 inherited_sequence_number, Int64 inherited_snapshot_id, @@ -135,11 +130,9 @@ class ManifestFileIterator : public boost::noncopyable /// Constant properties of this manifest file const std::shared_ptr manifest_file_deserializer; - const String path_to_manifest_file; - const String manifest_file_name; + const IcebergPathFromMetadata path_to_manifest_file; const Int32 format_version; - const String common_path; - const String table_location; + const IcebergPathResolver path_resolver; // always zero in case of format version 1 const Int64 inherited_sequence_number; const Int64 inherited_snapshot_id; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.cpp index bd924af181f1..c4d7cb2222c3 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.cpp @@ -105,7 +105,7 @@ Poco::JSON::Object::Ptr MetadataGenerator::getParentSnapshot(Int64 parent_snapsh MetadataGenerator::NextMetadataResult MetadataGenerator::generateNextMetadata( FileNamesGenerator & generator, - const String & metadata_filename, + const Iceberg::IcebergPathFromMetadata & metadata_file_path, Int64 parent_snapshot_id, Int64 added_files, Int64 added_records, @@ -126,7 +126,7 @@ MetadataGenerator::NextMetadataResult MetadataGenerator::generateNextMetadata( } Int64 snapshot_id = user_defined_snapshot_id.value_or(static_cast(dis(gen))); - auto [manifest_list_name, storage_manifest_list_name] = generator.generateManifestListName(snapshot_id, format_version); + auto manifest_list_path = generator.generateManifestListName(snapshot_id, format_version); new_snapshot->set(Iceberg::f_metadata_snapshot_id, snapshot_id); new_snapshot->set(Iceberg::f_parent_snapshot_id, parent_snapshot_id); @@ -171,7 +171,7 @@ MetadataGenerator::NextMetadataResult MetadataGenerator::generateNextMetadata( new_snapshot->set(Iceberg::f_summary, summary); new_snapshot->set(Iceberg::f_schema_id, metadata_object->getValue(Iceberg::f_current_schema_id)); - new_snapshot->set(Iceberg::f_manifest_list, manifest_list_name); + new_snapshot->set(Iceberg::f_manifest_list, manifest_list_path.serialize()); metadata_object->getArray(Iceberg::f_snapshots)->add(new_snapshot); metadata_object->set(Iceberg::f_current_snapshot_id, snapshot_id); @@ -192,7 +192,7 @@ MetadataGenerator::NextMetadataResult MetadataGenerator::generateNextMetadata( { Poco::JSON::Object::Ptr new_metadata_item = new Poco::JSON::Object; - new_metadata_item->set(Iceberg::f_metadata_file, metadata_filename); + new_metadata_item->set(Iceberg::f_metadata_file, metadata_file_path.serialize()); new_metadata_item->set(Iceberg::f_timestamp_ms, timestamp); metadata_object->getArray(Iceberg::f_metadata_log)->add(new_metadata_item); } @@ -216,7 +216,7 @@ MetadataGenerator::NextMetadataResult MetadataGenerator::generateNextMetadata( properties->set("write.merge.mode", "merge-on-read"); properties->set("write.update.mode", "merge-on-read"); } - return {new_snapshot, manifest_list_name, storage_manifest_list_name}; + return {new_snapshot, manifest_list_path}; } void MetadataGenerator::generateDropColumnMetadata(const String & column_name) diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.h index a5ebdfc593f8..4477430fc49f 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.h @@ -1,6 +1,13 @@ #pragma once +#include +#include "config.h" + +#include #include +#include +#include + namespace DB { @@ -15,13 +22,15 @@ class MetadataGenerator struct NextMetadataResult { Poco::JSON::Object::Ptr snapshot = nullptr; - String metadata_path; - String storage_metadata_path; + /// Metadata path for the manifest list file (e.g. "wasb://container@account/table/metadata/snap-xxx.avro"). + /// Use IcebergPathResolver::resolve to get storage path for I/O. + /// Use .serialize() to get the path for writing into Iceberg metadata. + Iceberg::IcebergPathFromMetadata manifest_list_path; }; NextMetadataResult generateNextMetadata( FileNamesGenerator & generator, - const String & metadata_filename, + const Iceberg::IcebergPathFromMetadata & metadata_file_path, Int64 parent_snapshot_id, Int64 added_files, Int64 added_records, diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.cpp index 5b4e514536d2..66a731aa1fa8 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.cpp @@ -15,6 +15,7 @@ MultipleFileWriter::MultipleFileWriter( UInt64 max_data_file_num_bytes_, Poco::JSON::Array::Ptr schema, FileNamesGenerator & filename_generator_, + const Iceberg::IcebergPathResolver & path_resolver_, ObjectStoragePtr object_storage_, ContextPtr context_, const std::optional & format_settings_, @@ -26,6 +27,7 @@ MultipleFileWriter::MultipleFileWriter( , aggregate_stats(schema) , current_file_stats(schema) , filename_generator(filename_generator_) + , path_resolver(path_resolver_) , object_storage(object_storage_) , context(context_) , format_settings(format_settings_) @@ -46,14 +48,15 @@ void MultipleFileWriter::startNewFile() current_file_num_rows = 0; current_file_num_bytes = 0; - auto filename = filename_generator.generateDataFileName(); + auto metadata_path = filename_generator.generateDataFileName(); + auto storage_path = path_resolver.resolve(metadata_path); - data_file_names.push_back(filename.path_in_storage); + data_file_names.push_back(metadata_path); if (new_file_path_callback) - new_file_path_callback(filename.path_in_storage); + new_file_path_callback(storage_path); buffer = object_storage->writeObject( - StoredObject(filename.path_in_storage), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + StoredObject(storage_path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); if (format_settings) { @@ -84,7 +87,14 @@ void MultipleFileWriter::finalize() output_format->flush(); output_format->finalize(); buffer->finalize(); - const UInt64 file_bytes = buffer->count(); + UInt64 file_bytes = buffer->count(); + if (file_bytes == 0 && !data_file_names.empty()) + { + /// Some storage backends (e.g. Azure) don't track bytes in the write buffer. + /// Fall back to querying the actual object size. + auto metadata = object_storage->getObjectMetadata(path_resolver.resolve(data_file_names.back()), /*with_tags=*/false); + file_bytes = metadata.size_bytes; + } total_bytes += file_bytes; per_file_record_counts.push_back(static_cast(*current_file_num_rows)); per_file_byte_sizes.push_back(static_cast(file_bytes)); @@ -101,7 +111,7 @@ std::vector MultipleFileWriter::getDataFileEntries() const for (size_t i = 0; i < data_file_names.size(); ++i) entries.emplace_back( - data_file_names[i], + path_resolver.resolve(data_file_names[i]), per_file_record_counts[i], per_file_byte_sizes[i], per_file_stats_list[i]); @@ -125,8 +135,8 @@ void MultipleFileWriter::cancel() void MultipleFileWriter::clearAllDataFiles() const { - for (const auto & data_filename : data_file_names) - object_storage->removeObjectIfExists(StoredObject(data_filename)); + for (const auto & metadata_path : data_file_names) + object_storage->removeObjectIfExists(StoredObject(path_resolver.resolve(metadata_path))); } UInt64 MultipleFileWriter::getResultBytes() const diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.h index eba5511c8d6d..e56400476ad4 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -18,6 +19,7 @@ class MultipleFileWriter UInt64 max_data_file_num_bytes_, Poco::JSON::Array::Ptr schema, FileNamesGenerator & filename_generator_, + const Iceberg::IcebergPathResolver & path_resolver_, ObjectStoragePtr object_storage_, ContextPtr context_, const std::optional & format_settings_, @@ -34,7 +36,7 @@ class MultipleFileWriter UInt64 getResultBytes() const; - const std::vector & getDataFiles() const + const std::vector & getDataFiles() const { return data_file_names; } @@ -56,13 +58,14 @@ class MultipleFileWriter DataFileStatistics current_file_stats; /// accumulates for the current file only std::optional current_file_num_rows = std::nullopt; std::optional current_file_num_bytes = std::nullopt; - std::vector data_file_names; + std::vector data_file_names; std::vector per_file_record_counts; std::vector per_file_byte_sizes; std::vector per_file_stats_list; std::unique_ptr buffer; OutputFormatPtr output_format; FileNamesGenerator & filename_generator; + const Iceberg::IcebergPathResolver & path_resolver; ObjectStoragePtr object_storage; ContextPtr context; std::optional format_settings; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Mutations.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/Mutations.cpp index e9ee547b1b44..55e1cb56150c 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Mutations.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Mutations.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -30,7 +31,6 @@ #include #include #include -#include #include #include #include @@ -48,13 +48,6 @@ namespace DB::DataLakeStorageSetting extern const DataLakeStorageSettingsBool iceberg_use_version_hint; } -namespace DB::Setting -{ -extern const SettingsInt64 iceberg_expire_default_min_snapshots_to_keep; -extern const SettingsInt64 iceberg_expire_default_max_snapshot_age_ms; -extern const SettingsInt64 iceberg_expire_default_max_ref_age_ms; -} - namespace DB::FailPoints { extern const char iceberg_writes_cleanup[]; @@ -65,13 +58,14 @@ namespace DB::Iceberg #if USE_AVRO -static constexpr const char * block_datafile_path = "_path"; +static constexpr const char * block_datafile_path = "_iceberg_metadata_file_path"; static constexpr const char * block_row_number = "_row_number"; static constexpr auto MAX_TRANSACTION_RETRIES = 100; struct DeleteFileWriteResult { - FileNamesGenerator::Result path; + /// Metadata path (e.g. "wasb://container@account/table/data/uuid-deletes.parquet") + Iceberg::IcebergPathFromMetadata path; Int32 total_rows; Int32 total_bytes; }; @@ -144,10 +138,10 @@ static std::optional writeDataFiles( ObjectStoragePtr object_storage, String write_format, FileNamesGenerator & generator, + const Iceberg::IcebergPathResolver & path_resolver, const std::optional & format_settings, std::optional & chunk_partitioner, - Poco::JSON::Object::Ptr data_schema, - const String& blob_storage_namespace_name) + Poco::JSON::Object::Ptr data_schema) { chassert(commands.size() == 1); @@ -201,11 +195,11 @@ static std::optional writeDataFiles( if (!delete_data_writers.contains(partition_key)) { - auto delete_file_info = generator.generatePositionDeleteFile(); + auto delete_file_path = generator.generatePositionDeleteFile(); - delete_data_result[partition_key].path = delete_file_info; + delete_data_result[partition_key].path = delete_file_path; auto write_buffer = object_storage->writeObject( - StoredObject(delete_file_info.path_in_storage), + StoredObject(path_resolver.resolve(delete_file_path)), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, @@ -243,21 +237,9 @@ static std::optional writeDataFiles( col_position.type = removeNullable(col_position.type); } - auto col_data_filename_without_namespaces = ColumnString::create(); - for (size_t i = 0; i < col_data_filename.column->size(); ++i) - { - Field cur_value; - col_data_filename.column->get(i, cur_value); - - String path_without_namespace; - if (cur_value.safeGet().starts_with(blob_storage_namespace_name)) - path_without_namespace = cur_value.safeGet().substr(blob_storage_namespace_name.size()); - - if (!path_without_namespace.starts_with('/')) - path_without_namespace = "/" + path_without_namespace; - col_data_filename_without_namespaces->insert(path_without_namespace); - } - col_data_filename.column = std::move(col_data_filename_without_namespaces); + /// _iceberg_metadata_file_path already contains the correct metadata path format + /// (e.g. wasb://container@host/.../data/xxx.parquet or /iceberg/.../data/xxx.parquet) + /// so no transformation is needed. Columns chunk_pos_delete; chunk_pos_delete.push_back(col_data_filename.column); chunk_pos_delete.push_back(col_position.column); @@ -278,7 +260,13 @@ static std::optional writeDataFiles( delete_data_writers[partition_key]->flush(); delete_data_writers[partition_key]->finalize(); delete_data_write_buffers[partition_key]->finalize(); - delete_data_result[partition_key].total_bytes = static_cast(delete_data_write_buffers[partition_key]->count()); + { + auto delete_bytes = delete_data_write_buffers[partition_key]->count(); + if (delete_bytes == 0) + delete_bytes = object_storage->getObjectMetadata( + path_resolver.resolve(delete_data_result[partition_key].path), /*with_tags=*/ false).size_bytes; + delete_data_result[partition_key].total_bytes = static_cast(delete_bytes); + } } } @@ -314,10 +302,10 @@ static std::optional writeDataFiles( auto it = update_data_writers.find(partition_key); if (it == update_data_writers.end()) { - auto data_file_info = generator.generateDataFileName(); - update_data_result[partition_key].path = data_file_info; + auto data_file_path = generator.generateDataFileName(); + update_data_result[partition_key].path = data_file_path; auto data_write_buffer = object_storage->writeObject( - StoredObject(data_file_info.path_in_storage), + StoredObject(path_resolver.resolve(data_file_path)), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, @@ -341,7 +329,13 @@ static std::optional writeDataFiles( update_data_writers[partition_key]->flush(); update_data_writers[partition_key]->finalize(); update_data_write_buffers[partition_key]->finalize(); - update_data_result[partition_key].total_bytes = static_cast(update_data_write_buffers[partition_key]->count()); + { + auto update_bytes = update_data_write_buffers[partition_key]->count(); + if (update_bytes == 0) + update_bytes = object_storage->getObjectMetadata( + path_resolver.resolve(update_data_result[partition_key].path), /*with_tags=*/ false).size_bytes; + update_data_result[partition_key].total_bytes = static_cast(update_bytes); + } } } @@ -358,6 +352,7 @@ static bool writeMetadataFiles( ObjectStoragePtr object_storage, ContextPtr context, FileNamesGenerator & filename_generator, + const Iceberg::IcebergPathResolver & path_resolver, const DataLakeStorageSettings & data_lake_settings, String write_format, std::shared_ptr catalog, @@ -368,12 +363,10 @@ static bool writeMetadataFiles( std::optional & chunk_partitioner, Iceberg::FileContentType content_type, SharedHeader sample_block, - CompressionMethod compression_method, - bool write_metadata_json_file, - const String& blob_storage_type_name, - const String& blob_storage_namespace_name) + bool write_metadata_json_file) { - auto [metadata_name, storage_metadata_name] = filename_generator.generateMetadataName(); + auto metadata_info = filename_generator.generateMetadataPathWithInfo(); + auto storage_metadata_name = path_resolver.resolve(metadata_info.path); Int64 parent_snapshot = -1; if (metadata->has(Iceberg::f_current_snapshot_id)) parent_snapshot = metadata->getValue(Iceberg::f_current_snapshot_id); @@ -389,53 +382,47 @@ static bool writeMetadataFiles( } Poco::JSON::Object::Ptr new_snapshot; - String manifest_list_name; String storage_manifest_list_name; if (content_type == Iceberg::FileContentType::POSITION_DELETE) { - auto result_generation_metadata = MetadataGenerator(metadata) - .generateNextMetadata( - filename_generator, - metadata_name, - parent_snapshot, - /* added_files */0, - /* added_records */0, - total_bytes, - /* num_partitions */total_files, - /* added_delete_files */total_files, - total_rows); - new_snapshot = result_generation_metadata.snapshot; - manifest_list_name = result_generation_metadata.metadata_path; - storage_manifest_list_name = result_generation_metadata.storage_metadata_path; + auto result = MetadataGenerator(metadata).generateNextMetadata( + filename_generator, + metadata_info.path, + parent_snapshot, + /* added_files */ 0, + /* added_records */ 0, + total_bytes, + /* num_partitions */ total_files, + /* added_delete_files */ total_files, + total_rows); + new_snapshot = result.snapshot; + storage_manifest_list_name = path_resolver.resolve(result.manifest_list_path); } else { - auto result_generation_metadata = MetadataGenerator(metadata) - .generateNextMetadata( - filename_generator, - metadata_name, - parent_snapshot, - /* added_files */total_files, - /* added_records */total_rows, - total_bytes, - /* num_partitions */total_files, - /* added_delete_files */0, - /*num_deleted_rows*/0); - new_snapshot = result_generation_metadata.snapshot; - manifest_list_name = result_generation_metadata.metadata_path; - storage_manifest_list_name = result_generation_metadata.storage_metadata_path; - + auto result = MetadataGenerator(metadata).generateNextMetadata( + filename_generator, + metadata_info.path, + parent_snapshot, + /* added_files */ total_files, + /* added_records */ total_rows, + total_bytes, + /* num_partitions */ total_files, + /* added_delete_files */ 0, + /*num_deleted_rows*/ 0); + new_snapshot = result.snapshot; + storage_manifest_list_name = path_resolver.resolve(result.manifest_list_path); } auto manifest_entries_in_storage = std::make_shared(); - Strings manifest_entries; - Int32 manifest_lengths = 0; + std::vector manifest_entries; + std::vector manifest_entry_sizes; - auto cleanup = [object_storage, delete_filenames, manifest_entries_in_storage, storage_manifest_list_name, storage_metadata_name]() + auto cleanup = [object_storage, &delete_filenames, &path_resolver, manifest_entries_in_storage, storage_manifest_list_name, storage_metadata_name]() { try { for (const auto & [_, data_file] : delete_filenames.delete_file) - object_storage->removeObjectIfExists(StoredObject(data_file.path.path_in_storage)); + object_storage->removeObjectIfExists(StoredObject(path_resolver.resolve(data_file.path))); for (const auto & manifest_filename_in_storage : *manifest_entries_in_storage) object_storage->removeObjectIfExists(StoredObject(manifest_filename_in_storage)); @@ -452,12 +439,12 @@ static bool writeMetadataFiles( { for (const auto & [partition_key, delete_filename] : delete_filenames.delete_file) { - auto [manifest_entry_name, storage_manifest_entry_name] = filename_generator.generateManifestEntryName(); - manifest_entries_in_storage->push_back(storage_manifest_entry_name); - manifest_entries.push_back(manifest_entry_name); + auto manifest_entry_path = filename_generator.generateManifestEntryName(); + manifest_entries_in_storage->push_back(path_resolver.resolve(manifest_entry_path)); + manifest_entries.push_back(manifest_entry_path); auto buffer_manifest_entry = object_storage->writeObject( - StoredObject(storage_manifest_entry_name), + StoredObject(path_resolver.resolve(manifest_entry_path)), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, @@ -469,7 +456,7 @@ static bool writeMetadataFiles( chunk_partitioner ? chunk_partitioner->getColumns() : std::vector{}, partition_key, chunk_partitioner ? chunk_partitioner->getResultTypes() : std::vector{}, - {delete_filename.path.path_in_metadata}, + {delete_filename.path}, delete_filenames.delete_statistic.at(partition_key), sample_block, new_snapshot, @@ -479,7 +466,12 @@ static bool writeMetadataFiles( *buffer_manifest_entry, content_type); buffer_manifest_entry->finalize(); - manifest_lengths += buffer_manifest_entry->count(); + auto size = buffer_manifest_entry->count(); + if (size == 0) + { + size = object_storage->getObjectMetadata(path_resolver.resolve(manifest_entry_path), /*with_tags=*/false).size_bytes; + } + manifest_entry_sizes.push_back(size); } catch (...) { @@ -499,13 +491,13 @@ static bool writeMetadataFiles( try { generateManifestList( - filename_generator, + path_resolver, metadata, object_storage, context, manifest_entries, new_snapshot, - manifest_lengths, + manifest_entry_sizes, *buffer_manifest_list, content_type); buffer_manifest_list->finalize(); @@ -528,15 +520,14 @@ static bool writeMetadataFiles( throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failpoint for cleanup enabled"); }); - auto hint = filename_generator.generateVersionHint(); + auto hint_path = filename_generator.generateVersionHint(); if (!writeMetadataFileAndVersionHint( - storage_metadata_name, + path_resolver, + metadata_info, json_representation, - hint.path_in_storage, - storage_metadata_name, + hint_path, object_storage, context, - compression_method, data_lake_settings[DataLakeStorageSetting::iceberg_use_version_hint])) { cleanup(); @@ -545,10 +536,7 @@ static bool writeMetadataFiles( if (catalog) { - String catalog_filename = metadata_name; - if (!catalog_filename.starts_with(blob_storage_type_name)) - catalog_filename = blob_storage_type_name + "://" + blob_storage_namespace_name + "/" + metadata_name; - + auto catalog_filename = path_resolver.resolveForCatalog(metadata_info.path); const auto & [namespace_name, table_name] = DataLake::parseTableName(table_id.getTableName()); if (!catalog->updateMetadata(namespace_name, table_name, catalog_filename, new_snapshot)) { @@ -576,9 +564,7 @@ void mutate( PersistentTableComponents & persistent_table_components, const String & write_format, const std::optional & format_settings, - std::shared_ptr catalog, - const String & blob_storage_type_name, - const String & blob_storage_namespace_name) + std::shared_ptr catalog) { auto common_path = persistent_table_components.table_path; if (!common_path.starts_with('/')) @@ -587,7 +573,6 @@ void mutate( int max_retries = MAX_TRANSACTION_RETRIES; while (--max_retries > 0) { - FileNamesGenerator filename_generator(common_path, common_path, false, CompressionMethod::None, write_format); auto log = getLogger("IcebergMutations"); /// Mutations must always operate on the actual latest metadata, regardless of /// any explicit iceberg_metadata_file_path set on the table (used for time-travel reads). @@ -599,9 +584,11 @@ void mutate( context, log.get(), persistent_table_components.table_uuid, + persistent_table_components.metadata_compression_method, /* force_fetch_latest_metadata */ true, /* ignore_explicit_metadata_file_path */ true); + FileNamesGenerator filename_generator(persistent_table_components.path_resolver.getTableLocation(), false, CompressionMethod::None, write_format); filename_generator.setVersion(last_version + 1); filename_generator.setCompressionMethod(compression_method); @@ -645,10 +632,10 @@ void mutate( object_storage, write_format, filename_generator, + persistent_table_components.path_resolver, format_settings, chunk_partitioner, - current_schema, - blob_storage_namespace_name); + current_schema); if (mutation_files) { @@ -657,6 +644,7 @@ void mutate( object_storage, context, filename_generator, + persistent_table_components.path_resolver, data_lake_settings, write_format, catalog, @@ -667,10 +655,7 @@ void mutate( chunk_partitioner, Iceberg::FileContentType::POSITION_DELETE, std::make_shared(getPositionDeleteFileSampleBlock()), - compression_method, - !mutation_files->data_file, - blob_storage_type_name, - blob_storage_namespace_name); + !mutation_files->data_file); if (!result_delete_files_metadata) continue; @@ -681,6 +666,7 @@ void mutate( object_storage, context, filename_generator, + persistent_table_components.path_resolver, data_lake_settings, write_format, catalog, @@ -691,10 +677,7 @@ void mutate( chunk_partitioner, Iceberg::FileContentType::DATA, sample_block, - compression_method, - true, - blob_storage_type_name, - blob_storage_namespace_name); + true); if (!result_data_files_metadata) { continue; @@ -722,8 +705,6 @@ void alter( size_t i = 0; while (i++ < MAX_TRANSACTION_RETRIES) { - FileNamesGenerator filename_generator( - persistent_table_components.table_path, persistent_table_components.table_path, false, CompressionMethod::None, write_format); auto log = getLogger("IcebergMutations"); auto [last_version, metadata_path, compression_method] = getLatestOrExplicitMetadataFileAndVersion( object_storage, @@ -733,9 +714,11 @@ void alter( context, log.get(), persistent_table_components.table_uuid, + persistent_table_components.metadata_compression_method, /* force_fetch_latest_metadata */ true, /* ignore_explicit_metadata_file_path */ true); + FileNamesGenerator filename_generator(persistent_table_components.path_resolver.getTableLocation(), false, CompressionMethod::None, write_format); filename_generator.setVersion(last_version + 1); filename_generator.setCompressionMethod(compression_method); @@ -767,17 +750,16 @@ void alter( Poco::JSON::Stringifier::stringify(metadata, oss, 4); std::string json_representation = removeEscapedSlashes(oss.str()); - auto [metadata_name, storage_metadata_name] = filename_generator.generateMetadataName(); + auto metadata_info = filename_generator.generateMetadataPathWithInfo(); - auto hint = filename_generator.generateVersionHint(); + auto hint_path = filename_generator.generateVersionHint(); if (writeMetadataFileAndVersionHint( - storage_metadata_name, + persistent_table_components.path_resolver, + metadata_info, json_representation, - hint.path_in_storage, - storage_metadata_name, + hint_path, object_storage, context, - compression_method, data_lake_settings[DataLakeStorageSetting::iceberg_use_version_hint])) break; } @@ -786,680 +768,6 @@ void alter( throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Too many unsuccessed retries to alter iceberg table"); } -/// Table-level snapshot retention policy read from Iceberg table properties. -struct RetentionPolicy -{ - Int32 min_snapshots_to_keep = Iceberg::default_min_snapshots_to_keep; - Int64 max_snapshot_age_ms = Iceberg::default_max_snapshot_age_ms; - Int64 max_ref_age_ms = Iceberg::default_max_ref_age_ms; -}; - -static RetentionPolicy readRetentionPolicy(const Poco::JSON::Object::Ptr & metadata, ContextPtr context, const ExpireSnapshotsOptions & options) -{ - RetentionPolicy policy; - const auto & settings = context->getSettingsRef(); - Int64 min_keep_from_settings = settings[Setting::iceberg_expire_default_min_snapshots_to_keep].value; - Int64 max_snapshot_age_from_settings = settings[Setting::iceberg_expire_default_max_snapshot_age_ms].value; - Int64 max_ref_age_from_settings = settings[Setting::iceberg_expire_default_max_ref_age_ms].value; - - if (min_keep_from_settings <= 0 || min_keep_from_settings > std::numeric_limits::max()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "iceberg_expire_default_min_snapshots_to_keep must be in range [1, {}], got {}", - std::numeric_limits::max(), - min_keep_from_settings); - if (max_snapshot_age_from_settings < 0) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "iceberg_expire_default_max_snapshot_age_ms must be non-negative, got {}", - max_snapshot_age_from_settings); - if (max_ref_age_from_settings < 0) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "iceberg_expire_default_max_ref_age_ms must be non-negative, got {}", - max_ref_age_from_settings); - - policy.min_snapshots_to_keep = static_cast(min_keep_from_settings); - policy.max_snapshot_age_ms = max_snapshot_age_from_settings; - policy.max_ref_age_ms = max_ref_age_from_settings; - - if (metadata->has(Iceberg::f_properties)) - { - auto props = metadata->getObject(Iceberg::f_properties); - if (props->has(Iceberg::f_min_snapshots_to_keep)) - policy.min_snapshots_to_keep = std::stoi(props->getValue(Iceberg::f_min_snapshots_to_keep)); - if (props->has(Iceberg::f_max_snapshot_age_ms)) - policy.max_snapshot_age_ms = std::stoll(props->getValue(Iceberg::f_max_snapshot_age_ms)); - if (props->has(Iceberg::f_max_ref_age_ms)) - policy.max_ref_age_ms = std::stoll(props->getValue(Iceberg::f_max_ref_age_ms)); - } - - /// Per-invocation overrides (only affect table-level defaults, not per-ref overrides). - if (options.retain_last.has_value()) - policy.min_snapshots_to_keep = *options.retain_last; - if (options.retention_period_ms.has_value()) - policy.max_snapshot_age_ms = *options.retention_period_ms; - - return policy; -} - -/// Snapshot parent graph built from metadata, used for branch ancestor traversal. -class SnapshotGraph -{ -public: - explicit SnapshotGraph(const Poco::JSON::Array::Ptr & snapshots) - { - for (UInt32 i = 0; i < snapshots->size(); ++i) - { - auto snapshot = snapshots->getObject(i); - Int64 snap_id = snapshot->getValue(Iceberg::f_metadata_snapshot_id); - timestamps[snap_id] = snapshot->getValue(Iceberg::f_timestamp_ms); - if (snapshot->has(Iceberg::f_parent_snapshot_id) && !snapshot->isNull(Iceberg::f_parent_snapshot_id)) - parent_chain[snap_id] = snapshot->getValue(Iceberg::f_parent_snapshot_id); - } - } - - bool hasSnapshot(Int64 snap_id) const { return timestamps.contains(snap_id); } - - Int64 getTimestamp(Int64 snap_id) const { return timestamps.at(snap_id); } - - std::optional getParent(Int64 snap_id) const - { - auto it = parent_chain.find(snap_id); - return it != parent_chain.end() ? std::optional(it->second) : std::nullopt; - } - - /// Retain ancestors from head_id while min-keep or max-age is satisfied. - void walkBranchAncestors(Int64 now_ms, Int64 head_id, Int32 min_keep, Int64 max_age_ms, std::set & retained) const - { - Int64 walk_id = head_id; - Int32 count = 0; - while (hasSnapshot(walk_id)) - { - bool within_min_keep = (count < min_keep); - bool within_max_age = (now_ms - getTimestamp(walk_id) <= max_age_ms); - if (!within_min_keep && !within_max_age) - break; - retained.insert(walk_id); - ++count; - auto parent = getParent(walk_id); - if (!parent) - break; - walk_id = *parent; - } - } - -private: - std::unordered_map parent_chain; - std::unordered_map timestamps; -}; - -/// Apply Iceberg Snapshot Retention Policy. Returns (retained IDs, expired ref names). -static std::pair, Strings> applyRetentionPolicy( - const Poco::JSON::Object::Ptr & metadata, - Int64 current_snapshot_id, - const SnapshotGraph & graph, - const RetentionPolicy & policy, - Int64 now_ms) -{ - std::set retained; - Strings expired_ref_names; - bool main_branch_walked = false; - if (metadata->has(Iceberg::f_refs)) - { - auto refs = metadata->getObject(Iceberg::f_refs); - for (const auto & ref_name : refs->getNames()) - { - auto ref_obj = refs->getObject(ref_name); - Int64 ref_snap_id = ref_obj->getValue(Iceberg::f_metadata_snapshot_id); - String ref_type = ref_obj->getValue(Iceberg::f_type); - - Int64 ref_max_ref_age = ref_obj->has(Iceberg::f_ref_max_ref_age_ms) - ? ref_obj->getValue(Iceberg::f_ref_max_ref_age_ms) - : policy.max_ref_age_ms; - - bool is_main = (ref_name == Iceberg::f_main); - - if (!is_main && !graph.hasSnapshot(ref_snap_id)) - { - LOG_WARNING(getLogger("IcebergExpireSnapshots"), - "Removing invalid ref {}: snapshot {} does not exist", ref_name, ref_snap_id); - expired_ref_names.push_back(ref_name); - continue; - } - - bool ref_expired = !is_main && (now_ms - graph.getTimestamp(ref_snap_id)) > ref_max_ref_age; - - if (ref_expired) - { - expired_ref_names.push_back(ref_name); - continue; - } - - if (ref_type == Iceberg::f_branch) - { - Int32 min_keep = ref_obj->has(Iceberg::f_ref_min_snapshots_to_keep) - ? ref_obj->getValue(Iceberg::f_ref_min_snapshots_to_keep) - : policy.min_snapshots_to_keep; - Int64 max_age = ref_obj->has(Iceberg::f_ref_max_snapshot_age_ms) - ? ref_obj->getValue(Iceberg::f_ref_max_snapshot_age_ms) - : policy.max_snapshot_age_ms; - graph.walkBranchAncestors(now_ms, ref_snap_id, min_keep, max_age, retained); - if (is_main) - main_branch_walked = true; - } - else if (ref_type == Iceberg::f_tag) - { - retained.insert(ref_snap_id); - } - else - { - UNREACHABLE(); - } - } - } - - if (!main_branch_walked) - graph.walkBranchAncestors(now_ms, current_snapshot_id, policy.min_snapshots_to_keep, policy.max_snapshot_age_ms, retained); - - return {retained, expired_ref_names}; -} - -static void collectAllFilePaths( - const Iceberg::ManifestFileIterator::ManifestFileEntriesHandle & entries_handle, - std::set & out) -{ - for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::DATA)) - out.insert(entry->file_path); - for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::POSITION_DELETE)) - out.insert(entry->file_path); - for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::EQUALITY_DELETE)) - out.insert(entry->file_path); -} - -/// Collect all file paths (manifest lists, manifests, data/delete files) -/// referenced by retained snapshots. -/// -/// NOTE: We only collect files with status ADDED/EXISTING (via getFilesWithoutDeleted). -/// Files with status DELETED are being removed by that snapshot and don't need retention -/// from it. A DELETED entry's data file was ADDED in an earlier snapshot — if that snapshot -/// is retained, the file is in the retained set from there; if expired, it will be collected -/// for cleanup from that snapshot's ADDED/EXISTING entries. -/// -/// TODO: To handle partially-failed prior expire_snapshots (where the ADDED snapshot -/// was removed but its data files were not cleaned up), we could also traverse DELETED -/// entries in expired manifests. This requires extending ManifestFileIterator to expose -/// DELETED entries. -static void collectRetainedFiles( - const Poco::JSON::Array::Ptr & retained_snapshots, - ObjectStoragePtr object_storage, - PersistentTableComponents & persistent_table_components, - ContextPtr context, - LoggerPtr log, - Int32 current_schema_id, - std::set & retained_manifest_paths, - std::set & retained_data_file_paths, - std::set & retained_manifest_list_paths) -{ - for (UInt32 i = 0; i < retained_snapshots->size(); ++i) - { - auto snapshot = retained_snapshots->getObject(i); - if (!snapshot->has(Iceberg::f_manifest_list)) - continue; - - String manifest_list_path = snapshot->getValue(Iceberg::f_manifest_list); - retained_manifest_list_paths.insert(manifest_list_path); - - String storage_manifest_list_path = getProperFilePathFromMetadataInfo( - manifest_list_path, persistent_table_components.table_path, persistent_table_components.table_location); - - auto manifest_keys = getManifestList( - object_storage, persistent_table_components, context, storage_manifest_list_path, log); - - for (const auto & mf_key : manifest_keys) - { - retained_manifest_paths.insert(mf_key.manifest_file_path); - auto entries_handle = getManifestFileEntriesHandle( - object_storage, persistent_table_components, context, log, - mf_key, current_schema_id); - collectAllFilePaths(entries_handle, retained_data_file_paths); - } - } -} - -struct ExpiredFiles -{ - Strings all_paths; - Int64 data_files = 0; - Int64 position_delete_files = 0; - Int64 equality_delete_files = 0; - Int64 manifest_files = 0; - Int64 manifest_lists = 0; -}; - -/// Collect files from expired snapshots that are not referenced by any retained snapshot. -static ExpiredFiles collectExpiredFiles( - const std::vector & expired_manifest_list_paths, - const std::set & retained_manifest_list_paths, - const std::set & retained_manifest_paths, - const std::set & retained_data_file_paths, - ObjectStoragePtr object_storage, - PersistentTableComponents & persistent_table_components, - ContextPtr context, - LoggerPtr log, - Int32 current_schema_id) -{ - ExpiredFiles result; - std::set seen_expired_manifest_list_paths; - std::set seen_expired_manifest_paths; - for (const auto & ml_path : expired_manifest_list_paths) - { - if (retained_manifest_list_paths.contains(ml_path)) - continue; - - if (seen_expired_manifest_list_paths.contains(ml_path)) - continue; - - String storage_ml_path = getProperFilePathFromMetadataInfo( - ml_path, persistent_table_components.table_path, persistent_table_components.table_location); - - ManifestFileCacheKeys manifest_keys; - try - { - manifest_keys = getManifestList( - object_storage, persistent_table_components, context, storage_ml_path, log); - } - catch (...) - { - LOG_WARNING(log, "Failed to read manifest list {}, skipping", storage_ml_path); - continue; - } - - for (const auto & mf_key : manifest_keys) - { - if (retained_manifest_paths.contains(mf_key.manifest_file_path)) - continue; - - if (seen_expired_manifest_paths.contains(mf_key.manifest_file_path)) - continue; - - try - { - auto entries_handle = getManifestFileEntriesHandle( - object_storage, persistent_table_components, context, log, - mf_key, current_schema_id); - - for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::DATA)) - if (!retained_data_file_paths.contains(entry->file_path)) - { - result.all_paths.push_back(entry->file_path); - ++result.data_files; - } - for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::POSITION_DELETE)) - if (!retained_data_file_paths.contains(entry->file_path)) - { - result.all_paths.push_back(entry->file_path); - ++result.position_delete_files; - } - for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::EQUALITY_DELETE)) - if (!retained_data_file_paths.contains(entry->file_path)) - { - result.all_paths.push_back(entry->file_path); - ++result.equality_delete_files; - } - } - catch (...) - { - LOG_WARNING(log, "Failed to read manifest file {}, skipping", mf_key.manifest_file_path); - continue; - } - - seen_expired_manifest_paths.insert(mf_key.manifest_file_path); - result.all_paths.push_back(mf_key.manifest_file_path); - ++result.manifest_files; - } - - seen_expired_manifest_list_paths.insert(ml_path); - result.all_paths.push_back(storage_ml_path); - ++result.manifest_lists; - } - return result; -} - -/// Trim snapshot-log to the suffix of entries referencing only retained snapshots. -static void trimSnapshotLog( - Poco::JSON::Object::Ptr metadata, - const std::set & expired_snapshot_ids) -{ - if (!metadata->has(Iceberg::f_snapshot_log)) - return; - - auto snapshot_log = metadata->get(Iceberg::f_snapshot_log).extract(); - Int32 suffix_start = static_cast(snapshot_log->size()); - for (Int32 j = static_cast(snapshot_log->size()) - 1; j >= 0; --j) - { - auto entry = snapshot_log->getObject(static_cast(j)); - Int64 snap_id = entry->getValue(Iceberg::f_metadata_snapshot_id); - if (expired_snapshot_ids.contains(snap_id)) - break; - suffix_start = j; - } - Poco::JSON::Array::Ptr retained_log = new Poco::JSON::Array; - for (UInt32 j = static_cast(suffix_start); j < snapshot_log->size(); ++j) - retained_log->add(snapshot_log->getObject(j)); - metadata->set(Iceberg::f_snapshot_log, retained_log); -} - -struct SnapshotPartition -{ - Poco::JSON::Array::Ptr retained_snapshots = new Poco::JSON::Array; - std::set expired_snapshot_ids; - std::vector expired_manifest_list_paths; -}; - -/// Split snapshots into retained and expired. -/// A snapshot is retained if the retention policy selected it, or if the -/// user-provided fuse timestamp protects it (snapshot newer than fuse). -static SnapshotPartition partitionSnapshots( - const Poco::JSON::Array::Ptr & snapshots, - const std::set & retention_retained_ids, - std::optional expire_before_ms) -{ - SnapshotPartition result; - for (UInt32 i = 0; i < snapshots->size(); ++i) - { - auto snapshot = snapshots->getObject(i); - Int64 snap_id = snapshot->getValue(Iceberg::f_metadata_snapshot_id); - Int64 snap_ts = snapshot->getValue(Iceberg::f_timestamp_ms); - - bool is_retained_by_policy = retention_retained_ids.contains(snap_id); - bool is_protected_by_fuse = expire_before_ms.has_value() && (snap_ts >= *expire_before_ms); - - if (is_retained_by_policy || is_protected_by_fuse) - { - result.retained_snapshots->add(snapshot); - } - else - { - result.expired_snapshot_ids.insert(snap_id); - if (snapshot->has(Iceberg::f_manifest_list)) - result.expired_manifest_list_paths.push_back(snapshot->getValue(Iceberg::f_manifest_list)); - } - } - return result; -} - -static SnapshotPartition partitionSnapshotsByIds( - const Poco::JSON::Object::Ptr & metadata, - const Poco::JSON::Array::Ptr & snapshots, - const std::vector & snapshot_ids, - Int64 current_snapshot_id, - std::optional expire_before_ms) -{ - std::unordered_set requested_ids(snapshot_ids.begin(), snapshot_ids.end()); - std::unordered_set existing_ids; - std::unordered_set ref_protected_ids; - SnapshotPartition result; - - if (metadata->has(Iceberg::f_refs)) - { - auto refs = metadata->getObject(Iceberg::f_refs); - for (const auto & ref_name : refs->getNames()) - { - auto ref = refs->getObject(ref_name); - if (ref->has(Iceberg::f_metadata_snapshot_id)) - ref_protected_ids.insert(ref->getValue(Iceberg::f_metadata_snapshot_id)); - } - } - - ref_protected_ids.insert(current_snapshot_id); - - for (UInt32 i = 0; i < snapshots->size(); ++i) - { - auto snapshot = snapshots->getObject(i); - Int64 snap_id = snapshot->getValue(Iceberg::f_metadata_snapshot_id); - Int64 snap_ts = snapshot->getValue(Iceberg::f_timestamp_ms); - - existing_ids.insert(snap_id); - bool requested = requested_ids.contains(snap_id); - bool is_protected_by_fuse = expire_before_ms.has_value() && (snap_ts >= *expire_before_ms); - - if (requested && ref_protected_ids.contains(snap_id)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "expire_snapshots cannot expire snapshot {} because it is referenced by current snapshot, branch, or tag", - snap_id); - - if (requested && !is_protected_by_fuse) - { - result.expired_snapshot_ids.insert(snap_id); - if (snapshot->has(Iceberg::f_manifest_list)) - result.expired_manifest_list_paths.push_back(snapshot->getValue(Iceberg::f_manifest_list)); - } - else - { - result.retained_snapshots->add(snapshot); - } - } - - for (Int64 requested_id : requested_ids) - { - if (!existing_ids.contains(requested_id)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots snapshot id {} does not exist", requested_id); - } - - return result; -} - -/// Mutate metadata: remove expired refs, update snapshots, trim log, bump timestamp. -static void updateMetadataForExpiration( - Poco::JSON::Object::Ptr metadata, - const Strings & expired_ref_names, - const Poco::JSON::Array::Ptr & retained_snapshots, - const std::set & expired_snapshot_ids) -{ - for (const auto & ref_name : expired_ref_names) - metadata->getObject(Iceberg::f_refs)->remove(ref_name); - - metadata->set(Iceberg::f_snapshots, retained_snapshots); - trimSnapshotLog(metadata, expired_snapshot_ids); - - auto now = std::chrono::system_clock::now(); - auto ms = duration_cast(now.time_since_epoch()); - metadata->set(Iceberg::f_last_updated_ms, ms.count()); -} - -static void deleteExpiredFiles( - const Strings & files_to_delete, - ObjectStoragePtr object_storage, - LoggerPtr log) -{ - for (const auto & file_path : files_to_delete) - { - try - { - object_storage->removeObjectIfExists(StoredObject(file_path)); - LOG_DEBUG(log, "Deleted expired file {}", file_path); - } - catch (...) - { - LOG_WARNING(log, "Failed to delete file {}: {}", file_path, getCurrentExceptionMessage(false)); - } - } -} - -/// Expire old Iceberg snapshots following the spec's Snapshot Retention Policy. -/// -/// The process: -/// 1. Read retention policy from table properties (with spec defaults). -/// 2. Build the snapshot parent graph and determine which snapshots to retain -/// based on branch/tag refs and their min-snapshots-to-keep / max-snapshot-age-ms. -/// 3. If the caller provided expire_before_ms, it acts as an additional safety -/// fuse — snapshots newer than this timestamp are never expired regardless -/// of retention policy. -/// 4. Collect files exclusively owned by expired snapshots and delete them. -/// 5. Write updated metadata with optimistic concurrency (retry on conflict). -ExpireSnapshotsResult expireSnapshots( - const ExpireSnapshotsOptions & options, - ContextPtr context, - ObjectStoragePtr object_storage, - const DataLakeStorageSettings & data_lake_settings, - PersistentTableComponents & persistent_table_components, - const String & write_format, - std::shared_ptr catalog, - const String & blob_storage_type_name, - const String & blob_storage_namespace_name, - const String & table_name) -{ - auto common_path = persistent_table_components.table_path; - if (!common_path.starts_with('/')) - common_path = "/" + common_path; - - int max_retries = MAX_TRANSACTION_RETRIES; - while (--max_retries > 0) - { - FileNamesGenerator filename_generator(common_path, common_path, false, CompressionMethod::None, write_format); - auto log = getLogger("IcebergExpireSnapshots"); - auto [last_version, metadata_path, compression_method] = getLatestOrExplicitMetadataFileAndVersion( - object_storage, - persistent_table_components.table_path, - data_lake_settings, - persistent_table_components.metadata_cache, - context, - log.get(), - persistent_table_components.table_uuid, - /* force_fetch_latest_metadata */ true, - /* ignore_explicit_metadata_file_path */ true); - - filename_generator.setVersion(last_version + 1); - filename_generator.setCompressionMethod(compression_method); - - auto metadata = getMetadataJSONObject( - metadata_path, object_storage, persistent_table_components.metadata_cache, context, log, compression_method, persistent_table_components.table_uuid); - - if (metadata->getValue(f_format_version) < 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "expire_snapshots is supported only for the second version of iceberg format"); - - if (!metadata->has(Iceberg::f_current_snapshot_id)) - { - LOG_INFO(log, "No snapshots to expire (table has no current snapshot)"); - return {.dry_run = options.dry_run}; - } - - Int64 current_snapshot_id = metadata->getValue(Iceberg::f_current_snapshot_id); - if (current_snapshot_id < 0) - { - LOG_INFO(log, "No snapshots to expire (table has no current snapshot)"); - return {.dry_run = options.dry_run}; - } - - auto snapshots = metadata->get(Iceberg::f_snapshots).extract(); - auto now_ms = duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - - Strings expired_ref_names; - SnapshotPartition partition; - if (options.snapshot_ids.has_value()) - { - partition = partitionSnapshotsByIds(metadata, snapshots, *options.snapshot_ids, current_snapshot_id, options.expire_before_ms); - } - else - { - auto policy = readRetentionPolicy(metadata, context, options); - SnapshotGraph graph(snapshots); - auto [retention_retained_ids, retention_expired_ref_names] = applyRetentionPolicy(metadata, current_snapshot_id, graph, policy, now_ms); - expired_ref_names = std::move(retention_expired_ref_names); - partition = partitionSnapshots(snapshots, retention_retained_ids, options.expire_before_ms); - } - - if (partition.expired_snapshot_ids.empty()) - { - LOG_INFO(log, "No snapshots to expire"); - return {.dry_run = options.dry_run}; - } - LOG_INFO(log, "Expiring {} snapshots", partition.expired_snapshot_ids.size()); - - Int32 current_schema_id = metadata->getValue(Iceberg::f_current_schema_id); - - std::set retained_manifest_paths; - std::set retained_data_file_paths; - std::set retained_manifest_list_paths; - collectRetainedFiles( - partition.retained_snapshots, object_storage, persistent_table_components, context, log, - current_schema_id, retained_manifest_paths, retained_data_file_paths, retained_manifest_list_paths); - auto expired_files = collectExpiredFiles( - partition.expired_manifest_list_paths, retained_manifest_list_paths, retained_manifest_paths, retained_data_file_paths, - object_storage, persistent_table_components, context, log, current_schema_id); - - if (options.dry_run) - { - LOG_INFO(log, "Dry-run mode: skip metadata commit and file deletion"); - return ExpireSnapshotsResult{ - .deleted_data_files_count = expired_files.data_files, - .deleted_position_delete_files_count = expired_files.position_delete_files, - .deleted_equality_delete_files_count = expired_files.equality_delete_files, - .deleted_manifest_files_count = expired_files.manifest_files, - .deleted_manifest_lists_count = expired_files.manifest_lists, - .dry_run = true, - }; - } - - updateMetadataForExpiration(metadata, expired_ref_names, partition.retained_snapshots, partition.expired_snapshot_ids); - - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - Poco::JSON::Stringifier::stringify(metadata, oss, 4); - std::string json_representation = removeEscapedSlashes(oss.str()); - auto [metadata_name, storage_metadata_name] = filename_generator.generateMetadataName(); - auto hint = filename_generator.generateVersionHint(); - if (!writeMetadataFileAndVersionHint( - storage_metadata_name, - json_representation, - hint.path_in_storage, - storage_metadata_name, - object_storage, - context, - compression_method, - data_lake_settings[DataLakeStorageSetting::iceberg_use_version_hint])) - { - LOG_WARNING(log, "Metadata commit conflict during expire_snapshots, retrying ({} retries left)", max_retries); - continue; - } - - if (catalog) - { - String catalog_filename = metadata_name; - if (!catalog_filename.starts_with(blob_storage_type_name)) - catalog_filename = blob_storage_type_name + "://" + blob_storage_namespace_name + "/" + metadata_name; - - const auto & [namespace_name, parsed_table_name] = DataLake::parseTableName(table_name); - if (!catalog->updateMetadata(namespace_name, parsed_table_name, catalog_filename, nullptr)) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Failed to update catalog metadata after writing new metadata file. " - "The table metadata may be in an inconsistent state"); - } - } - - LOG_INFO(log, "Deleting {} expired files for {} expired snapshots", expired_files.all_paths.size(), partition.expired_snapshot_ids.size()); - deleteExpiredFiles(expired_files.all_paths, object_storage, log); - LOG_INFO(log, "Expired {} snapshots, deleted {} files", partition.expired_snapshot_ids.size(), expired_files.all_paths.size()); - - return ExpireSnapshotsResult{ - .deleted_data_files_count = expired_files.data_files, - .deleted_position_delete_files_count = expired_files.position_delete_files, - .deleted_equality_delete_files_count = expired_files.equality_delete_files, - .deleted_manifest_files_count = expired_files.manifest_files, - .deleted_manifest_lists_count = expired_files.manifest_lists, - .dry_run = false, - }; - } - - if (max_retries == 0) - throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Too many unsuccessful retries to expire iceberg snapshots"); - - UNREACHABLE(); -} - #endif } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Mutations.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Mutations.h index f46ec2823509..c4bd0c76dd03 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Mutations.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Mutations.h @@ -15,7 +15,6 @@ #include #include #include -#include #include namespace DB::Iceberg @@ -31,9 +30,7 @@ void mutate( PersistentTableComponents & persistent_table_components, const String & write_format, const std::optional & format_settings, - std::shared_ptr catalog, - const String & blob_storage_type_name, - const String & blob_storage_namespace_name); + std::shared_ptr catalog); void alter( const AlterCommands & params, @@ -43,17 +40,6 @@ void alter( PersistentTableComponents & persistent_table_components, const String & write_format); -ExpireSnapshotsResult expireSnapshots( - const ExpireSnapshotsOptions & options, - ContextPtr context, - ObjectStoragePtr object_storage, - const DataLakeStorageSettings & data_lake_settings, - PersistentTableComponents & persistent_table_components, - const String & write_format, - std::shared_ptr catalog, - const String & blob_storage_type_name, - const String & blob_storage_namespace_name, - const String & table_name); } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/PersistentTableComponents.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/PersistentTableComponents.h index 376e2ef536d2..133713df16ff 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/PersistentTableComponents.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/PersistentTableComponents.h @@ -5,6 +5,7 @@ #include #include +#include #include namespace DB::Iceberg @@ -21,6 +22,7 @@ struct PersistentTableComponents const String table_path; const std::optional table_uuid; const String common_namespace; + const IcebergPathResolver path_resolver; }; } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/PositionDeleteTransform.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/PositionDeleteTransform.cpp index 8d7820f9ea71..d60719dc3b4f 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/PositionDeleteTransform.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/PositionDeleteTransform.cpp @@ -60,7 +60,7 @@ Poco::JSON::Array::Ptr IcebergPositionDeleteTransform::getSchemaFields() void IcebergPositionDeleteTransform::initializeDeleteSources() { /// Create filter on the data object to get interested rows - auto iceberg_data_path = iceberg_object_info->info.data_object_file_path_key; + auto iceberg_data_path = iceberg_object_info->info.data_object_file_path_key.serialize(); ASTPtr where_ast = makeASTFunction( "equals", make_intrusive(IcebergPositionDeleteTransform::data_file_path_column_name), @@ -155,10 +155,10 @@ void IcebergBitmapPositionDeleteTransform::initialize() { // Add filename matching check auto filename_in_delete_record = filename_column->getDataAt(i); - auto current_data_file_path = iceberg_object_info->info.data_object_file_path_key; + auto current_data_file_path = iceberg_object_info->info.data_object_file_path_key.serialize(); // Only add to delete bitmap when the filename in delete record matches current data file path - if (filename_in_delete_record == current_data_file_path || filename_in_delete_record == "/" + current_data_file_path) + if (filename_in_delete_record == current_data_file_path) { auto position_to_delete = position_column->get64(i); bitmap.add(position_to_delete); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/RemoveOrphanFilesExecute.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/RemoveOrphanFilesExecute.cpp new file mode 100644 index 000000000000..6783fdd4ea66 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/RemoveOrphanFilesExecute.cpp @@ -0,0 +1,353 @@ +#include "config.h" +#if USE_AVRO + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +namespace Setting +{ +extern const SettingsUInt64 iceberg_orphan_files_older_than_seconds; +} + +namespace Iceberg +{ + +struct RemoveOrphanFilesParams +{ + std::optional older_than; + std::optional location; + bool dry_run = false; +}; + +struct RemoveOrphanFilesResult +{ + Int64 deleted_data_files_count = 0; + Int64 deleted_position_delete_files_count = 0; + Int64 deleted_equality_delete_files_count = 0; + Int64 deleted_manifest_files_count = 0; + Int64 deleted_manifest_lists_count = 0; + Int64 deleted_metadata_files_count = 0; + Int64 deleted_statistics_files_count = 0; + Int64 skipped_missing_metadata_count = 0; + Int64 failed_deletions_count = 0; +}; + +struct OrphanScanResult +{ + std::vector orphan_paths; + Int64 skipped_missing_metadata = 0; +}; + +namespace +{ + +// --------------------------------------------------------------------------- +// Argument parsing +// --------------------------------------------------------------------------- + +ExecuteCommandArgs makeSchema() +{ + ExecuteCommandArgs schema("remove_orphan_files"); + schema.addPositional("older_than", Field::Types::String); + schema.addNamed("location", Field::Types::String); + schema.addNamed("dry_run", Field::Types::UInt64); + schema.addDefault("dry_run", Field(UInt64(0))); + return schema; +} + +// --------------------------------------------------------------------------- +// Orphan file detection +// --------------------------------------------------------------------------- + +String resolveScanPath(const String & table_path, const RemoveOrphanFilesParams & params) +{ + String scan_path = table_path; + if (!scan_path.ends_with('/')) + scan_path += '/'; + + if (params.location.has_value()) + { + String loc = *params.location; + if (loc.find("..") != String::npos || loc.starts_with('/')) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "location must be a relative path under the table root, got '{}'", loc); + + while (loc.starts_with("./")) + loc = loc.substr(2); + + scan_path += loc; + + if (!scan_path.ends_with('/')) + scan_path += '/'; + } + return scan_path; +} + +OrphanScanResult findOrphanFiles( + const RelativePathsWithMetadata & actual_files, + const std::unordered_set & reachable, + time_t older_than_threshold, + ObjectStoragePtr object_storage, + LoggerPtr log) +{ + OrphanScanResult scan; + + for (const auto & file_ptr : actual_files) + { + const String & path = file_ptr->relative_path; + + if (reachable.contains(path)) + continue; + + std::optional fetched_metadata; + if (!file_ptr->metadata.has_value()) + { + fetched_metadata = object_storage->tryGetObjectMetadata(path, /* with_tags = */ false); + if (!fetched_metadata) + { + ++scan.skipped_missing_metadata; + LOG_DEBUG(log, "Skipping file without metadata (no last_modified): {}", path); + continue; + } + } + + const auto & metadata = file_ptr->metadata.has_value() ? *file_ptr->metadata : *fetched_metadata; + auto file_modified = metadata.last_modified.epochTime(); + if (static_cast(file_modified) >= older_than_threshold) + continue; + + LOG_DEBUG(log, "Orphan file: {}", path); + scan.orphan_paths.push_back(path); + } + + if (scan.skipped_missing_metadata > 0) + LOG_WARNING(log, "Skipped {} unreferenced file(s) because last_modified metadata was unavailable; " + "these files could not be age-checked and were conservatively kept", scan.skipped_missing_metadata); + + return scan; +} + +// --------------------------------------------------------------------------- +// Orphan file deletion +// --------------------------------------------------------------------------- + +struct DeleteOrphanResult +{ + std::vector deleted_paths; + Int64 failed_count = 0; +}; + +DeleteOrphanResult deleteOrphanFiles( + const std::vector & orphan_paths, + ObjectStoragePtr object_storage, + LoggerPtr log) +{ + DeleteOrphanResult result; + + for (const auto & path : orphan_paths) + { + try + { + object_storage->removeObjectIfExists(StoredObject(path)); + LOG_DEBUG(log, "Deleted orphan file {}", path); + result.deleted_paths.push_back(path); + } + catch (...) + { + ++result.failed_count; + LOG_WARNING(log, "Failed to delete orphan file {}: {}", path, getCurrentExceptionMessage(false)); + } + } + + return result; +} + +// --------------------------------------------------------------------------- +// Result aggregation +// --------------------------------------------------------------------------- + +RemoveOrphanFilesResult tallyByCategory(const std::vector & paths, Int64 skipped_missing_metadata) +{ + RemoveOrphanFilesResult r; + for (const auto & path : paths) + { + switch (inspectFileCategory(path)) + { + case FileCategory::DATA_FILE: ++r.deleted_data_files_count; break; + case FileCategory::POSITION_DELETE_FILE: ++r.deleted_position_delete_files_count; break; + case FileCategory::EQUALITY_DELETE_FILE: ++r.deleted_equality_delete_files_count; break; + case FileCategory::MANIFEST_FILE: ++r.deleted_manifest_files_count; break; + case FileCategory::MANIFEST_LIST: ++r.deleted_manifest_lists_count; break; + case FileCategory::METADATA_JSON: ++r.deleted_metadata_files_count; break; + case FileCategory::STATISTICS_FILE: ++r.deleted_statistics_files_count; break; + } + } + r.skipped_missing_metadata_count = skipped_missing_metadata; + return r; +} + +// --------------------------------------------------------------------------- +// Result → Pipe +// --------------------------------------------------------------------------- + +Pipe resultToPipe(const RemoveOrphanFilesResult & result) +{ + Block header{ + ColumnWithTypeAndName(std::make_shared(), "metric_name"), + ColumnWithTypeAndName(std::make_shared(), "metric_value"), + }; + + MutableColumns columns = header.cloneEmptyColumns(); + + auto add = [&](const char * name, Int64 value) + { + columns[0]->insert(String(name)); + columns[1]->insert(value); + }; + + add("deleted_data_files_count", result.deleted_data_files_count); + add("deleted_position_delete_files_count", result.deleted_position_delete_files_count); + add("deleted_equality_delete_files_count", result.deleted_equality_delete_files_count); + add("deleted_manifest_files_count", result.deleted_manifest_files_count); + add("deleted_manifest_lists_count", result.deleted_manifest_lists_count); + add("deleted_metadata_files_count", result.deleted_metadata_files_count); + add("deleted_statistics_files_count", result.deleted_statistics_files_count); + add("skipped_missing_metadata_count", result.skipped_missing_metadata_count); + add("failed_deletions_count", result.failed_deletions_count); + + const size_t rows = columns[0]->size(); + Chunk chunk(std::move(columns), rows); + return Pipe(std::make_shared(std::make_shared(std::move(header)), std::move(chunk))); +} + +// --------------------------------------------------------------------------- +// Core orchestration +// --------------------------------------------------------------------------- + +RemoveOrphanFilesResult removeOrphanFiles( + const RemoveOrphanFilesParams & params, + ContextPtr context, + ObjectStoragePtr object_storage, + const DataLakeStorageSettings & data_lake_settings, + const PersistentTableComponents & persistent_table_components) +{ + auto log = getLogger("IcebergRemoveOrphanFiles"); + + auto [reachable, metadata_version] = collectReachableFiles( + object_storage, persistent_table_components, data_lake_settings, context, log); + + String scan_path = resolveScanPath(persistent_table_components.table_path, params); + if (!object_storage->existsOrHasAnyChild(scan_path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Scan path '{}' does not exist", scan_path); + RelativePathsWithMetadata actual_files; + object_storage->listObjects(scan_path, actual_files, /* max_keys = */ 0); + LOG_INFO(log, "Found {} actual files under scan path '{}'", actual_files.size(), scan_path); + + chassert(params.older_than.has_value()); + auto scan = findOrphanFiles(actual_files, reachable, *params.older_than, object_storage, log); + LOG_INFO(log, "Found {} orphan files (dry_run={})", scan.orphan_paths.size(), params.dry_run); + + if (params.dry_run || scan.orphan_paths.empty()) + return tallyByCategory(scan.orphan_paths, scan.skipped_missing_metadata); + + auto [_recheck_files, recheck_version] = collectReachableFiles( + object_storage, persistent_table_components, data_lake_settings, context, log); + if (recheck_version != metadata_version) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Metadata version changed during orphan scan (v{} -> v{}); " + "aborting to avoid deleting files referenced by a concurrent commit", + metadata_version, recheck_version); + + auto delete_result = deleteOrphanFiles(scan.orphan_paths, object_storage, log); + LOG_INFO(log, "Deleted {}/{} orphan files ({} failed)", + delete_result.deleted_paths.size(), scan.orphan_paths.size(), delete_result.failed_count); + + auto result = tallyByCategory(delete_result.deleted_paths, scan.skipped_missing_metadata); + result.failed_deletions_count = delete_result.failed_count; + return result; +} + +} + + +// --------------------------------------------------------------------------- +// Public entry point +// --------------------------------------------------------------------------- + +Pipe executeRemoveOrphanFiles( + const ASTPtr & args, + ContextPtr context, + ObjectStoragePtr object_storage, + const DataLakeStorageSettings & data_lake_settings, + const PersistentTableComponents & persistent_components) +{ + if (persistent_components.format_version < 2) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "remove_orphan_files requires Iceberg format version >= 2, " + "but this table uses format version {}", + persistent_components.format_version); + + auto parsed = makeSchema().parse(args); + + RemoveOrphanFilesParams params; + if (parsed.has("older_than")) + { + String older_than_str = parsed.getAs("older_than"); + ReadBufferFromString buf(older_than_str); + time_t ts; + readDateTimeText(ts, buf); + + auto now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + if (ts > now) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "older_than must not be in the future; a future value would bypass the in-progress-write safety window"); + + params.older_than = ts; + } + else + { + UInt64 threshold_seconds = context->getSettingsRef()[Setting::iceberg_orphan_files_older_than_seconds].value; + auto now = std::chrono::system_clock::now(); + params.older_than = std::chrono::system_clock::to_time_t(now - std::chrono::seconds(threshold_seconds)); + } + if (parsed.has("location")) + params.location = parsed.getAs("location"); + params.dry_run = parsed.getAs("dry_run") != 0; + + auto result = removeOrphanFiles(params, context, object_storage, data_lake_settings, persistent_components); + + return resultToPipe(result); +} + +} +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/RemoveOrphanFilesExecute.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/RemoveOrphanFilesExecute.h new file mode 100644 index 000000000000..1809d3880bad --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/RemoveOrphanFilesExecute.h @@ -0,0 +1,24 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO + +#include +#include +#include +#include + +namespace DB::Iceberg +{ + +Pipe executeRemoveOrphanFiles( + const ASTPtr & args, + ContextPtr context, + ObjectStoragePtr object_storage, + const DataLakeStorageSettings & data_lake_settings, + const PersistentTableComponents & persistent_components); + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h index 47fc360ad13d..7ace7a71b364 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h @@ -36,7 +36,7 @@ struct IcebergHistoryRecord DB::DateTime64 made_current_at; Int64 parent_id; bool is_current_ancestor; - String manifest_list_path; + Iceberg::IcebergPathFromMetadata manifest_list_path; Int32 added_files = 0; Int32 added_records = 0; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/SnapshotFilesTraversal.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/SnapshotFilesTraversal.cpp new file mode 100644 index 000000000000..572bf495340e --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/SnapshotFilesTraversal.cpp @@ -0,0 +1,190 @@ +#include "config.h" + +#if USE_AVRO + +#include + +#include + +#include + +#include +#include +#include +#include + +namespace DB::Iceberg +{ + +SnapshotReferencedFiles collectSnapshotReferencedFiles( + const Poco::JSON::Array::Ptr & snapshots, + ObjectStoragePtr object_storage, + const PersistentTableComponents & persistent_table_components, + ContextPtr context, + LoggerPtr log, + Int32 current_schema_id) +{ + SnapshotReferencedFiles files; + + for (UInt32 i = 0; i < snapshots->size(); ++i) + { + auto snapshot = snapshots->getObject(i); + if (!snapshot->has(Iceberg::f_manifest_list)) + continue; + + auto manifest_list_path = IcebergPathFromMetadata::deserialize(snapshot->getValue(Iceberg::f_manifest_list)); + files.manifest_list_paths.insert(manifest_list_path); + + auto manifest_keys = getManifestList( + object_storage, persistent_table_components, context, manifest_list_path, log); + + for (const auto & manifest_entry : manifest_keys) + { + files.manifest_paths.insert(manifest_entry.manifest_file_path); + + auto entries_handle = getManifestFileEntriesHandle( + object_storage, persistent_table_components, context, log, manifest_entry, current_schema_id); + + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::DATA)) + files.data_file_paths.insert(entry->parsed_entry->file_path_key); + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::POSITION_DELETE)) + files.data_file_paths.insert(entry->parsed_entry->file_path_key); + for (const auto & entry : entries_handle.getFilesWithoutDeleted(FileContentType::EQUALITY_DELETE)) + files.data_file_paths.insert(entry->parsed_entry->file_path_key); + } + } + + return files; +} + +namespace +{ + +void collectStatisticsPaths( + const Poco::JSON::Object::Ptr & metadata, + const char * field_name, + const IcebergPathResolver & resolver, + std::unordered_set & out) +{ + if (!metadata->has(field_name)) + return; + auto arr = metadata->get(field_name).extract(); + if (!arr) + return; + for (UInt32 j = 0; j < arr->size(); ++j) + { + auto entry = arr->getObject(j); + if (entry->has(f_statistics_path)) + { + String stat_path = entry->getValue(f_statistics_path); + out.insert(resolver.resolve(IcebergPathFromMetadata::deserialize(stat_path))); + } + } +} + +/// Collect files reachable directly from the metadata JSON root: +/// the current metadata file, historical metadata files from metadata-log, +/// statistics, partition-statistics, and version-hint files. +void collectMetadataRootFiles( + const String & metadata_path, + const Poco::JSON::Object::Ptr & metadata, + const IcebergPathResolver & resolver, + std::unordered_set & out) +{ + out.insert(metadata_path); + + auto version_hint = IcebergPathFromMetadata::deserialize(fmt::format("{}metadata/version-hint.text", resolver.getTableLocation())); + out.insert(resolver.resolve(version_hint)); + + if (metadata->has(f_metadata_log)) + { + auto metadata_log = metadata->get(f_metadata_log).extract(); + if (metadata_log) + { + for (UInt32 i = 0; i < metadata_log->size(); ++i) + { + auto entry = metadata_log->getObject(i); + if (entry->has(f_metadata_file)) + { + String mf_path = entry->getValue(f_metadata_file); + out.insert(resolver.resolve(IcebergPathFromMetadata::deserialize(mf_path))); + } + } + } + } + + collectStatisticsPaths(metadata, f_statistics, resolver, out); + collectStatisticsPaths(metadata, f_partition_statistics, resolver, out); +} + +} + + +ReachableFilesResult collectReachableFiles( + ObjectStoragePtr object_storage, + const PersistentTableComponents & persistent_table_components, + const DataLakeStorageSettings & data_lake_settings, + ContextPtr context, + LoggerPtr log) +{ + auto [version, metadata_path, compression_method] = getLatestOrExplicitMetadataFileAndVersion( + object_storage, + persistent_table_components.table_path, + data_lake_settings, + persistent_table_components.metadata_cache, + context, + log.get(), + persistent_table_components.table_uuid, + persistent_table_components.metadata_compression_method, + /* force_fetch_latest_metadata */ true, + /* ignore_explicit_metadata_file_path */ true); + + auto metadata = getMetadataJSONObject( + metadata_path, + object_storage, + persistent_table_components.metadata_cache, + context, + log, + compression_method, + persistent_table_components.table_uuid); + + std::unordered_set reachable; + const auto & resolver = persistent_table_components.path_resolver; + + collectMetadataRootFiles( + metadata_path, metadata, + resolver, + reachable); + + if (!metadata->has(f_snapshots)) + { + LOG_INFO(log, "No snapshots in metadata, reachable set contains only metadata-root files"); + return {std::move(reachable), version}; + } + + auto snapshots = metadata->get(f_snapshots).extract(); + if (!snapshots || snapshots->size() == 0) + { + LOG_INFO(log, "Empty snapshots array, reachable set contains only metadata-root files"); + return {std::move(reachable), version}; + } + + Int32 current_schema_id = metadata->getValue(f_current_schema_id); + + auto snapshot_files = collectSnapshotReferencedFiles( + snapshots, object_storage, persistent_table_components, context, log, current_schema_id); + + for (const auto & path : snapshot_files.manifest_list_paths) + reachable.insert(resolver.resolve(path)); + for (const auto & path : snapshot_files.manifest_paths) + reachable.insert(resolver.resolve(path)); + for (const auto & path : snapshot_files.data_file_paths) + reachable.insert(resolver.resolve(path)); + + LOG_INFO(log, "Collected {} reachable files from metadata graph", reachable.size()); + return {std::move(reachable), version}; +} + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/SnapshotFilesTraversal.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/SnapshotFilesTraversal.h new file mode 100644 index 000000000000..a9961d8e469b --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/SnapshotFilesTraversal.h @@ -0,0 +1,61 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO + +#include + +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB::Iceberg +{ + +/// Paths collected from Iceberg snapshot metadata, stored as opaque +/// IcebergPathFromMetadata values that must be resolved through +/// IcebergPathResolver before I/O operations. +struct SnapshotReferencedFiles +{ + std::unordered_set manifest_list_paths; + std::unordered_set manifest_paths; + std::unordered_set data_file_paths; +}; + +SnapshotReferencedFiles collectSnapshotReferencedFiles( + const Poco::JSON::Array::Ptr & snapshots, + ObjectStoragePtr object_storage, + const PersistentTableComponents & persistent_table_components, + ContextPtr context, + LoggerPtr log, + Int32 current_schema_id); + +struct ReachableFilesResult +{ + std::unordered_set files; + Int32 metadata_version; +}; + +/// Collect all files reachable through the metadata graph. +/// +/// Traverses: metadata JSON files (from metadata-log), manifest lists (from snapshots), +/// manifest files (from manifest lists), data/delete files (from manifest files), +/// and statistics files. All returned paths are resolved storage paths. +/// Also returns the metadata version used, for TOCTOU detection. +ReachableFilesResult collectReachableFiles( + ObjectStoragePtr object_storage, + const PersistentTableComponents & persistent_table_components, + const DataLakeStorageSettings & data_lake_settings, + ContextPtr context, + LoggerPtr log); + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/StatelessMetadataFileGetter.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/StatelessMetadataFileGetter.cpp index c55e215cc95d..b44c3d86c9e9 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/StatelessMetadataFileGetter.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/StatelessMetadataFileGetter.cpp @@ -69,7 +69,7 @@ Iceberg::ManifestFileCacheableInfo getManifestFile( const PersistentTableComponents & persistent_table_components, ContextPtr local_context, LoggerPtr log, - const String & filename, + const IcebergPathFromMetadata & filename, size_t bytes_size) { auto log_level = local_context->getSettingsRef()[Setting::iceberg_metadata_log_level].value; @@ -79,7 +79,7 @@ Iceberg::ManifestFileCacheableInfo getManifestFile( auto create_fn = [&, use_iceberg_metadata_cache]() { - RelativePathWithMetadata manifest_object_info(filename); + RelativePathWithMetadata manifest_object_info(persistent_table_components.path_resolver.resolve(filename)); auto read_settings = local_context->getReadSettings(); /// Do not utilize filesystem cache if more precise cache enabled @@ -96,7 +96,7 @@ Iceberg::ManifestFileCacheableInfo getManifestFile( if (use_iceberg_metadata_cache && persistent_table_components.table_uuid.has_value()) { auto manifest_file = persistent_table_components.metadata_cache->getOrSetManifestFile( - IcebergMetadataFilesCache::getKey(persistent_table_components.table_uuid.value(), filename), create_fn); + IcebergMetadataFilesCache::getKey(persistent_table_components.table_uuid.value(), filename.serialize()), create_fn); return manifest_file; } return create_fn(); @@ -122,13 +122,11 @@ Iceberg::ManifestFileIterator::ManifestFileEntriesHandle getManifestFileEntriesH cacheable_info.deserializer, cache_key.manifest_file_path, persistent_table_components.format_version, - persistent_table_components.table_path, + persistent_table_components.path_resolver, *persistent_table_components.schema_processor, cache_key.added_sequence_number, cache_key.added_snapshot_id, - persistent_table_components.table_location, local_context, - cache_key.manifest_file_path, nullptr, table_snapshot_schema_id); @@ -143,7 +141,7 @@ ManifestFileCacheKeys getManifestList( ObjectStoragePtr object_storage, const PersistentTableComponents & persistent_table_components, ContextPtr local_context, - const String & filename, + const IcebergPathFromMetadata & filename, LoggerPtr log) { IcebergMetadataLogLevel log_level = local_context->getSettingsRef()[Setting::iceberg_metadata_log_level].value; @@ -153,7 +151,7 @@ ManifestFileCacheKeys getManifestList( auto create_fn = [&, use_iceberg_metadata_cache]() { - RelativePathWithMetadata object_info(filename); + RelativePathWithMetadata object_info(persistent_table_components.path_resolver.resolve(filename)); auto read_settings = local_context->getReadSettings(); /// Do not utilize filesystem cache if more precise cache enabled @@ -170,17 +168,15 @@ ManifestFileCacheKeys getManifestList( local_context, dump_metadata, DB::IcebergMetadataLogLevel::ManifestListMetadata, - persistent_table_components.table_path, + persistent_table_components.path_resolver.getTableRoot(), filename, std::nullopt, std::nullopt); for (size_t i = 0; i < manifest_list_deserializer.rows(); ++i) { - const std::string file_path - = manifest_list_deserializer.getValueFromRowByName(i, f_manifest_path, TypeIndex::String).safeGet(); - const auto manifest_file_name = getProperFilePathFromMetadataInfo( - file_path, persistent_table_components.table_path, persistent_table_components.table_location); + const IcebergPathFromMetadata manifest_file_name = IcebergPathFromMetadata::deserialize( + manifest_list_deserializer.getValueFromRowByName(i, f_manifest_path, TypeIndex::String).safeGet()); Int64 added_sequence_number = 0; auto added_snapshot_id = manifest_list_deserializer.getValueFromRowByName(i, f_added_snapshot_id); if (added_snapshot_id.isNull()) @@ -216,7 +212,7 @@ ManifestFileCacheKeys getManifestList( local_context, dump_row_metadata, DB::IcebergMetadataLogLevel::ManifestListEntry, - persistent_table_components.table_path, + persistent_table_components.path_resolver.getTableRoot(), filename, i, std::nullopt); @@ -230,7 +226,7 @@ ManifestFileCacheKeys getManifestList( ManifestFileCacheKeys manifest_file_cache_keys; if (use_iceberg_metadata_cache && persistent_table_components.table_uuid.has_value()) manifest_file_cache_keys = persistent_table_components.metadata_cache->getOrSetManifestFileCacheKeys( - IcebergMetadataFilesCache::getKey(persistent_table_components.table_uuid.value(), filename), create_fn); + IcebergMetadataFilesCache::getKey(persistent_table_components.table_uuid.value(), filename.serialize()), create_fn); else manifest_file_cache_keys = create_fn(); return manifest_file_cache_keys; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/StatelessMetadataFileGetter.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/StatelessMetadataFileGetter.h index ac5a7f7bc06e..4949de8d33e1 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/StatelessMetadataFileGetter.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/StatelessMetadataFileGetter.h @@ -28,7 +28,7 @@ Iceberg::ManifestFileCacheableInfo getManifestFile( const PersistentTableComponents & persistent_table_components, ContextPtr local_context, LoggerPtr log, - const String & filename, + const IcebergPathFromMetadata & filename, size_t bytes_size); /// Creates a fully initialized ManifestFileIterator from a cache key. @@ -47,7 +47,7 @@ ManifestFileCacheKeys getManifestList( ObjectStoragePtr object_storage, const PersistentTableComponents & persistent_table_components, ContextPtr local_context, - const String & filename, + const IcebergPathFromMetadata & filename, LoggerPtr log); } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp index 0ed4e7d78f7a..e4822d713300 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -79,6 +80,11 @@ namespace DB::DataLakeStorageSetting extern const DataLakeStorageSettingsNonZeroUInt64 iceberg_format_version; } +namespace DB::Setting +{ +extern const SettingsString iceberg_metadata_compression_method; +} + namespace ProfileEvents { extern const Event IcebergVersionHintUsed; @@ -101,6 +107,36 @@ namespace DB::Iceberg { using namespace DB; + +/// Best-effort heuristic based on ClickHouse naming conventions. +/// Files produced by other engines (Spark, Flink, Trino) may use different +/// patterns and fall through to DATA_FILE; this only affects per-category +/// reporting metrics, not deletion safety. +FileCategory inspectFileCategory(const String & relative_path) +{ + if (relative_path.find("/metadata/") != String::npos || relative_path.starts_with("metadata/")) + { + if (relative_path.find(".metadata.json") != String::npos) + return FileCategory::METADATA_JSON; + if (relative_path.ends_with(".avro")) + { + if (relative_path.find("snap-") != String::npos) + return FileCategory::MANIFEST_LIST; + return FileCategory::MANIFEST_FILE; + } + if (relative_path.ends_with(".puffin") || relative_path.ends_with(".stats")) + return FileCategory::STATISTICS_FILE; + } + + if (relative_path.find("eq-del") != String::npos) + return FileCategory::EQUALITY_DELETE_FILE; + + if (relative_path.find("-deletes.parquet") != String::npos || relative_path.find("-delete-") != String::npos) + return FileCategory::POSITION_DELETE_FILE; + + return FileCategory::DATA_FILE; +} + static CompressionMethod getCompressionMethodFromMetadataFile(const String & path) { constexpr std::string_view metadata_suffix = ".metadata.json"; @@ -151,9 +187,53 @@ Iceberg::MetadataFileWithInfo getMetadataFileAndVersion(const std::string & path ErrorCodes::BAD_ARGUMENTS, "Bad metadata file name: '{}'. Expected vN.metadata.json where N is a number", file_name); return MetadataFileWithInfo{ - .version = std::stoi(version_str), - .path = path, - .compression_method = getCompressionMethodFromMetadataFile(path)}; + .version = std::stoi(version_str), .path = path, .compression_method = getCompressionMethodFromMetadataFile(path)}; +} + +/// Resolve metadata filename from version hint content. +/// Version hint may contain just a version number (e.g. "1") or a full filename (e.g. "v1.metadata.json"). +/// When only a version number is given, we try to find the actual file, which may have a compression suffix. +static std::optional resolveMetadataFilenameFromVersionHint( + const String & version_hint_content, + const String & table_path, + const ObjectStoragePtr & object_storage, + CompressionMethod known_compression_method, + const ContextPtr & local_context) +{ + String metadata_file = version_hint_content; + if (metadata_file.ends_with(".metadata.json")) + return metadata_file; + + if (!std::all_of(metadata_file.begin(), metadata_file.end(), isdigit)) + return metadata_file + ".metadata.json"; + + /// Version hint is a number. Try to find the actual file. + String version_number = metadata_file; + + /// First try without compression. + String candidate = "v" + version_number + ".metadata.json"; + auto candidate_path = std::filesystem::path(table_path) / "metadata" / candidate; + if (object_storage->exists(StoredObject(candidate_path))) + return candidate; + + /// Try with known compression method. + auto compression_method = known_compression_method; + if (compression_method == CompressionMethod::None) + { + auto compression_method_str = local_context->getSettingsRef()[Setting::iceberg_metadata_compression_method].value; + compression_method = chooseCompressionMethod(compression_method_str, compression_method_str); + } + if (compression_method != CompressionMethod::None) + { + auto suffix = toContentEncodingName(compression_method); + String compressed_candidate = "v" + version_number + "." + suffix + ".metadata.json"; + auto compressed_path = std::filesystem::path(table_path) / "metadata" / compressed_candidate; + if (object_storage->exists(StoredObject(compressed_path))) + return compressed_candidate; + } + + /// Nothing found via direct checks. + return std::nullopt; } @@ -186,21 +266,29 @@ void writeMessageToFile( } bool writeMetadataFileAndVersionHint( - const std::string & metadata_file_path, + const IcebergPathResolver & resolver, + const GeneratedMetadataFileWithInfo & metadata_file_info, const std::string & metadata_file_content, - const std::string & version_hint_path, - std::string version_hint_content, + const IcebergPathFromMetadata & version_hint_path, DB::ObjectStoragePtr object_storage, DB::ContextPtr context, - DB::CompressionMethod compression_method, bool try_write_version_hint) { + auto storage_metadata_path = resolver.resolve(metadata_file_info.path); + auto storage_version_hint_path = resolver.resolve(version_hint_path); try { - if (object_storage->exists(StoredObject(metadata_file_path))) + if (object_storage->exists(StoredObject(storage_metadata_path))) return false; - Iceberg::writeMessageToFile(metadata_file_content, metadata_file_path, object_storage, context, /* write-if-none-match */ "*", "", compression_method); + Iceberg::writeMessageToFile( + metadata_file_content, + storage_metadata_path, + object_storage, + context, + /* write-if-none-match */ "*", + "", + metadata_file_info.compression_method); } catch (...) { @@ -210,13 +298,10 @@ bool writeMetadataFileAndVersionHint( if (try_write_version_hint) { - if (version_hint_content.starts_with('/')) - version_hint_content = version_hint_content.substr(1); - size_t i = 0; while (i < MAX_TRANSACTION_RETRIES) { - StoredObject object_info(version_hint_path); + StoredObject object_info(storage_version_hint_path); std::string version_hint_value; std::string etag; std::string write_if_none_match = "*"; @@ -224,17 +309,35 @@ bool writeMetadataFileAndVersionHint( { auto [object_data, object_metadata] = object_storage->readSmallObjectAndGetObjectMetadata(object_info, context->getReadSettings(), MAX_HINT_FILE_SIZE); version_hint_value = object_data; + boost::algorithm::trim(version_hint_value); etag = object_metadata.etag; write_if_none_match.clear(); } - auto [old_version, _1, _2] = getMetadataFileAndVersion(version_hint_value); - auto [new_version, _3, _4] = getMetadataFileAndVersion(version_hint_content); - if (old_version < new_version) + Int32 old_version = 0; + if (!version_hint_value.empty()) + { + if (std::all_of(version_hint_value.begin(), version_hint_value.end(), isdigit)) + { + old_version = std::stoi(version_hint_value); + } + else + { + old_version = getMetadataFileAndVersion(version_hint_value).version; + } + } + if (old_version < metadata_file_info.version) { try { - Iceberg::writeMessageToFile(version_hint_content, version_hint_path, object_storage, context, write_if_none_match, /* write-if-match */ etag); + /// Write just the version number for Spark/spec compatibility. + Iceberg::writeMessageToFile( + std::to_string(metadata_file_info.version), + storage_version_hint_path, + object_storage, + context, + write_if_none_match, + /* write-if-match */ etag); break; } catch (...) @@ -312,78 +415,6 @@ std::optional parseTransformAndArgument(const String & tra return std::nullopt; } -// This function is used to get the file path inside the directory which corresponds to iceberg table from the full blob path which is written in manifest and metadata files. -// For example, if the full blob path is s3://bucket/table_name/data/00000-1-1234567890.avro, the function will return table_name/data/00000-1-1234567890.avro -// Common path should end with "" or "/". -std::string getProperFilePathFromMetadataInfo(std::string_view data_path, std::string_view common_path, std::string_view table_location) -{ - auto trim_backward_slash = [](std::string_view str) -> std::string_view - { - if (str.ends_with('/')) - { - return str.substr(0, str.size() - 1); - } - return str; - }; - auto trim_forward_slash = [](std::string_view str) -> std::string_view - { - if (str.starts_with('/')) - { - return str.substr(1); - } - return str; - }; - common_path = trim_backward_slash(common_path); - table_location = trim_backward_slash(table_location); - - if (data_path.starts_with(table_location) && table_location.ends_with(common_path)) - { - return std::filesystem::path{common_path} / trim_forward_slash(data_path.substr(table_location.size())); - } - - - auto pos = data_path.find(common_path); - /// Valid situation when data and metadata files are stored in different directories. - if (pos == std::string::npos) - { - /// connection://bucket - auto prefix = table_location.substr(0, table_location.size() - common_path.size()); - return std::string{data_path.substr(prefix.size())}; - } - - size_t good_pos = std::string::npos; - while (pos != std::string::npos) - { - auto potential_position = pos + common_path.size(); - if ((std::string_view(data_path.data() + potential_position, 6) == "/data/") - || (std::string_view(data_path.data() + potential_position, 10) == "/metadata/")) - { - good_pos = pos; - break; - } - size_t new_pos = data_path.find(common_path, pos + 1); - if (new_pos == std::string::npos) - { - break; - } - pos = new_pos; - } - - - if (good_pos != std::string::npos) - { - return std::string{data_path.substr(good_pos)}; - } - else if (pos != std::string::npos) - { - return std::string{data_path.substr(pos)}; - } - else - { - throw ::DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Expected to find '{}' in data path: '{}'", common_path, data_path); - } -} - enum class MostRecentMetadataFileSelectionWay { BY_LAST_UPDATED_MS_FIELD, @@ -1156,6 +1187,7 @@ MetadataFileWithInfo getLatestOrExplicitMetadataFileAndVersion( const ContextPtr & local_context, Poco::Logger * log, const std::optional & table_uuid, + CompressionMethod known_compression_method, bool force_fetch_latest_metadata, bool ignore_explicit_metadata_file_path) { @@ -1204,19 +1236,17 @@ MetadataFileWithInfo getLatestOrExplicitMetadataFileAndVersion( StoredObject version_hint(version_hint_path); auto buf = object_storage->readObject(version_hint, ReadSettings{}); readString(metadata_file, *buf); - if (!metadata_file.ends_with(".metadata.json")) + auto resolved + = resolveMetadataFilenameFromVersionHint(metadata_file, table_path, object_storage, known_compression_method, local_context); + if (resolved.has_value()) { - if (std::all_of(metadata_file.begin(), metadata_file.end(), isdigit)) - metadata_file = "v" + metadata_file + ".metadata.json"; - else - metadata_file = metadata_file + ".metadata.json"; + LOG_TEST(log, "Version hint file points to {}, will read from this metadata file", *resolved); + ProfileEvents::increment(ProfileEvents::IcebergVersionHintUsed); + return getMetadataFileAndVersion(std::filesystem::path(table_path) / "metadata" / fs::path(*resolved).filename()); } - LOG_TEST(log, "Version hint file points to {}, will read from this metadata file", metadata_file); - ProfileEvents::increment(ProfileEvents::IcebergVersionHintUsed); - - return getMetadataFileAndVersion(std::filesystem::path(table_path) / "metadata" / fs::path(metadata_file).filename()); + LOG_WARNING(log, "Version hint content '{}' could not be resolved to a metadata file, falling back to listing", metadata_file); } - else + { return getLatestMetadataFileAndVersion( object_storage, table_path, data_lake_settings, metadata_cache, local_context, table_uuid, false, force_fetch_latest_metadata); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h index 7695ca7edeff..af6429ce2e41 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -42,17 +43,13 @@ void writeMessageToFile( /// Maybe return false if failed to write metadata.json /// Will try to write hint multiple times, but will not report failure to write hint. bool writeMetadataFileAndVersionHint( - const std::string & metadata_file_path, + const IcebergPathResolver & resolver, + const DB::GeneratedMetadataFileWithInfo & metadata_file_info, const std::string & metadata_file_content, - const std::string & version_hint_path, - std::string version_hint_content, + const IcebergPathFromMetadata & version_hint_path, DB::ObjectStoragePtr object_storage, DB::ContextPtr context, - DB::CompressionMethod compression_method, - bool try_write_version_hint -); - -std::string getProperFilePathFromMetadataInfo(std::string_view data_path, std::string_view common_path, std::string_view table_location); + bool try_write_version_hint); struct TransformAndArgument { @@ -105,6 +102,7 @@ MetadataFileWithInfo getLatestOrExplicitMetadataFileAndVersion( const ContextPtr & local_context, Poco::Logger * log, const std::optional & table_uuid, + CompressionMethod known_compression_method, bool force_fetch_latest_metadata = true, bool ignore_explicit_metadata_file_path = false); @@ -114,6 +112,19 @@ std::string normalizeUuid(const std::string & uuid); DataTypePtr getFunctionResultType(const String & iceberg_transform_name, DataTypePtr source_type); +enum class FileCategory : uint8_t +{ + DATA_FILE, + POSITION_DELETE_FILE, + EQUALITY_DELETE_FILE, + MANIFEST_FILE, + MANIFEST_LIST, + METADATA_JSON, + STATISTICS_FILE, +}; + +FileCategory inspectFileCategory(const String & relative_path); + KeyDescription getSortingKeyDescriptionFromMetadata( Poco::JSON::Object::Ptr metadata_object, const NamesAndTypesList & ch_schema, ContextPtr local_context); /// Returns Iceberg/Spark-style display string for sort order, e.g. "id desc, hour(ts) asc". diff --git a/src/Storages/ObjectStorage/DataLakes/Paimon/PaimonClient.cpp b/src/Storages/ObjectStorage/DataLakes/Paimon/PaimonClient.cpp index 78d7fb7f62a1..464ba68e1d64 100644 --- a/src/Storages/ObjectStorage/DataLakes/Paimon/PaimonClient.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Paimon/PaimonClient.cpp @@ -254,7 +254,7 @@ std::vector PaimonTableClient::getManifestMeta(String ma RelativePathWithMetadata relative_path(std::filesystem::path(table_location) / (PAIMON_MANIFEST_DIR) / manifest_list_path); auto manifest_list_buf = createReadBuffer(relative_path, object_storage, context, log); Iceberg::AvroForIcebergDeserializer manifest_list_deserializer( - std::move(manifest_list_buf), manifest_list_path, getFormatSettings(getContext())); + std::move(manifest_list_buf), Iceberg::IcebergPathFromMetadata::deserialize(manifest_list_path), getFormatSettings(getContext())); std::vector paimon_manifest_file_meta_vec; paimon_manifest_file_meta_vec.reserve(manifest_list_deserializer.rows()); @@ -276,7 +276,7 @@ PaimonTableClient::getDataManifest(String manifest_path, const PaimonTableSchema auto context = getContext(); RelativePathWithMetadata object_info(std::filesystem::path(table_location) / (PAIMON_MANIFEST_DIR) / manifest_path); auto manifest_buf = createReadBuffer(object_info, object_storage, context, log); - Iceberg::AvroForIcebergDeserializer manifest_deserializer(std::move(manifest_buf), manifest_path, getFormatSettings(getContext())); + Iceberg::AvroForIcebergDeserializer manifest_deserializer(std::move(manifest_buf), Iceberg::IcebergPathFromMetadata::deserialize(manifest_path), getFormatSettings(getContext())); PaimonManifest paimon_manifest; paimon_manifest.entries.reserve(manifest_deserializer.rows()); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 07ef358d0396..6c334453f4cd 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -41,6 +41,7 @@ #include #include #include +#include #if ENABLE_DISTRIBUTED_CACHE #include #include @@ -424,6 +425,12 @@ Chunk StorageObjectStorageSource::generate() path); } + const String * iceberg_metadata_file_path = nullptr; +#if USE_AVRO + if (const auto * iceberg_info = dynamic_cast(object_info.get())) + iceberg_metadata_file_path = &iceberg_info->info.data_object_file_path_key.serialize(); +#endif + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, @@ -435,6 +442,7 @@ Chunk StorageObjectStorageSource::generate() .etag = &(object_metadata->etag), .tags = &(object_metadata->tags), .data_lake_snapshot_version = file_iterator->getSnapshotVersion(), + .iceberg_metadata_file_path = iceberg_metadata_file_path, }, read_context); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index cce131a5b959..9ff0d939b3dc 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -147,6 +147,7 @@ static NamesAndTypesList getCommonVirtualsForFileLikeStorage() {"_tags", std::make_shared(std::make_shared(), std::make_shared())}, {"_data_lake_snapshot_version", makeNullable(std::make_shared())}, {"_row_number", makeNullable(std::make_shared())}, + {"_iceberg_metadata_file_path", std::make_shared(std::make_shared())}, }; } @@ -399,6 +400,7 @@ void addRequestedFileLikeStorageVirtualsToChunk( } else if (virtual_column.name == "_row_number") { +#if USE_PARQUET auto chunk_info = chunk.getChunkInfos().get(); if (chunk_info) { @@ -413,8 +415,22 @@ void addRequestedFileLikeStorageVirtualsToChunk( chunk.addColumn(ColumnNullable::create(std::move(column), std::move(null_map))); return; } - /// Row numbers not known, _row_number = NULL. + else + { + /// Row numbers not known, _row_number = NULL. + chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); + } +#else + // If Parquet format is not used, we don't have row numbers info, so _row_number = NULL. chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); +#endif + } + else if (virtual_column.name == "_iceberg_metadata_file_path") + { + if (virtual_values.iceberg_metadata_file_path) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *virtual_values.iceberg_metadata_file_path)->convertToFullColumnIfConst()); + else + chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } else if (auto it = hive_map.find(virtual_column.getNameInStorage()); it != hive_map.end()) { diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index bcd3b9974b1d..8b40a4e5fd58 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -135,6 +135,9 @@ struct VirtualsForFileLikeStorage const String * etag { nullptr }; const std::map * tags { nullptr }; std::optional data_lake_snapshot_version { std::nullopt }; + /// Original file path as stored in Iceberg metadata (before resolution to storage path). + /// Used by Iceberg position deletes to reference data files in the metadata path format. + const String * iceberg_metadata_file_path { nullptr }; }; void addRequestedFileLikeStorageVirtualsToChunk( diff --git a/tests/integration/test_storage_iceberg_concurrent/configs/config.d/query_log.xml b/tests/integration/test_storage_iceberg_concurrent/configs/config.d/query_log.xml deleted file mode 100644 index a63e91f41fbc..000000000000 --- a/tests/integration/test_storage_iceberg_concurrent/configs/config.d/query_log.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - system - query_log
-
-
diff --git a/tests/integration/test_storage_iceberg_concurrent/conftest.py b/tests/integration/test_storage_iceberg_concurrent/conftest.py index 2e61a6d4170e..4f2e78eb76ee 100644 --- a/tests/integration/test_storage_iceberg_concurrent/conftest.py +++ b/tests/integration/test_storage_iceberg_concurrent/conftest.py @@ -55,12 +55,6 @@ def get_spark(cluster : ClickHouseCluster): builder = ( pyspark.sql.SparkSession.builder \ .appName("IcebergS3Example") \ - .config("spark.jars.repositories", "https://repo1.maven.org/maven2") \ - .config("spark.jars.packages", - f'org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:{iceberg_version},' - f'org.apache.spark:spark-avro_2.12:{spark_version},' - f'org.apache.hadoop:hadoop-aws:{hadoop_aws_version},' - f'com.amazonaws:aws-java-sdk-bundle:{jdk_bundle}')\ .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") \ .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") \ .config("spark.sql.catalog.spark_catalog.type", "hadoop") \ @@ -88,8 +82,6 @@ def started_cluster_iceberg(): cluster.add_instance( "node1", main_configs=[ - "configs/config.d/query_log.xml", - "configs/config.d/cluster.xml", "configs/config.d/named_collections.xml", ], user_configs=["configs/users.d/users.xml"], diff --git a/tests/integration/test_storage_iceberg_interoperability_azure/__init__.py b/tests/integration/test_storage_iceberg_interoperability_azure/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_storage_iceberg_interoperability_azure/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg_interoperability_azure/configs/config.d/named_collections.xml new file mode 100644 index 000000000000..b5bad0ffdc71 --- /dev/null +++ b/tests/integration/test_storage_iceberg_interoperability_azure/configs/config.d/named_collections.xml @@ -0,0 +1,8 @@ + + + + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + + diff --git a/tests/integration/test_storage_iceberg_interoperability_azure/configs/users.d/users.xml b/tests/integration/test_storage_iceberg_interoperability_azure/configs/users.d/users.xml new file mode 100644 index 000000000000..4b6ba057ecb1 --- /dev/null +++ b/tests/integration/test_storage_iceberg_interoperability_azure/configs/users.d/users.xml @@ -0,0 +1,9 @@ + + + + + default + 1 + + + diff --git a/tests/integration/test_storage_iceberg_interoperability_azure/conftest.py b/tests/integration/test_storage_iceberg_interoperability_azure/conftest.py new file mode 100644 index 000000000000..11ebea147acd --- /dev/null +++ b/tests/integration/test_storage_iceberg_interoperability_azure/conftest.py @@ -0,0 +1,132 @@ +import pytest +import logging +import pyspark +import os +import os.path as p + +from helpers.cluster import ClickHouseCluster +from helpers.iceberg_utils import get_uuid_str +from helpers.spark_tools import ResilientSparkSession, write_spark_log_config + + +AZURE_ACCOUNT_NAME = "devstoreaccount1" +AZURE_CONTAINER = "testcontainer" + + +def get_spark(log_dir=None): + """ + Configure Spark to write Iceberg tables to Azurite via WASB (HTTP). + + Key insight: hadoop-azure's emulator mode config must use the FQDN + (devstoreaccount1.blob.core.windows.net) because it does an exact match + with the account name extracted from the WASB URI. + """ + hadoop_version = "3.3.4" + + builder = ( + pyspark.sql.SparkSession.builder + .appName("IcebergAzureInteroperability") + .config("spark.jars.repositories", "https://repo1.maven.org/maven2") + .config("spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") + .config("spark.sql.catalog.spark_catalog", + "org.apache.iceberg.spark.SparkSessionCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", + f"wasb://{AZURE_CONTAINER}@{AZURE_ACCOUNT_NAME}.blob.core.windows.net" + f"/iceberg_data") + # Enable emulator mode with FQDN — this makes hadoop-azure + # connect to http://127.0.0.1:10000 using dev storage credentials. + .config("spark.hadoop.fs.azure.storage.emulator.account.name", + f"{AZURE_ACCOUNT_NAME}.blob.core.windows.net") + .master("local") + ) + + if log_dir: + props_path = write_spark_log_config(log_dir) + builder = builder.config( + "spark.driver.extraJavaOptions", + f"-Dlog4j2.configurationFile=file:{props_path}", + ) + + return builder.getOrCreate() + + +@pytest.fixture(scope="package") +def started_cluster_iceberg(): + try: + # Force Azurite to port 10000 — the emulator mode hardcodes this port. + cluster = ClickHouseCluster( + __file__, with_spark=True, azurite_default_port=10000 + ) + cluster.add_instance( + "node1", + main_configs=[ + "configs/config.d/named_collections.xml", + ], + user_configs=["configs/users.d/users.xml"], + with_azurite=True, + stay_alive=True, + mem_limit="15g", + ) + + logging.info("Starting cluster...") + cluster.start() + + # Create test container + container_client = cluster.blob_service_client.get_container_client( + AZURE_CONTAINER + ) + if not container_client.exists(): + container_client.create_container() + cluster.azure_container_name = AZURE_CONTAINER + + cluster.spark_session = ResilientSparkSession( + lambda: get_spark(cluster.instances_dir) + ) + + yield cluster + + finally: + cluster.shutdown() + + +def test_spark_write_and_read(started_cluster_iceberg): + """Verify Spark can write to and read from Azurite via WASB emulator mode.""" + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_spark_roundtrip_" + get_uuid_str() + + # Write + spark.sql( + f""" + CREATE TABLE {TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + + spark.sql( + f""" + INSERT INTO {TABLE_NAME} + SELECT id as number FROM range(100) + """ + ) + + # Read back + df = spark.sql(f"SELECT count(*) as cnt FROM {TABLE_NAME}").collect() + count = df[0].cnt + logging.info(f"Spark read back {count} rows") + assert count == 100, f"Expected 100 rows, got {count}" + + # List blobs to see what paths Spark actually wrote + blob_client = started_cluster_iceberg.blob_service_client + container_client = blob_client.get_container_client(AZURE_CONTAINER) + blobs = list(container_client.list_blobs()) + print(f"Blobs in container ({len(blobs)}):") + for blob in blobs[:20]: + print(f" {blob.name}") + + assert len(blobs) > 0, "No blobs written to Azurite!" diff --git a/tests/integration/test_storage_iceberg_interoperability_azure/test_interoperability.py b/tests/integration/test_storage_iceberg_interoperability_azure/test_interoperability.py new file mode 100644 index 000000000000..06acc86383c6 --- /dev/null +++ b/tests/integration/test_storage_iceberg_interoperability_azure/test_interoperability.py @@ -0,0 +1,270 @@ +import logging + +from helpers.iceberg_utils import get_uuid_str + +AZURE_CONTAINER = "testcontainer" + +def test_spark_write_ch_read_append(started_cluster_iceberg): + """Spark writes, CH reads, Spark appends, CH reads updated data.""" + instance = started_cluster_iceberg.instances["node1"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_spark_append_" + get_uuid_str() + azurite_url = started_cluster_iceberg.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + blob_path = f"iceberg_data/default/{TABLE_NAME}/" + + # Spark creates the table and inserts initial data + spark.sql( + f""" + CREATE TABLE {TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + spark.sql(f"INSERT INTO {TABLE_NAME} SELECT id as number FROM range(100)") + + # Create ClickHouse table pointing to the same Azurite location + instance.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergAzure(azure, + container = '{AZURE_CONTAINER}', + storage_account_url = '{azurite_url}', + blob_path = '{blob_path}') + """ + ) + + # CH reads Spark's data + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 100, f"Expected 100 rows, got {rows}" + + result = instance.query(f"SELECT sum(number) FROM {TABLE_NAME}") + assert int(result) == 4950, f"Expected sum 4950, got {result.strip()}" + + # Spark appends more data + spark.sql(f"INSERT INTO {TABLE_NAME} SELECT id + 100 as number FROM range(50)") + + # CH reads the updated data + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 150, f"Expected 150 rows after append, got {rows}" + + +def test_ch_write_spark_read(started_cluster_iceberg): + """ClickHouse writes to an Iceberg table on Azurite that Spark created. + Tests that CH can correctly resolve Spark's wasb:// metadata paths and + append new data while preserving the existing Spark-written data. + Verifies Spark can read back via both SQL catalog and wasb:// path.""" + instance = started_cluster_iceberg.instances["node1"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_ch_write_" + get_uuid_str() + azurite_url = started_cluster_iceberg.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + blob_path = f"iceberg_data/default/{TABLE_NAME}/" + + # Spark creates the table and inserts initial data + spark.sql( + f""" + CREATE TABLE {TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + spark.sql(f"INSERT INTO {TABLE_NAME} SELECT id as number FROM range(10)") + + # Create ClickHouse table pointing to the same Azurite location. + # iceberg_use_version_hint writes version-hint.text so Spark's HadoopCatalog + # can discover the latest metadata version after session restart. + instance.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergAzure(azure, + container = '{AZURE_CONTAINER}', + storage_account_url = '{azurite_url}', + blob_path = '{blob_path}') + SETTINGS iceberg_use_version_hint = 1 + """ + ) + + # CH reads Spark's data + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 10, f"Expected 10 rows from Spark, got {rows}" + + # ClickHouse writes more data. + # write_full_path_in_iceberg_metadata is needed so Spark can resolve the + # data file paths (Spark expects wasb:// URIs, not relative paths). + insert_settings = { + "allow_insert_into_iceberg": 1, + "write_full_path_in_iceberg_metadata": 1, + } + instance.query(f"INSERT INTO {TABLE_NAME} VALUES (42)", settings=insert_settings) + instance.query(f"INSERT INTO {TABLE_NAME} VALUES (123)", settings=insert_settings) + + # ClickHouse can read its own writes (10 from Spark + 2 from CH) + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 12, f"Expected 12 rows, got {rows}" + + result = instance.query(f"SELECT sum(number) FROM {TABLE_NAME}") + expected_sum = sum(range(10)) + 42 + 123 # 45 + 42 + 123 = 210 + assert int(result) == expected_sum, f"Expected sum {expected_sum}, got {result.strip()}" + + # Spark should also see the data written by ClickHouse. + started_cluster_iceberg.spark_session._restart() + spark = started_cluster_iceberg.spark_session + + # Read via SQL catalog + df = spark.sql(f"SELECT * FROM {TABLE_NAME}").collect() + assert len(df) == 12, f"Spark SQL expected 12 rows, got {len(df)}" + spark_values = sorted([row.number for row in df]) + assert 42 in spark_values, f"Spark SQL missing CH-written value 42: {spark_values}" + assert 123 in spark_values, f"Spark SQL missing CH-written value 123: {spark_values}" + + +def test_spark_delete_ch_read(started_cluster_iceberg): + """Spark creates a table, inserts data, deletes some rows, and CH sees the deletions.""" + instance = started_cluster_iceberg.instances["node1"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_spark_delete_" + get_uuid_str() + azurite_url = started_cluster_iceberg.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + blob_path = f"iceberg_data/default/{TABLE_NAME}/" + + # Spark creates the table with merge-on-read delete mode (position deletes) + spark.sql( + f""" + CREATE TABLE {TABLE_NAME} ( + number INT + ) + USING iceberg + TBLPROPERTIES ( + 'format-version' = '2', + 'write.update.mode' = 'merge-on-read', + 'write.delete.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' + ); + """ + ) + spark.sql(f"INSERT INTO {TABLE_NAME} SELECT id as number FROM range(100)") + + # Create ClickHouse table + instance.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergAzure(azure, + container = '{AZURE_CONTAINER}', + storage_account_url = '{azurite_url}', + blob_path = '{blob_path}') + """ + ) + + # CH reads all 100 rows + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 100, f"Expected 100 rows, got {rows}" + + # Spark deletes rows where number < 20 + spark.sql(f"DELETE FROM {TABLE_NAME} WHERE number < 20") + + # CH should see only 80 rows + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 80, f"Expected 80 rows after first delete, got {rows}" + + result = int(instance.query(f"SELECT min(number) FROM {TABLE_NAME}")) + assert result == 20, f"Expected min 20 after delete, got {result}" + + # Spark deletes more rows + spark.sql(f"DELETE FROM {TABLE_NAME} WHERE number >= 90") + + # CH should see only 70 rows (20..89) + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 70, f"Expected 70 rows after second delete, got {rows}" + + result = int(instance.query(f"SELECT max(number) FROM {TABLE_NAME}")) + assert result == 89, f"Expected max 89 after delete, got {result}" + + result = int(instance.query(f"SELECT sum(number) FROM {TABLE_NAME}")) + expected_sum = sum(range(20, 90)) + assert result == expected_sum, f"Expected sum {expected_sum}, got {result}" + + +def test_ch_delete_spark_read(started_cluster_iceberg): + """Spark creates a table, CH deletes some rows, and Spark sees the deletions.""" + instance = started_cluster_iceberg.instances["node1"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_ch_delete_" + get_uuid_str() + azurite_url = started_cluster_iceberg.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + blob_path = f"iceberg_data/default/{TABLE_NAME}/" + + # Spark creates the table and inserts data + spark.sql( + f""" + CREATE TABLE {TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + spark.sql(f"INSERT INTO {TABLE_NAME} SELECT id as number FROM range(50)") + + # Create ClickHouse table with version hint so Spark can discover CH's metadata updates + instance.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergAzure(azure, + container = '{AZURE_CONTAINER}', + storage_account_url = '{azurite_url}', + blob_path = '{blob_path}') + SETTINGS iceberg_use_version_hint = 1 + """ + ) + + # CH reads all 50 rows + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 50, f"Expected 50 rows, got {rows}" + + # CH deletes some rows + delete_settings = { + "allow_insert_into_iceberg": 1, + "write_full_path_in_iceberg_metadata": 1, + } + instance.query( + f"ALTER TABLE {TABLE_NAME} DELETE WHERE number < 10", + settings=delete_settings, + ) + + # CH sees the deletion + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 40, f"Expected 40 rows after CH delete, got {rows}" + + # Spark should also see the deletion + started_cluster_iceberg.spark_session._restart() + spark = started_cluster_iceberg.spark_session + + df = spark.sql(f"SELECT * FROM {TABLE_NAME}").collect() + assert len(df) == 40, f"Spark expected 40 rows after CH delete, got {len(df)}" + spark_values = sorted([row.number for row in df]) + assert min(spark_values) == 10, f"Spark expected min 10, got {min(spark_values)}" + + # CH deletes more rows + instance.query( + f"ALTER TABLE {TABLE_NAME} DELETE WHERE number >= 40", + settings=delete_settings, + ) + + # CH sees the deletion + rows = int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 30, f"Expected 30 rows after second CH delete, got {rows}" + + # Spark should also see the second deletion + started_cluster_iceberg.spark_session._restart() + spark = started_cluster_iceberg.spark_session + + df = spark.sql(f"SELECT * FROM {TABLE_NAME}").collect() + assert len(df) == 30, f"Spark expected 30 rows after second CH delete, got {len(df)}" + spark_values = sorted([row.number for row in df]) + assert spark_values == list(range(10, 40)), \ + f"Spark expected values 10..39, got {spark_values}" \ No newline at end of file diff --git a/tests/integration/test_storage_iceberg_interoperability_local/__init__.py b/tests/integration/test_storage_iceberg_interoperability_local/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_storage_iceberg_concurrent/configs/config.d/cluster.xml b/tests/integration/test_storage_iceberg_interoperability_local/configs/config.d/cluster.xml similarity index 68% rename from tests/integration/test_storage_iceberg_concurrent/configs/config.d/cluster.xml rename to tests/integration/test_storage_iceberg_interoperability_local/configs/config.d/cluster.xml index c66b0e1a1057..7927d4613a28 100644 --- a/tests/integration/test_storage_iceberg_concurrent/configs/config.d/cluster.xml +++ b/tests/integration/test_storage_iceberg_interoperability_local/configs/config.d/cluster.xml @@ -6,14 +6,6 @@ node1 9000 - - node2 - 9000 - - - node3 - 9000 - diff --git a/tests/integration/test_storage_iceberg_interoperability_local/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg_interoperability_local/configs/config.d/named_collections.xml new file mode 100644 index 000000000000..ca41a070a178 --- /dev/null +++ b/tests/integration/test_storage_iceberg_interoperability_local/configs/config.d/named_collections.xml @@ -0,0 +1,6 @@ + + + + + + diff --git a/tests/integration/test_storage_iceberg_interoperability_local/configs/users.d/users.xml b/tests/integration/test_storage_iceberg_interoperability_local/configs/users.d/users.xml new file mode 100644 index 000000000000..4b6ba057ecb1 --- /dev/null +++ b/tests/integration/test_storage_iceberg_interoperability_local/configs/users.d/users.xml @@ -0,0 +1,9 @@ + + + + + default + 1 + + + diff --git a/tests/integration/test_storage_iceberg_interoperability_local/conftest.py b/tests/integration/test_storage_iceberg_interoperability_local/conftest.py new file mode 100644 index 000000000000..b38a67513246 --- /dev/null +++ b/tests/integration/test_storage_iceberg_interoperability_local/conftest.py @@ -0,0 +1,160 @@ +import pytest +import logging +import pyspark +import os + +from helpers.cluster import ClickHouseCluster +from helpers.iceberg_utils import get_uuid_str +from helpers.spark_tools import ResilientSparkSession, write_spark_log_config + + +# Each node gets its own iceberg data directory so they don't see each other's writes. +# external_dirs mounts / (host) → (container). +# We then create symlinks on the host so that the container path also resolves +# on the host — this way Iceberg metadata with absolute paths works for both +# Spark (host) and ClickHouse (container). +ICEBERG_DIR_NODE1 = "/var/lib/clickhouse/user_files/iceberg_node1" +ICEBERG_DIR_NODE2 = "/var/lib/clickhouse/user_files/iceberg_node2" + + +def create_host_symlink(container_path, host_path): + """ + Create a symlink on the host from container_path → host_path. + After this, both Spark (on host) and ClickHouse (in container) + can use the same absolute path to access the data. + """ + os.makedirs(os.path.dirname(container_path), exist_ok=True) + if os.path.exists(container_path): + if os.path.islink(container_path): + os.remove(container_path) + else: + return # Real directory exists (e.g., actual ClickHouse installation), don't touch it + os.symlink(host_path, container_path) + logging.info(f"Created symlink: {container_path} → {host_path}") + + +def cleanup_host_symlink(container_path): + """Remove symlink created by create_host_symlink.""" + if os.path.islink(container_path): + os.remove(container_path) + logging.info(f"Removed symlink: {container_path}") + + +def get_spark(warehouse_node1, warehouse_node2, log_dir=None): + builder = ( + pyspark.sql.SparkSession.builder + .appName("IcebergLocalTwoNodes") + .config("spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") + # Catalog for node1 + .config("spark.sql.catalog.node1_catalog", + "org.apache.iceberg.spark.SparkCatalog") + .config("spark.sql.catalog.node1_catalog.type", "hadoop") + .config("spark.sql.catalog.node1_catalog.warehouse", warehouse_node1) + # Catalog for node2 + .config("spark.sql.catalog.node2_catalog", + "org.apache.iceberg.spark.SparkCatalog") + .config("spark.sql.catalog.node2_catalog.type", "hadoop") + .config("spark.sql.catalog.node2_catalog.warehouse", warehouse_node2) + .master("local") + ) + + if log_dir: + props_path = write_spark_log_config(log_dir) + builder = builder.config( + "spark.driver.extraJavaOptions", + f"-Dlog4j2.configurationFile=file:{props_path}", + ) + + return builder.getOrCreate() + + +@pytest.fixture(scope="package") +def started_cluster_iceberg(): + symlinks = [] + try: + cluster = ClickHouseCluster(__file__, with_spark=True) + cluster.add_instance( + "node1", + main_configs=[ + "configs/config.d/cluster.xml", + "configs/config.d/named_collections.xml", + ], + user_configs=["configs/users.d/users.xml"], + stay_alive=True, + mem_limit="15g", + external_dirs=[ICEBERG_DIR_NODE1], + ) + cluster.add_instance( + "node2", + main_configs=[ + "configs/config.d/cluster.xml", + "configs/config.d/named_collections.xml", + ], + user_configs=["configs/users.d/users.xml"], + stay_alive=True, + mem_limit="15g", + external_dirs=[ICEBERG_DIR_NODE2], + ) + + logging.info("Starting cluster...") + cluster.start() + + # external_dirs creates: + # host: /var/lib/clickhouse/user_files/iceberg_node1 + # container: /var/lib/clickhouse/user_files/iceberg_node1 + # + # Create symlinks on the host so the container path resolves to the + # host path. Now both Spark and ClickHouse use the same absolute paths. + for iceberg_dir in [ICEBERG_DIR_NODE1, ICEBERG_DIR_NODE2]: + host_path = os.path.join( + cluster.instances_dir, iceberg_dir.lstrip("/") + ) + create_host_symlink(iceberg_dir, host_path) + symlinks.append(iceberg_dir) + + # Both Spark and ClickHouse use the container paths. + # On the host, these resolve via symlinks to the actual data. + cluster.spark_session = ResilientSparkSession( + lambda: get_spark(ICEBERG_DIR_NODE1, ICEBERG_DIR_NODE2, cluster.instances_dir) + ) + + yield cluster + + finally: + for link in symlinks: + cleanup_host_symlink(link) + cluster.shutdown() + + +def test_spark_write_and_read(started_cluster_iceberg): + """Verify Spark can write to and read from local filesystem via Iceberg.""" + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_spark_roundtrip_" + get_uuid_str() + + # Write + spark.sql( + f""" + CREATE TABLE node1_catalog.default.{TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + + spark.sql( + f""" + INSERT INTO node1_catalog.default.{TABLE_NAME} + SELECT id as number FROM range(100) + """ + ) + + # Read back + df = spark.sql( + f"SELECT count(*) as cnt FROM node1_catalog.default.{TABLE_NAME}" + ).collect() + count = df[0].cnt + logging.info(f"Spark read back {count} rows") + assert count == 100, f"Expected 100 rows, got {count}" diff --git a/tests/integration/test_storage_iceberg_interoperability_local/test_interoperability.py b/tests/integration/test_storage_iceberg_interoperability_local/test_interoperability.py new file mode 100644 index 000000000000..0d11b0f1a1f7 --- /dev/null +++ b/tests/integration/test_storage_iceberg_interoperability_local/test_interoperability.py @@ -0,0 +1,326 @@ +from helpers.iceberg_utils import get_uuid_str + +ICEBERG_DIR_NODE1 = "/var/lib/clickhouse/user_files/iceberg_node1" + + +def test_nodes_dont_see_each_other(started_cluster_iceberg): + """ + Spark writes different data to each node's local directory. + Each node only sees its own data. + """ + node1 = started_cluster_iceberg.instances["node1"] + node2 = started_cluster_iceberg.instances["node2"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_isolation_" + get_uuid_str() + + # Create Iceberg tables via Spark — one per node catalog + spark.sql( + f""" + CREATE TABLE node1_catalog.default.{TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + + spark.sql( + f""" + CREATE TABLE node2_catalog.default.{TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + + # Write 100 rows to node1, 200 rows to node2 + spark.sql( + f""" + INSERT INTO node1_catalog.default.{TABLE_NAME} + SELECT id as number FROM range(100) + """ + ) + + spark.sql( + f""" + INSERT INTO node2_catalog.default.{TABLE_NAME} + SELECT id as number FROM range(200) + """ + ) + + # Create ClickHouse tables — each node reads from its own iceberg directory + node1.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergLocal(local, + path = '/var/lib/clickhouse/user_files/iceberg_node1/default/{TABLE_NAME}') + """ + ) + node2.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergLocal(local, + path = '/var/lib/clickhouse/user_files/iceberg_node2/default/{TABLE_NAME}') + """ + ) + + # Each node should only see its own data + rows_node1 = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + rows_node2 = int(node2.query(f"SELECT count() FROM {TABLE_NAME}")) + + assert rows_node1 == 100, f"node1: expected 100 rows, got {rows_node1}" + assert rows_node2 == 200, f"node2: expected 200 rows, got {rows_node2}" + + # Append more data to node1 only + spark.sql( + f""" + INSERT INTO node1_catalog.default.{TABLE_NAME} + SELECT id + 100 as number FROM range(50) + """ + ) + + rows_node1 = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + rows_node2 = int(node2.query(f"SELECT count() FROM {TABLE_NAME}")) + + assert rows_node1 == 150, f"node1: expected 150 rows after append, got {rows_node1}" + assert rows_node2 == 200, f"node2: should still have 200 rows, got {rows_node2}" + + +def test_ch_write_spark_read(started_cluster_iceberg): + """ + Spark creates a table, ClickHouse writes to it, Spark reads back. + Validates that the external_dirs mount works bidirectionally. + """ + node1 = started_cluster_iceberg.instances["node1"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_ch_write_spark_read_" + get_uuid_str() + + # Spark creates the table structure + spark.sql( + f""" + CREATE TABLE node1_catalog.default.{TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + + # Create ClickHouse table pointing to the same location + node1.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergLocal(local, + path = '/var/lib/clickhouse/user_files/iceberg_node1/default/{TABLE_NAME}') + """ + ) + + # ClickHouse writes data + node1.query( + f"INSERT INTO {TABLE_NAME} VALUES (42)", + settings={"allow_insert_into_iceberg": 1}, + ) + node1.query( + f"INSERT INTO {TABLE_NAME} VALUES (123)", + settings={"allow_insert_into_iceberg": 1}, + ) + + # ClickHouse can read its own writes + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 2 + + # Spark should also see the data written by ClickHouse. + # Spark's catalog caches metadata, so we need to refresh it first. + spark.sql(f"REFRESH TABLE node1_catalog.default.{TABLE_NAME}") + + df = spark.sql( + f"SELECT * FROM node1_catalog.default.{TABLE_NAME}" + ).collect() + assert len(df) == 2, f"Spark expected 2 rows, got {len(df)}" + + spark_values = sorted([row.number for row in df]) + assert spark_values == [42, 123], f"Spark got unexpected values: {spark_values}" + + +def test_spark_write_ch_read_append(started_cluster_iceberg): + """Spark writes, CH reads, Spark appends, CH reads updated data.""" + node1 = started_cluster_iceberg.instances["node1"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_spark_append_" + get_uuid_str() + + # Spark creates the table and inserts initial data + spark.sql( + f""" + CREATE TABLE node1_catalog.default.{TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + spark.sql( + f"INSERT INTO node1_catalog.default.{TABLE_NAME} SELECT id as number FROM range(100)" + ) + + # Create ClickHouse table pointing to the same location + node1.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergLocal(local, + path = '{ICEBERG_DIR_NODE1}/default/{TABLE_NAME}') + """ + ) + + # CH reads Spark's data + rows = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 100, f"Expected 100 rows, got {rows}" + + result = node1.query(f"SELECT sum(number) FROM {TABLE_NAME}") + assert int(result) == 4950, f"Expected sum 4950, got {result.strip()}" + + # Spark appends more data + spark.sql( + f"INSERT INTO node1_catalog.default.{TABLE_NAME} SELECT id + 100 as number FROM range(50)" + ) + + # CH reads the updated data + rows = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 150, f"Expected 150 rows after append, got {rows}" + + +def test_spark_delete_ch_read(started_cluster_iceberg): + """Spark creates a table, inserts data, deletes some rows, and CH sees the deletions.""" + node1 = started_cluster_iceberg.instances["node1"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_spark_delete_" + get_uuid_str() + + # Spark creates the table with merge-on-read delete mode (position deletes) + spark.sql( + f""" + CREATE TABLE node1_catalog.default.{TABLE_NAME} ( + number INT + ) + USING iceberg + TBLPROPERTIES ( + 'format-version' = '2', + 'write.update.mode' = 'merge-on-read', + 'write.delete.mode' = 'merge-on-read', + 'write.merge.mode' = 'merge-on-read' + ); + """ + ) + spark.sql( + f"INSERT INTO node1_catalog.default.{TABLE_NAME} SELECT id as number FROM range(100)" + ) + + # Create ClickHouse table + node1.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergLocal(local, + path = '{ICEBERG_DIR_NODE1}/default/{TABLE_NAME}') + """ + ) + + # CH reads all 100 rows + rows = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 100, f"Expected 100 rows, got {rows}" + + # Spark deletes rows where number < 20 + spark.sql(f"DELETE FROM node1_catalog.default.{TABLE_NAME} WHERE number < 20") + + # CH should see only 80 rows + rows = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 80, f"Expected 80 rows after first delete, got {rows}" + + result = int(node1.query(f"SELECT min(number) FROM {TABLE_NAME}")) + assert result == 20, f"Expected min 20 after delete, got {result}" + + # Spark deletes more rows + spark.sql(f"DELETE FROM node1_catalog.default.{TABLE_NAME} WHERE number >= 90") + + # CH should see only 70 rows (20..89) + rows = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 70, f"Expected 70 rows after second delete, got {rows}" + + result = int(node1.query(f"SELECT max(number) FROM {TABLE_NAME}")) + assert result == 89, f"Expected max 89 after delete, got {result}" + + result = int(node1.query(f"SELECT sum(number) FROM {TABLE_NAME}")) + expected_sum = sum(range(20, 90)) + assert result == expected_sum, f"Expected sum {expected_sum}, got {result}" + + +def test_ch_delete_spark_read(started_cluster_iceberg): + """Spark creates a table, CH deletes some rows, and Spark sees the deletions.""" + node1 = started_cluster_iceberg.instances["node1"] + spark = started_cluster_iceberg.spark_session + + TABLE_NAME = "test_ch_delete_" + get_uuid_str() + + # Spark creates the table and inserts data + spark.sql( + f""" + CREATE TABLE node1_catalog.default.{TABLE_NAME} ( + number INT + ) + USING iceberg + OPTIONS('format-version'='2'); + """ + ) + spark.sql( + f"INSERT INTO node1_catalog.default.{TABLE_NAME} SELECT id as number FROM range(50)" + ) + + # Create ClickHouse table + node1.query( + f""" + CREATE TABLE {TABLE_NAME} + ENGINE=IcebergLocal(local, + path = '{ICEBERG_DIR_NODE1}/default/{TABLE_NAME}') + """ + ) + + # CH reads all 50 rows + rows = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 50, f"Expected 50 rows, got {rows}" + + # CH deletes some rows + delete_settings = {"allow_insert_into_iceberg": 1} + node1.query( + f"ALTER TABLE {TABLE_NAME} DELETE WHERE number < 10", + settings=delete_settings, + ) + + # CH sees the deletion + rows = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 40, f"Expected 40 rows after CH delete, got {rows}" + + # Spark should also see the deletion + spark.sql(f"REFRESH TABLE node1_catalog.default.{TABLE_NAME}") + df = spark.sql(f"SELECT * FROM node1_catalog.default.{TABLE_NAME}").collect() + assert len(df) == 40, f"Spark expected 40 rows after CH delete, got {len(df)}" + spark_values = sorted([row.number for row in df]) + assert min(spark_values) == 10, f"Spark expected min 10, got {min(spark_values)}" + + # CH deletes more rows + node1.query( + f"ALTER TABLE {TABLE_NAME} DELETE WHERE number >= 40", + settings=delete_settings, + ) + + # CH sees the deletion + rows = int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) + assert rows == 30, f"Expected 30 rows after second CH delete, got {rows}" + + # Spark should also see the second deletion + spark.sql(f"REFRESH TABLE node1_catalog.default.{TABLE_NAME}") + df = spark.sql(f"SELECT * FROM node1_catalog.default.{TABLE_NAME}").collect() + assert len(df) == 30, f"Spark expected 30 rows after second CH delete, got {len(df)}" + spark_values = sorted([row.number for row in df]) + assert spark_values == list(range(10, 40)), \ + f"Spark expected values 10..39, got {spark_values}" diff --git a/tests/integration/test_storage_iceberg_with_spark/test_remove_orphan_files.py b/tests/integration/test_storage_iceberg_with_spark/test_remove_orphan_files.py new file mode 100644 index 000000000000..6600e1ef6eb2 --- /dev/null +++ b/tests/integration/test_storage_iceberg_with_spark/test_remove_orphan_files.py @@ -0,0 +1,583 @@ +import io +import json +import time + +import pytest + +from helpers.iceberg_utils import ( + create_iceberg_table, + drop_iceberg_table, + get_uuid_str, +) + + +ICEBERG_SETTINGS = { + "allow_insert_into_iceberg": 1, + "allow_iceberg_remove_orphan_files": 1, +} + +LOCAL_TABLE_PREFIX = "/var/lib/clickhouse/user_files/iceberg_data/default" +S3_TABLE_PREFIX = "var/lib/clickhouse/user_files/iceberg_data/default" + + +# --------------------------------------------------------------------------- +# Test environment — binds cluster/instance/storage_type/table_name once +# so individual tests only express the orphan-specific logic. +# --------------------------------------------------------------------------- + +class OrphanTestEnv: + def __init__(self, cluster, storage_type, table_prefix): + self.cluster = cluster + self.instance = cluster.instances["node1"] + self.storage_type = storage_type + self.table_name = f"{table_prefix}_{storage_type}_{get_uuid_str()}" + + # -- table lifecycle ----------------------------------------------------- + + def populate(self, n_rows, format_version=2): + create_iceberg_table( + self.storage_type, self.instance, self.table_name, + self.cluster, "(x Int)", format_version, + ) + for val in range(1, n_rows + 1): + self.instance.query( + f"INSERT INTO {self.table_name} VALUES ({val});", + settings=ICEBERG_SETTINGS, + ) + self._n_rows = n_rows + + def assert_data_intact(self): + expected = "".join(f"{i}\n" for i in range(1, self._n_rows + 1)) + assert self.instance.query( + f"SELECT * FROM {self.table_name} ORDER BY x" + ) == expected + + # -- orphan file manipulation ------------------------------------------- + + def add_orphan(self, subdir="data", filename="orphan.parquet"): + if self.storage_type == "local": + table_dir = f"{LOCAL_TABLE_PREFIX}/{self.table_name}" + target_dir = f"{table_dir}/{subdir}" + self.instance.exec_in_container( + ["bash", "-c", + f"mkdir -p {target_dir} && echo 'orphan_data' > {target_dir}/{filename}"] + ) + elif self.storage_type == "azure": + blob_path = f"/var/lib/clickhouse/user_files/iceberg_data/default/{self.table_name}/{subdir}/{filename}" + blob_client = self.cluster.blob_service_client.get_blob_client( + self.cluster.azure_container_name, blob_path, + ) + blob_client.upload_blob(b"orphan_data", overwrite=True) + else: + key = f"{S3_TABLE_PREFIX}/{self.table_name}/{subdir}/{filename}" + data = b"orphan_data" + self.cluster.minio_client.put_object( + self.cluster.minio_bucket, key, io.BytesIO(data), len(data), + ) + + def add_orphan_metadata(self, filename="v0.metadata.json"): + self.add_orphan(subdir="metadata", filename=filename) + + # -- storage queries ---------------------------------------------------- + + def exists(self, subdir, filename): + if self.storage_type == "local": + path = f"{LOCAL_TABLE_PREFIX}/{self.table_name}/{subdir}/{filename}" + ret = self.instance.exec_in_container( + ["bash", "-c", f"test -f {path} && echo 'exists' || echo 'missing'"] + ).strip() + return ret == "exists" + elif self.storage_type == "azure": + blob_path = f"/var/lib/clickhouse/user_files/iceberg_data/default/{self.table_name}/{subdir}/{filename}" + blob_client = self.cluster.blob_service_client.get_blob_client( + self.cluster.azure_container_name, blob_path, + ) + try: + blob_client.get_blob_properties() + return True + except Exception: + return False + else: + key = f"{S3_TABLE_PREFIX}/{self.table_name}/{subdir}/{filename}" + try: + self.cluster.minio_client.stat_object(self.cluster.minio_bucket, key) + return True + except Exception: + return False + + def list_files(self): + if self.storage_type == "local": + table_dir = f"{LOCAL_TABLE_PREFIX}/{self.table_name}" + output = self.instance.exec_in_container( + ["bash", "-c", f"find {table_dir} -type f 2>/dev/null | sort"] + ).strip() + return output.split("\n") if output else [] + elif self.storage_type == "azure": + prefix = f"/var/lib/clickhouse/user_files/iceberg_data/default/{self.table_name}/" + container_client = self.cluster.blob_service_client.get_container_client( + self.cluster.azure_container_name, + ) + return sorted(b.name for b in container_client.list_blobs(name_starts_with=prefix)) + else: + prefix = f"{S3_TABLE_PREFIX}/{self.table_name}/" + return sorted( + obj.object_name + for obj in self.cluster.minio_client.list_objects( + self.cluster.minio_bucket, prefix=prefix, recursive=True, + ) + ) + + # -- command execution -------------------------------------------------- + + def remove_orphans(self, **kwargs): + args_parts = [] + if "older_than" in kwargs: + args_parts.append(f"older_than = '{kwargs['older_than']}'") + if "location" in kwargs: + args_parts.append(f"location = '{kwargs['location']}'") + if "dry_run" in kwargs: + args_parts.append(f"dry_run = {kwargs['dry_run']}") + if "positional_ts" in kwargs: + args_str = f"'{kwargs['positional_ts']}'" + if args_parts: + args_str += ", " + ", ".join(args_parts) + else: + args_str = ", ".join(args_parts) + + raw = self.instance.query( + f"ALTER TABLE {self.table_name} EXECUTE remove_orphan_files({args_str});", + settings=ICEBERG_SETTINGS, + ) + counts = {} + for line in raw.strip().split("\n"): + if not line: + continue + parts = line.split("\t") + if len(parts) == 2: + counts[parts[0]] = int(parts[1]) + return counts + + @staticmethod + def now_ts(): + return time.strftime("%Y-%m-%d %H:%M:%S") + + +def make_env(cluster, storage_type, prefix): + return OrphanTestEnv(cluster, storage_type, prefix) + + +# --------------------------------------------------------------------------- +# Tests +# --------------------------------------------------------------------------- + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_basic(started_cluster_iceberg_with_spark, storage_type): + """Create orphan files, run remove_orphan_files, verify they are removed + and legitimate files are preserved.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_basic") + env.populate(3) + + env.add_orphan("data", "orphan-data-001.parquet") + env.add_orphan("data", "orphan-data-002.parquet") + time.sleep(2) + + files_before = env.list_files() + assert any("orphan-data-001.parquet" in f for f in files_before) + assert any("orphan-data-002.parquet" in f for f in files_before) + + counts = env.remove_orphans(older_than=env.now_ts()) + assert len(counts) == 9, f"Expected 9 metrics, got {counts}" + assert counts["deleted_data_files_count"] >= 2, f"Expected >= 2 deleted data files, got {counts}" + + assert not env.exists("data", "orphan-data-001.parquet") + assert not env.exists("data", "orphan-data-002.parquet") + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_no_orphans(started_cluster_iceberg_with_spark, storage_type): + """When there are no user-created orphan files, data/manifest/statistics counts should be zero. + Metadata files (old v*.metadata.json) may legitimately be orphaned after multiple inserts.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_no_orphans") + env.populate(2) + time.sleep(2) + + counts = env.remove_orphans(older_than=env.now_ts()) + non_metadata = {k: v for k, v in counts.items() if k not in ("deleted_metadata_files_count", "skipped_missing_metadata_count")} + assert all(v == 0 for v in non_metadata.values()), f"Expected data/manifest/stat zeros, got {counts}" + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local"]) +def test_remove_orphan_files_default_older_than(started_cluster_iceberg_with_spark, storage_type): + """Zero-argument form: older_than defaults to now - iceberg_orphan_files_older_than_seconds.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_default") + env.populate(1) + + env.add_orphan("data", "orphan-default.parquet") + time.sleep(2) + + settings_with_short_threshold = { + **ICEBERG_SETTINGS, + "iceberg_orphan_files_older_than_seconds": 1, + } + + raw = env.instance.query( + f"ALTER TABLE {env.table_name} EXECUTE remove_orphan_files();", + settings=settings_with_short_threshold, + ) + counts = {} + for line in raw.strip().split("\n"): + if not line: + continue + parts = line.split("\t") + if len(parts) == 2: + counts[parts[0]] = int(parts[1]) + + assert counts["deleted_data_files_count"] >= 1, \ + f"Zero-arg form with 1s threshold should delete orphan, got {counts}" + assert not env.exists("data", "orphan-default.parquet"), \ + "Orphan file should be deleted via setting-driven default older_than" + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_older_than(started_cluster_iceberg_with_spark, storage_type): + """Orphan files newer than older_than threshold should be preserved.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_older_than") + env.populate(1) + + past_ts = env.now_ts() + time.sleep(2) + + env.add_orphan("data", "orphan-new.parquet") + time.sleep(1) + + env.remove_orphans(older_than=past_ts) + assert env.exists("data", "orphan-new.parquet"), \ + "Orphan newer than older_than should NOT be deleted" + + time.sleep(1) + counts = env.remove_orphans(older_than=env.now_ts()) + assert counts["deleted_data_files_count"] >= 1 + assert not env.exists("data", "orphan-new.parquet"), \ + "Orphan older than threshold should be deleted" + + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_dry_run(started_cluster_iceberg_with_spark, storage_type): + """dry_run=1 should report counts but not delete files.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_dry_run") + env.populate(1) + + env.add_orphan("data", "orphan-dry.parquet") + time.sleep(2) + + counts = env.remove_orphans(older_than=env.now_ts(), dry_run=1) + assert counts["deleted_data_files_count"] >= 1 + assert env.exists("data", "orphan-dry.parquet"), "dry_run should NOT delete files" + + counts = env.remove_orphans(older_than=env.now_ts(), dry_run=0) + assert counts["deleted_data_files_count"] >= 1 + assert not env.exists("data", "orphan-dry.parquet"), "Without dry_run, orphan should be deleted" + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_location(started_cluster_iceberg_with_spark, storage_type): + """location parameter should restrict the scan to a subdirectory.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_location") + env.populate(1) + + env.add_orphan("data", "orphan-data.parquet") + env.add_orphan_metadata("v0.metadata.json") + time.sleep(2) + + now_ts = env.now_ts() + + counts = env.remove_orphans(older_than=now_ts, location="data/") + assert counts["deleted_data_files_count"] >= 1 + assert not env.exists("data", "orphan-data.parquet"), \ + "Data orphan in scanned location should be deleted" + assert env.exists("metadata", "v0.metadata.json"), \ + "Metadata orphan outside scanned location should survive" + + counts = env.remove_orphans(older_than=now_ts, location="metadata/") + assert counts["deleted_metadata_files_count"] >= 1 + assert not env.exists("metadata", "v0.metadata.json"), \ + "Metadata orphan in scanned location should be deleted" + + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_many_orphans(started_cluster_iceberg_with_spark, storage_type): + """remove_orphan_files should delete multiple orphan files in one run.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_many") + env.populate(1) + + for i in range(10): + env.add_orphan("data", f"orphan-par-{i:03d}.parquet") + time.sleep(2) + + counts = env.remove_orphans(older_than=env.now_ts()) + assert counts["deleted_data_files_count"] >= 10 + + for i in range(10): + assert not env.exists("data", f"orphan-par-{i:03d}.parquet") + + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_multiple_snapshots(started_cluster_iceberg_with_spark, storage_type): + """Files referenced by any snapshot are not considered orphans.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_multi_snap") + env.populate(3) + + env.add_orphan("data", "orphan-multi.parquet") + time.sleep(2) + + counts = env.remove_orphans(older_than=env.now_ts()) + assert counts["deleted_data_files_count"] >= 1 + assert not env.exists("data", "orphan-multi.parquet") + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_positional_arg(started_cluster_iceberg_with_spark, storage_type): + """Positional older_than argument should work the same as named.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_positional") + env.populate(1) + + env.add_orphan("data", "orphan-pos.parquet") + time.sleep(2) + + counts = env.remove_orphans(positional_ts=env.now_ts()) + assert counts["deleted_data_files_count"] >= 1 + assert not env.exists("data", "orphan-pos.parquet") + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_gate_setting(started_cluster_iceberg_with_spark, storage_type): + """Without allow_iceberg_remove_orphan_files, the command should fail.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_gate") + env.populate(1) + + error = env.instance.query_and_get_error( + f"ALTER TABLE {env.table_name} EXECUTE remove_orphan_files();", + settings={"allow_insert_into_iceberg": 1, "allow_iceberg_remove_orphan_files": 0}, + ) + assert "SUPPORT_IS_DISABLED" in error, f"Expected SUPPORT_IS_DISABLED error, got: {error}" + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_future_timestamp_rejected(started_cluster_iceberg_with_spark, storage_type): + """Passing an older_than in the future should be rejected with BAD_ARGUMENTS.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_future_ts") + env.populate(1) + + error = env.instance.query_and_get_error( + f"ALTER TABLE {env.table_name} EXECUTE remove_orphan_files('2099-01-01 00:00:00');", + settings=ICEBERG_SETTINGS, + ) + assert "BAD_ARGUMENTS" in error, f"Expected BAD_ARGUMENTS error, got: {error}" + + +@pytest.mark.parametrize("storage_type", ["local"]) +def test_remove_orphan_files_location_validation(started_cluster_iceberg_with_spark, storage_type): + """Path-traversal and absolute location values should be rejected.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_loc_val") + env.populate(1) + + for bad_loc in ["../escape", "/absolute/path"]: + error = env.instance.query_and_get_error( + f"ALTER TABLE {env.table_name} EXECUTE remove_orphan_files(" + f"older_than = '{env.now_ts()}', location = '{bad_loc}');", + settings=ICEBERG_SETTINGS, + ) + assert "BAD_ARGUMENTS" in error, f"location='{bad_loc}' should fail, got: {error}" + + env.add_orphan("data", "orphan-dotslash.parquet") + time.sleep(2) + counts = env.remove_orphans(older_than=env.now_ts(), location="./data/") + assert counts["deleted_data_files_count"] >= 1, \ + "location='./data/' should work the same as 'data/'" + + +@pytest.mark.parametrize("storage_type", ["local"]) +def test_remove_orphan_files_rejected_on_v1(started_cluster_iceberg_with_spark, storage_type): + """remove_orphan_files must reject Iceberg format-version 1 tables.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_v1") + env.populate(1, format_version=1) + + error = env.instance.query_and_get_error( + f"ALTER TABLE {env.table_name} EXECUTE remove_orphan_files();", + settings=ICEBERG_SETTINGS, + ) + assert "BAD_ARGUMENTS" in error, f"Expected BAD_ARGUMENTS error, got: {error}" + assert "format version" in error.lower(), f"Error should mention format version, got: {error}" + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_delete_file_categories(started_cluster_iceberg_with_spark, storage_type): + """Equality-delete and position-delete orphan files must be tallied + under their respective metrics, not misclassified.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_del_cat") + env.populate(1) + + env.add_orphan("data", "eq-delete-001.parquet") + env.add_orphan("data", "00000-0-eq-del-00001.parquet") + env.add_orphan("data", "00000-0-deletes.parquet") + env.add_orphan("data", "00000-0-delete-00001.parquet") + time.sleep(2) + + counts = env.remove_orphans(older_than=env.now_ts()) + + assert counts["deleted_equality_delete_files_count"] == 2, \ + f"Expected 2 equality-delete files, got {counts}" + assert counts["deleted_position_delete_files_count"] == 2, \ + f"Expected 2 position-delete files, got {counts}" + + +@pytest.mark.parametrize("storage_type", ["local", "s3"]) +def test_remove_orphan_files_location_scoped_deletion(started_cluster_iceberg_with_spark, storage_type): + """Files outside the specified location folder must NOT be deleted.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_loc_scope") + env.populate(1) + + env.add_orphan("data", "orphan-in-data.parquet") + env.add_orphan("data/subdir", "orphan-nested.parquet") + env.add_orphan_metadata("v0.metadata.json") + time.sleep(2) + + now_ts = env.now_ts() + counts = env.remove_orphans(older_than=now_ts, location="data/") + + assert counts["deleted_data_files_count"] >= 1, \ + f"Orphan in data/ should be deleted, got {counts}" + assert not env.exists("data", "orphan-in-data.parquet"), \ + "Orphan inside scanned location should be deleted" + + assert env.exists("metadata", "v0.metadata.json"), \ + "Orphan in metadata/ must survive when scanning data/" + + counts2 = env.remove_orphans(older_than=now_ts, location="metadata/") + assert counts2["deleted_metadata_files_count"] >= 1 + assert not env.exists("metadata", "v0.metadata.json"), \ + "Metadata orphan should be deleted when scanning metadata/" + + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["azure"]) +def test_remove_orphan_files_azure(started_cluster_iceberg_with_spark, storage_type): + """Orphan removal on Azure (Azurite) backend: create orphans, verify deletion.""" + env = make_env(started_cluster_iceberg_with_spark, storage_type, "test_orphan_azure") + env.populate(2) + + env.add_orphan("data", "orphan-azure-001.parquet") + env.add_orphan("data", "orphan-azure-002.parquet") + time.sleep(2) + + assert env.exists("data", "orphan-azure-001.parquet") + assert env.exists("data", "orphan-azure-002.parquet") + + counts = env.remove_orphans(older_than=env.now_ts()) + assert len(counts) == 9, f"Expected 9 metrics, got {counts}" + assert counts["deleted_data_files_count"] >= 2, f"Expected >= 2 deleted data files, got {counts}" + + assert not env.exists("data", "orphan-azure-001.parquet") + assert not env.exists("data", "orphan-azure-002.parquet") + env.assert_data_intact() + + +@pytest.mark.parametrize("storage_type", ["local"]) +def test_remove_orphan_files_ignores_pinned_metadata(started_cluster_iceberg_with_spark, storage_type): + """When iceberg_metadata_file_path pins the table to an older metadata + version, remove_orphan_files must still use the *latest* metadata to + determine reachable files. Otherwise it would treat data belonging to + newer (valid) snapshots as orphans and delete them.""" + + cluster = started_cluster_iceberg_with_spark + instance = cluster.instances["node1"] + table_name = f"test_orphan_pinned_{storage_type}_{get_uuid_str()}" + metadata_dir = f"{LOCAL_TABLE_PREFIX}/{table_name}/metadata" + + insert_settings = {"allow_insert_into_iceberg": 1} + + create_iceberg_table( + storage_type, instance, table_name, cluster, "(x Int)", format_version=2, + ) + instance.query( + f"INSERT INTO {table_name} VALUES (1);", + settings=insert_settings, + ) + instance.query( + f"INSERT INTO {table_name} VALUES (2);", + settings=insert_settings, + ) + + metadata_files_before = instance.exec_in_container( + ["bash", "-c", + f"ls -v {metadata_dir}/v*.metadata.json"] + ).strip().split("\n") + old_metadata_path = "metadata/" + metadata_files_before[-1].split("/")[-1] + + instance.query( + f"INSERT INTO {table_name} VALUES (3);", + settings=insert_settings, + ) + + data_files_before = instance.exec_in_container( + ["bash", "-c", + f"find {LOCAL_TABLE_PREFIX}/{table_name}/data -type f 2>/dev/null | sort"] + ).strip().split("\n") + assert len(data_files_before) >= 3, ( + f"Expected at least 3 data files (3 inserts), got {data_files_before}" + ) + + drop_iceberg_table(instance, table_name) + + create_iceberg_table( + storage_type, instance, table_name, cluster, "(x Int)", + format_version=2, + if_not_exists=True, + explicit_metadata_path=old_metadata_path, + ) + + pinned_result = instance.query(f"SELECT count() FROM {table_name}") + assert pinned_result.strip() == "2", ( + f"Pinned table should see 2 rows (first two inserts), got {pinned_result.strip()}" + ) + + time.sleep(2) + raw = instance.query( + f"ALTER TABLE {table_name} EXECUTE remove_orphan_files('{OrphanTestEnv.now_ts()}');", + settings=ICEBERG_SETTINGS, + ) + + data_files_after = instance.exec_in_container( + ["bash", "-c", + f"find {LOCAL_TABLE_PREFIX}/{table_name}/data -type f 2>/dev/null | sort"] + ).strip().split("\n") + assert data_files_after == data_files_before, ( + f"remove_orphan_files must not delete data from newer snapshots.\n" + f" Before: {data_files_before}\n" + f" After: {data_files_after}" + ) + + drop_iceberg_table(instance, table_name) + create_iceberg_table( + storage_type, instance, table_name, cluster, "(x Int)", + format_version=2, + if_not_exists=True, + ) + full_result = instance.query( + f"SELECT * FROM {table_name} ORDER BY x" + ) + assert full_result == "1\n2\n3\n", ( + f"All data should be intact when reading latest metadata, got: {full_result}" + ) diff --git a/tests/integration/test_storage_iceberg_with_spark/test_writes_create_version_hint.py b/tests/integration/test_storage_iceberg_with_spark/test_writes_create_version_hint.py index bc785ba08280..cd8cce7eda29 100644 --- a/tests/integration/test_storage_iceberg_with_spark/test_writes_create_version_hint.py +++ b/tests/integration/test_storage_iceberg_with_spark/test_writes_create_version_hint.py @@ -24,9 +24,8 @@ def test_writes_create_version_hint(started_cluster_iceberg_with_spark, format_v f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}/", ) - target_suffix = b'v1.metadata.json' with open(f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}/metadata/version-hint.text", "rb") as f: - assert f.read()[-len(target_suffix):] == target_suffix + assert f.read().strip() == b'1' instance.query(f"INSERT INTO {TABLE_NAME} VALUES ('123', 1);", settings={"allow_insert_into_iceberg": 1}) assert instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY ALL", ) == '123\t1\n' @@ -38,9 +37,8 @@ def test_writes_create_version_hint(started_cluster_iceberg_with_spark, format_v f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}/", ) - target_suffix = b'v2.metadata.json' with open(f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}/metadata/version-hint.text", "rb") as f: - assert f.read()[-len(target_suffix):] == target_suffix + assert f.read().strip() == b'2' df = spark.read.format("iceberg").load(f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}").collect() assert len(df) == 1