From ceb254ebd230c8fbcb89078115a91837617276fa Mon Sep 17 00:00:00 2001 From: Daniil Ivanik <61067749+divanik@users.noreply.github.com> Date: Wed, 19 Mar 2025 22:29:48 +0000 Subject: [PATCH 01/14] Merge pull request #77439 from ClickHouse/divanik/iceberg_time_travel_by_snapshots Iceberg time travel by snapshots --- .../table-engines/integrations/iceberg.md | 176 +++++++++- .../sql-reference/table-functions/iceberg.md | 173 +++++++++- src/Core/Settings.cpp | 6 + src/Core/SettingsChangesHistory.cpp | 2 + .../DataLakes/Iceberg/IcebergMetadata.cpp | 233 +++++++++---- .../DataLakes/Iceberg/IcebergMetadata.h | 36 +- .../DataLakes/Iceberg/SchemaProcessor.cpp | 4 + .../DataLakes/Iceberg/SchemaProcessor.h | 2 + .../DataLakes/Iceberg/Snapshot.h | 1 + .../integration/test_storage_iceberg/test.py | 325 +++++++++++++++++- 10 files changed, 856 insertions(+), 102 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index ffe121665d4b..3d12bf00f29e 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -7,7 +7,7 @@ slug: /engines/table-engines/integrations/iceberg title: 'Iceberg Table Engine' --- -# Iceberg Table Engine +# Iceberg Table Engine {#iceberg-table-engine} :::warning We recommend using the [Iceberg Table Function](/sql-reference/table-functions/iceberg.md) for working with Iceberg data in ClickHouse. The Iceberg Table Function currently provides sufficient functionality, offering a partial read-only interface for Iceberg tables. @@ -37,14 +37,14 @@ CREATE TABLE iceberg_table_local ENGINE = IcebergLocal(path_to_table, [,format] [,compression_method]) ``` -**Engine arguments** +## Engine arguments {#engine-arguments} Description of the arguments coincides with description of arguments in engines `S3`, `AzureBlobStorage`, `HDFS` and `File` correspondingly. `format` stands for the format of data files in the Iceberg table. Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) -**Example** +### Example {#example} ```sql CREATE TABLE iceberg_table ENGINE=IcebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') @@ -69,12 +69,11 @@ CREATE TABLE iceberg_table ENGINE=IcebergS3(iceberg_conf, filename = 'test_table ``` -**Aliases** - +## Aliases {#aliases} Table engine `Iceberg` is an alias to `IcebergS3` now. -**Schema Evolution** +## Schema Evolution {#schema-evolution} At the moment, with the help of CH, you can read iceberg tables, the schema of which has changed over time. We currently support reading tables where columns have been added and removed, and their order has changed. You can also change a column where a value is required to one where NULL is allowed. Additionally, we support permitted type casting for simple types, namely:   * int -> long * float -> double @@ -84,11 +83,172 @@ Currently, it is not possible to change nested structures or the types of elemen To read a table where the schema has changed after its creation with dynamic schema inference, set allow_dynamic_metadata_for_data_lakes = true when creating the table. -**Partition Pruning** +## Partition Pruning {#partition-pruning} ClickHouse supports partition pruning during SELECT queries for Iceberg tables, which helps optimize query performance by skipping irrelevant data files. Now it works with only identity transforms and time-based transforms (hour, day, month, year). To enable partition pruning, set `use_iceberg_partition_pruning = 1`. -### Data cache {#data-cache} + +## Time Travel {#time-travel} + +ClickHouse supports time travel for Iceberg tables, allowing you to query historical data with a specific timestamp or snapshot ID. + +### Basic usage {#basic-usage} + ```sql + SELECT * FROM example_table ORDER BY 1 + SETTINGS iceberg_timestamp_ms = 1714636800000 + ``` + + ```sql + SELECT * FROM example_table ORDER BY 1 + SETTINGS iceberg_snapshot_id = 3547395809148285433 + ``` + +Note: You cannot specify both `iceberg_timestamp_ms` and `iceberg_snapshot_id` parameters in the same query. + +### Important considerations {#important-considerations} + +- **Snapshots** are typically created when: + - New data is written to the table + - Some kind of data compaction is performed + +- **Schema changes typically don't create snapshots** - This leads to important behaviors when using time travel with tables that have undergone schema evolution. + +### Example scenarios {#example-scenarios} + +All scenarios are written in Spark because CH doesn't support writing to Iceberg tables yet. + +#### Scenario 1: Schema Changes Without New Snapshots {#scenario-1} + +Consider this sequence of operations: + + ```sql + -- Create a table with two columns + CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example ( + order_number int, + product_code string + ) + USING iceberg + OPTIONS ('format-version'='2') + +-- Insert data into the table + INSERT INTO spark_catalog.db.time_travel_example VALUES + (1, 'Mars') + + ts1 = now() // A piece of pseudo code + +-- Alter table to add a new column + ALTER TABLE spark_catalog.db.time_travel_example ADD COLUMN (price double) + + ts2 = now() + +-- Insert data into the table + INSERT INTO spark_catalog.db.time_travel_example VALUES (2, 'Venus', 100) + + ts3 = now() + +-- Query the table at each timestamp + SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts1; + ++------------+------------+ +|order_number|product_code| ++------------+------------+ +| 1| Mars| ++------------+------------+ + + + SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts2; + ++------------+------------+ +|order_number|product_code| ++------------+------------+ +| 1| Mars| ++------------+------------+ + + SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts3; + ++------------+------------+-----+ +|order_number|product_code|price| ++------------+------------+-----+ +| 1| Mars| NULL| +| 2| Venus|100.0| ++------------+------------+-----+ +``` + +Query results at different timestamps: + +- At ts1 & ts2: Only the original two columns appear +- At ts3: All three columns appear, with NULL for the price of the first row + +#### Scenario 2: Historical vs. Current Schema Differences {#scenario-2} + + +A time travel query at a current moment might show a different schema than the current table: + + +```sql +-- Create a table + CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example_2 ( + order_number int, + product_code string + ) + USING iceberg + OPTIONS ('format-version'='2') + +-- Insert initial data into the table + INSERT INTO spark_catalog.db.time_travel_example_2 VALUES (2, 'Venus'); + +-- Alter table to add a new column + ALTER TABLE spark_catalog.db.time_travel_example_2 ADD COLUMN (price double); + + ts = now(); + +-- Query the table at a current moment but using timestamp syntax + + SELECT * FROM spark_catalog.db.time_travel_example_2 TIMESTAMP AS OF ts; + + +------------+------------+ + |order_number|product_code| + +------------+------------+ + | 2| Venus| + +------------+------------+ + +-- Query the table at a current moment + SELECT * FROM spark_catalog.db.time_travel_example_2; + + + +------------+------------+-----+ + |order_number|product_code|price| + +------------+------------+-----+ + | 2| Venus| NULL| + +------------+------------+-----+ +``` + +This happens because `ALTER TABLE` doesn't create a new snapshot but for the current table Spark takes value of `schema_id` from the latest metadata file, not a snapshot. + +#### Scenario 3: Historical vs. Current Schema Differences {#scenario-3} + +The second one is that while doing time travel you can't get state of table before any data was written to it: + +```sql +-- Create a table + CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example_3 ( + order_number int, + product_code string + ) + USING iceberg + OPTIONS ('format-version'='2'); + + ts = now(); + +-- Query the table at a specific timestamp + SELECT * FROM spark_catalog.db.time_travel_example_3 TIMESTAMP AS OF ts; -- Finises with error: Cannot find a snapshot older than ts. +``` + + +In Clickhouse the behavior is consistent with Spark. You can mentally replace Spark Select queries with Clickhouse Select queries and it will work the same way. + + +## Data cache {#data-cache} `Iceberg` table engine and table function support data caching same as `S3`, `AzureBlobStorage`, `HDFS` storages. See [here](../../../engines/table-engines/integrations/s3.md#data-cache). diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index 0df71713faae..a4e86981d52b 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -7,7 +7,7 @@ slug: /sql-reference/table-functions/iceberg title: 'iceberg' --- -# iceberg Table Function +# iceberg Table Function {#iceberg-table-function} Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure, HDFS or locally stored. @@ -32,10 +32,10 @@ icebergLocal(named_collection[, option=value [,..]]) Description of the arguments coincides with description of arguments in table functions `s3`, `azureBlobStorage`, `HDFS` and `file` correspondingly. `format` stands for the format of data files in the Iceberg table. -**Returned value** +### Returned value {#returned-value} A table with the specified structure for reading data in the specified Iceberg table. -**Example** +### Example {#example} ```sql SELECT * FROM icebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') @@ -68,7 +68,7 @@ SELECT * FROM icebergS3(iceberg_conf, filename = 'test_table') DESCRIBE icebergS3(iceberg_conf, filename = 'test_table') ``` -**Schema Evolution** +## Schema Evolution {#schema-evolution} At the moment, with the help of CH, you can read iceberg tables, the schema of which has changed over time. We currently support reading tables where columns have been added and removed, and their order has changed. You can also change a column where a value is required to one where NULL is allowed. Additionally, we support permitted type casting for simple types, namely:   * int -> long * float -> double @@ -76,15 +76,174 @@ At the moment, with the help of CH, you can read iceberg tables, the schema of w Currently, it is not possible to change nested structures or the types of elements within arrays and maps. -**Partition Pruning** +## Partition Pruning {#partition-pruning} ClickHouse supports partition pruning during SELECT queries for Iceberg tables, which helps optimize query performance by skipping irrelevant data files. Now it works with only identity transforms and time-based transforms (hour, day, month, year). To enable partition pruning, set `use_iceberg_partition_pruning = 1`. -**Aliases** + +## Time Travel {#time-travel} + +ClickHouse supports time travel for Iceberg tables, allowing you to query historical data with a specific timestamp or snapshot ID. + +### Basic usage {#basic-usage} + ```sql + SELECT * FROM example_table ORDER BY 1 + SETTINGS iceberg_timestamp_ms = 1714636800000 + ``` + + ```sql + SELECT * FROM example_table ORDER BY 1 + SETTINGS iceberg_snapshot_id = 3547395809148285433 + ``` + +Note: You cannot specify both `iceberg_timestamp_ms` and `iceberg_snapshot_id` parameters in the same query. + +### Important considerations {#important-considerations} + +- **Snapshots** are typically created when: + - New data is written to the table + - Some kind of data compaction is performed + +- **Schema changes typically don't create snapshots** - This leads to important behaviors when using time travel with tables that have undergone schema evolution. + +### Example scenarios {#example-scenarios} + +All scenarios are written in Spark because CH doesn't support writing to Iceberg tables yet. + +#### Scenario 1: Schema Changes Without New Snapshots {#scenario-1} + +Consider this sequence of operations: + + ```sql + -- Create a table with two columns + CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example ( + order_number bigint, + product_code string + ) + USING iceberg + OPTIONS ('format-version'='2') + +-- Insert data into the table + INSERT INTO spark_catalog.db.time_travel_example VALUES + (1, 'Mars') + + ts1 = now() // A piece of pseudo code + +-- Alter table to add a new column + ALTER TABLE spark_catalog.db.time_travel_example ADD COLUMN (price double) + + ts2 = now() + +-- Insert data into the table + INSERT INTO spark_catalog.db.time_travel_example VALUES (2, 'Venus', 100) + + ts3 = now() + +-- Query the table at each timestamp + SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts1; + ++------------+------------+ +|order_number|product_code| ++------------+------------+ +| 1| Mars| ++------------+------------+ + + + SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts2; + ++------------+------------+ +|order_number|product_code| ++------------+------------+ +| 1| Mars| ++------------+------------+ + + SELECT * FROM spark_catalog.db.time_travel_example TIMESTAMP AS OF ts3; + ++------------+------------+-----+ +|order_number|product_code|price| ++------------+------------+-----+ +| 1| Mars| NULL| +| 2| Venus|100.0| ++------------+------------+-----+ +``` + +Query results at different timestamps: + +- At ts1 & ts2: Only the original two columns appear +- At ts3: All three columns appear, with NULL for the price of the first row + +#### Scenario 2: Historical vs. Current Schema Differences {#scenario-2} + + +A time travel query at a current moment might show a different schema than the current table: + + +```sql +-- Create a table + CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example_2 ( + order_number bigint, + product_code string + ) + USING iceberg + OPTIONS ('format-version'='2') + +-- Insert initial data into the table + INSERT INTO spark_catalog.db.time_travel_example_2 VALUES (2, 'Venus'); + +-- Alter table to add a new column + ALTER TABLE spark_catalog.db.time_travel_example_2 ADD COLUMN (price double); + + ts = now(); + +-- Query the table at a current moment but using timestamp syntax + + SELECT * FROM spark_catalog.db.time_travel_example_2 TIMESTAMP AS OF ts; + + +------------+------------+ + |order_number|product_code| + +------------+------------+ + | 2| Venus| + +------------+------------+ + +-- Query the table at a current moment + SELECT * FROM spark_catalog.db.time_travel_example_2; + + + +------------+------------+-----+ + |order_number|product_code|price| + +------------+------------+-----+ + | 2| Venus| NULL| + +------------+------------+-----+ +``` + +This happens because `ALTER TABLE` doesn't create a new snapshot but for the current table Spark takes value of `schema_id` from the latest metadata file, not a snapshot. + +#### Scenario 3: Historical vs. Current Schema Differences {#scenario-3} + +The second one is that while doing time travel you can't get state of table before any data was written to it: + +```sql +-- Create a table + CREATE TABLE IF NOT EXISTS spark_catalog.db.time_travel_example_3 ( + order_number bigint, + product_code string + ) + USING iceberg + OPTIONS ('format-version'='2'); + + ts = now(); + +-- Query the table at a specific timestamp + SELECT * FROM spark_catalog.db.time_travel_example_3 TIMESTAMP AS OF ts; -- Finises with error: Cannot find a snapshot older than ts. +``` + +In Clickhouse the behavior is consistent with Spark. You can mentally replace Spark Select queries with Clickhouse Select queries and it will work the same way. + +## Aliases {#aliases} Table function `iceberg` is an alias to `icebergS3` now. -**See Also** +## See Also {#see-also} - [Iceberg engine](/engines/table-engines/integrations/iceberg.md) - [Iceberg cluster table function](/sql-reference/table-functions/icebergCluster.md) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 525ac86e8b94..98c1d5e01f2d 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5705,6 +5705,12 @@ When the ratio of rows containing NULL values to the total number of rows exceed )", 0) \ DECLARE(Int64, prefer_warmed_unmerged_parts_seconds, 0, R"( Only has an effect in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see [cache_populated_by_fetch](merge-tree-settings.md/#cache_populated_by_fetch)), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for Replicated-/SharedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm. +)", 0) \ + DECLARE(Int64, iceberg_timestamp_ms, 0, R"( +Query Iceberg table using the snapshot that was current at a specific timestamp. +)", 0) \ + DECLARE(Int64, iceberg_snapshot_id, 0, R"( +Query Iceberg table using the specific snapshot id. )", 0) \ DECLARE(Bool, allow_deprecated_error_prone_window_functions, false, R"( Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0afb542e0cbf..6cadfdfa7d90 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -82,6 +82,8 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_experimental_database_glue_catalog", false, false, "Allow experimental database engine DataLakeCatalog with catalog_type = 'glue'"}, {"use_page_cache_with_distributed_cache", false, false, "New setting"}, {"use_query_condition_cache", false, false, "New setting."}, + {"iceberg_timestamp_ms", 0, 0, "New setting."}, + {"iceberg_snapshot_id", 0, 0, "New setting."}, {"parallel_replicas_for_cluster_engines", false, true, "New setting."}, /// Release closed. Please use 25.4 }); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index e8afb5462514..6e9d37a67b25 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -1,3 +1,5 @@ +#include +#include "base/types.h" #include "Core/NamesAndTypes.h" #include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h" #include "config.h" @@ -51,12 +53,27 @@ extern const int LOGICAL_ERROR; extern const int ICEBERG_SPECIFICATION_VIOLATION; } +namespace Setting +{ +extern const SettingsInt64 iceberg_timestamp_ms; +extern const SettingsInt64 iceberg_snapshot_id; +} + + using namespace Iceberg; -constexpr const char * COLUMN_SEQ_NUMBER_NAME = "sequence_number"; -constexpr const char * COLUMN_MANIFEST_FILE_PATH_NAME = "manifest_path"; -constexpr const char * FIELD_FORMAT_VERSION_NAME = "format-version"; +constexpr const char * SEQUENCE_NUMBER_COLUMN = "sequence_number"; +constexpr const char * MANIFEST_FILE_PATH_COLUMN = "manifest_path"; +constexpr const char * FORMAT_VERSION_FIELD = "format-version"; +constexpr const char * CURRENT_SNAPSHOT_ID_FIELD_IN_METADATA_FILE = "current-snapshot-id"; +constexpr const char * SNAPSHOT_ID_FIELD_IN_SNAPSHOT = "snapshot-id"; +constexpr const char * MANIFEST_LIST_PATH_FIELD = "manifest-list"; +constexpr const char * SNAPSHOT_LOG_FIELD = "snapshot-log"; +constexpr const char * TIMESTAMP_FIELD_INSIDE_SNAPSHOT = "timestamp-ms"; +constexpr const char * TABLE_LOCATION_FIELD = "location"; +constexpr const char * SNAPSHOTS_FIELD = "snapshots"; + std::pair parseTableSchemaFromManifestFile(const avro::DataFileReaderBase & manifest_file_reader, const String & manifest_file_name) @@ -84,22 +101,19 @@ IcebergMetadata::IcebergMetadata( const DB::ContextPtr & context_, Int32 metadata_version_, Int32 format_version_, - const Poco::JSON::Object::Ptr & object) + const Poco::JSON::Object::Ptr & metadata_object_) : WithContext(context_) , object_storage(std::move(object_storage_)) , configuration(std::move(configuration_)) , schema_processor(IcebergSchemaProcessor()) , log(getLogger("IcebergMetadata")) - , current_metadata_version(metadata_version_) + , last_metadata_version(metadata_version_) + , last_metadata_object(metadata_object_) , format_version(format_version_) - , table_location(object->getValue("location")) + , relevant_snapshot_schema_id(-1) + , table_location(last_metadata_object->getValue(TABLE_LOCATION_FIELD)) { - auto manifest_list_file = getRelevantManifestList(object); - if (manifest_list_file) - { - current_snapshot = getSnapshot(manifest_list_file.value()); - } - current_schema_id = parseTableSchema(object, schema_processor, log); + updateState(context_); } std::pair parseTableSchemaV2Method(const Poco::JSON::Object::Ptr & metadata_object) @@ -145,10 +159,36 @@ std::pair parseTableSchemaV1Method(const Poco::J return {schema, current_schema_id}; } + +void IcebergMetadata::addTableSchemaById(Int32 schema_id) +{ + if (schema_processor.hasClickhouseTableSchemaById(schema_id)) + return; + if (!last_metadata_object->has("schemas")) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema with id `{}`: 'schemas' field is missing in metadata", schema_id); + } + auto schemas = last_metadata_object->get("schemas").extract(); + for (uint32_t i = 0; i != schemas->size(); ++i) + { + auto current_schema = schemas->getObject(i); + if (current_schema->has("schema-id") && current_schema->getValue("schema-id") == schema_id) + { + schema_processor.addIcebergTableSchema(current_schema); + return; + } + } + throw Exception( + ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, + "Cannot parse Iceberg table schema with id `{}`: schema with such id is not found in metadata", + schema_id); +} + Int32 IcebergMetadata::parseTableSchema( const Poco::JSON::Object::Ptr & metadata_object, IcebergSchemaProcessor & schema_processor, LoggerPtr metadata_logger) { - Int32 format_version = metadata_object->getValue(FIELD_FORMAT_VERSION_NAME); + const auto format_version = metadata_object->getValue(FORMAT_VERSION_FIELD); if (format_version == 2) { auto [schema, current_schema_id] = parseTableSchemaV2Method(metadata_object); @@ -277,46 +317,121 @@ bool IcebergMetadata::update(const ContextPtr & local_context) const auto [metadata_version, metadata_file_path] = getLatestOrExplicitMetadataFileAndVersion(object_storage, *configuration_ptr, log.get()); - if (metadata_version == current_metadata_version) - return false; + last_metadata_version = metadata_version; - current_metadata_version = metadata_version; + last_metadata_object = readJSON(metadata_file_path, local_context); - auto metadata_object = readJSON(metadata_file_path, local_context); + chassert(format_version == last_metadata_object->getValue(FORMAT_VERSION_FIELD)); - chassert(format_version == metadata_object->getValue(FIELD_FORMAT_VERSION_NAME)); + auto previous_snapshot_id = relevant_snapshot_id; + auto previous_snapshot_schema_id = relevant_snapshot_schema_id; + updateState(local_context); - auto manifest_list_file = getRelevantManifestList(metadata_object); - if (manifest_list_file - && (!current_snapshot.has_value() || (manifest_list_file.value() != current_snapshot->manifest_list_iterator.getName()))) + if (previous_snapshot_id != relevant_snapshot_id) { - current_snapshot = getSnapshot(manifest_list_file.value()); - cached_unprunned_files_for_current_snapshot = std::nullopt; + cached_unprunned_files_for_last_processed_snapshot = std::nullopt; + return true; } - current_schema_id = parseTableSchema(metadata_object, schema_processor, log); - return true; + return previous_snapshot_schema_id != relevant_snapshot_schema_id; } -std::optional IcebergMetadata::getRelevantManifestList(const Poco::JSON::Object::Ptr & metadata) +void IcebergMetadata::updateSnapshot() { auto configuration_ptr = configuration.lock(); - - auto snapshots = metadata->get("snapshots").extract(); - - auto current_snapshot_id = metadata->getValue("current-snapshot-id"); - + if (!last_metadata_object->has(SNAPSHOTS_FIELD)) + throw Exception( + ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, + "No snapshot set found in metadata for iceberg table `{}`, it is impossible to get manifest list by snapshot id `{}`", + configuration_ptr->getPath(), + relevant_snapshot_id); + auto snapshots = last_metadata_object->get(SNAPSHOTS_FIELD).extract(); for (size_t i = 0; i < snapshots->size(); ++i) { const auto snapshot = snapshots->getObject(static_cast(i)); + if (snapshot->getValue(SNAPSHOT_ID_FIELD_IN_SNAPSHOT) == relevant_snapshot_id) + { + if (!snapshot->has("manifest-list")) + throw Exception( + ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, + "No manifest list found for snapshot id `{}` for iceberg table `{}`", + relevant_snapshot_id, + configuration_ptr->getPath()); + relevant_snapshot = IcebergSnapshot{ + getManifestList(getProperFilePathFromMetadataInfo( + snapshot->getValue(MANIFEST_LIST_PATH_FIELD), configuration_ptr->getPath(), table_location)), + relevant_snapshot_id}; + if (!snapshot->has("schema-id")) + throw Exception( + ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, + "No schema id found for snapshot id `{}` for iceberg table `{}`", + relevant_snapshot_id, + configuration_ptr->getPath()); + relevant_snapshot_schema_id = snapshot->getValue("schema-id"); + addTableSchemaById(relevant_snapshot_schema_id); + return; + } + } + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "No manifest list is found for snapshot id `{}` in metadata for iceberg table `{}`", + relevant_snapshot_id, + configuration_ptr->getPath()); +} - if (snapshot->getValue("snapshot-id") == current_snapshot_id) +void IcebergMetadata::updateState(const ContextPtr & local_context) +{ + auto configuration_ptr = configuration.lock(); + std::optional manifest_list_file; + + bool timestamp_changed = local_context->getSettingsRef()[Setting::iceberg_timestamp_ms].changed; + bool snapshot_id_changed = local_context->getSettingsRef()[Setting::iceberg_snapshot_id].changed; + if (timestamp_changed && snapshot_id_changed) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Time travel with timestamp and snapshot id for iceberg table by path {} cannot be changed simultaneously", + configuration_ptr->getPath()); + } + if (timestamp_changed) + { + Int64 closest_timestamp = 0; + Int64 query_timestamp = local_context->getSettingsRef()[Setting::iceberg_timestamp_ms]; + if (!last_metadata_object->has(SNAPSHOT_LOG_FIELD)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No snapshot log found in metadata for iceberg table {} so it is impossible to get relevant snapshot id using timestamp", configuration_ptr->getPath()); + auto snapshots = last_metadata_object->get(SNAPSHOT_LOG_FIELD).extract(); + relevant_snapshot_id = -1; + for (size_t i = 0; i < snapshots->size(); ++i) + { + const auto snapshot = snapshots->getObject(static_cast(i)); + Int64 snapshot_timestamp = snapshot->getValue(TIMESTAMP_FIELD_INSIDE_SNAPSHOT); + if (snapshot_timestamp <= query_timestamp && snapshot_timestamp > closest_timestamp) + { + closest_timestamp = snapshot_timestamp; + relevant_snapshot_id = snapshot->getValue(SNAPSHOT_ID_FIELD_IN_SNAPSHOT); + } + } + if (relevant_snapshot_id < 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No snapshot found in snapshot log before requested timestamp for iceberg table {}", configuration_ptr->getPath()); + updateSnapshot(); + } + else if (snapshot_id_changed) + { + relevant_snapshot_id = local_context->getSettingsRef()[Setting::iceberg_snapshot_id]; + updateSnapshot(); + } + else + { + if (!last_metadata_object->has(CURRENT_SNAPSHOT_ID_FIELD_IN_METADATA_FILE)) + relevant_snapshot_id = -1; + else + relevant_snapshot_id = last_metadata_object->getValue(CURRENT_SNAPSHOT_ID_FIELD_IN_METADATA_FILE); + if (relevant_snapshot_id != -1) { - const auto path = snapshot->getValue("manifest-list"); - return getProperFilePathFromMetadataInfo(std::string_view(path), configuration_ptr->getPath(), table_location); + updateSnapshot(); } + relevant_snapshot_schema_id = parseTableSchema(last_metadata_object, schema_processor, log); } - return std::nullopt; } std::optional IcebergMetadata::getSchemaVersionByFileIfOutdated(String data_path) const @@ -328,7 +443,7 @@ std::optional IcebergMetadata::getSchemaVersionByFileIfOutdated(String da } const ManifestFileContent & manifest_file = *manifest_file_it->second; auto schema_id = manifest_file.getSchemaId(); - if (schema_id == current_schema_id) + if (schema_id == relevant_snapshot_schema_id) return std::nullopt; return std::optional{schema_id}; } @@ -357,7 +472,7 @@ DataLakeMetadataPtr IcebergMetadata::create( IcebergSchemaProcessor schema_processor; - auto format_version = object->getValue(FIELD_FORMAT_VERSION_NAME); + auto format_version = object->getValue(FORMAT_VERSION_FIELD); auto ptr = std::make_unique(object_storage, configuration_ptr, local_context, metadata_version, format_version, object); @@ -380,34 +495,36 @@ ManifestList IcebergMetadata::initializeManifestList(const String & filename) co auto [name_to_index, name_to_data_type, header] = getColumnsAndTypesFromAvroByNames( manifest_list_file_reader->dataSchema().root(), - {COLUMN_MANIFEST_FILE_PATH_NAME, COLUMN_SEQ_NUMBER_NAME}, + {MANIFEST_FILE_PATH_COLUMN, SEQUENCE_NUMBER_COLUMN}, {avro::Type::AVRO_STRING, avro::Type::AVRO_LONG}); - if (name_to_index.find(COLUMN_MANIFEST_FILE_PATH_NAME) == name_to_index.end()) + if (name_to_index.find(MANIFEST_FILE_PATH_COLUMN) == name_to_index.end()) throw Exception( DB::ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "Required columns are not found in manifest file: {}", - COLUMN_MANIFEST_FILE_PATH_NAME); - if (format_version > 1 && name_to_index.find(COLUMN_SEQ_NUMBER_NAME) == name_to_index.end()) + MANIFEST_FILE_PATH_COLUMN); + if (format_version > 1 && name_to_index.find(SEQUENCE_NUMBER_COLUMN) == name_to_index.end()) throw Exception( - DB::ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "Required columns are not found in manifest file: `{}`", COLUMN_SEQ_NUMBER_NAME); + DB::ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, + "Required columns are not found in manifest file: `{}`", + SEQUENCE_NUMBER_COLUMN); auto columns = parseAvro(*manifest_list_file_reader, header, getFormatSettings(context)); - const auto & manifest_path_col = columns.at(name_to_index.at(COLUMN_MANIFEST_FILE_PATH_NAME)); + const auto & manifest_path_col = columns.at(name_to_index.at(MANIFEST_FILE_PATH_COLUMN)); std::optional sequence_number_column = std::nullopt; if (format_version > 1) { - if (columns.at(name_to_index.at(COLUMN_SEQ_NUMBER_NAME))->getDataType() != TypeIndex::Int64) + if (columns.at(name_to_index.at(SEQUENCE_NUMBER_COLUMN))->getDataType() != TypeIndex::Int64) { throw Exception( DB::ErrorCodes::ILLEGAL_COLUMN, "The parsed column from Avro file of `{}` field should be Int64 type, got `{}`", - COLUMN_SEQ_NUMBER_NAME, - columns.at(name_to_index.at(COLUMN_SEQ_NUMBER_NAME))->getFamilyName()); + SEQUENCE_NUMBER_COLUMN, + columns.at(name_to_index.at(SEQUENCE_NUMBER_COLUMN))->getFamilyName()); } - sequence_number_column = assert_cast(columns.at(name_to_index.at(COLUMN_SEQ_NUMBER_NAME)).get()); + sequence_number_column = assert_cast(columns.at(name_to_index.at(SEQUENCE_NUMBER_COLUMN)).get()); } if (manifest_path_col->getDataType() != TypeIndex::String) @@ -415,7 +532,7 @@ ManifestList IcebergMetadata::initializeManifestList(const String & filename) co throw Exception( ErrorCodes::ILLEGAL_COLUMN, "The parsed column from Avro file of `{}` field should be String type, got `{}`", - COLUMN_MANIFEST_FILE_PATH_NAME, + MANIFEST_FILE_PATH_COLUMN, manifest_path_col->getFamilyName()); } @@ -496,24 +613,18 @@ ManifestListIterator IcebergMetadata::getManifestList(const String & filename) c return ManifestListIterator{manifest_file_iterator}; } - -IcebergSnapshot IcebergMetadata::getSnapshot(const String & filename) const -{ - return IcebergSnapshot{getManifestList(filename)}; -} - Strings IcebergMetadata::getDataFilesImpl(const ActionsDAG * filter_dag) const { - if (!current_snapshot) + if (!relevant_snapshot) return {}; - if (!filter_dag && cached_unprunned_files_for_current_snapshot.has_value()) - return cached_unprunned_files_for_current_snapshot.value(); + if (!filter_dag && cached_unprunned_files_for_last_processed_snapshot.has_value()) + return cached_unprunned_files_for_last_processed_snapshot.value(); Strings data_files; - for (const auto & manifest_list_entry : *(current_snapshot->manifest_list_iterator)) + for (const auto & manifest_list_entry : *(relevant_snapshot->manifest_list_iterator)) { - PartitionPruner pruner(schema_processor, current_schema_id, filter_dag, *manifest_list_entry.manifest_file, getContext()); + PartitionPruner pruner(schema_processor, relevant_snapshot_schema_id, filter_dag, *manifest_list_entry.manifest_file, getContext()); const auto & data_files_in_manifest = manifest_list_entry.manifest_file->getFiles(); for (const auto & manifest_file_entry : data_files_in_manifest) { @@ -534,8 +645,8 @@ Strings IcebergMetadata::getDataFilesImpl(const ActionsDAG * filter_dag) const if (!filter_dag) { - cached_unprunned_files_for_current_snapshot = data_files; - return cached_unprunned_files_for_current_snapshot.value(); + cached_unprunned_files_for_last_processed_snapshot = data_files; + return cached_unprunned_files_for_last_processed_snapshot.value(); } return data_files; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h index c252ad636fa0..6dff1e8301e9 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h @@ -17,6 +17,8 @@ #include "Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h" #include "Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h" +#include + namespace DB { @@ -36,7 +38,7 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext const DB::ContextPtr & context_, Int32 metadata_version_, Int32 format_version_, - const Poco::JSON::Object::Ptr & object); + const Poco::JSON::Object::Ptr & metadata_object); /// Get data files. On first request it reads manifest_list file and iterates through manifest files to find all data files. @@ -45,7 +47,10 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext Strings getDataFiles() const override { return getDataFilesImpl(nullptr); } /// Get table schema parsed from metadata. - NamesAndTypesList getTableSchema() const override { return *schema_processor.getClickhouseTableSchemaById(current_schema_id); } + NamesAndTypesList getTableSchema() const override + { + return *schema_processor.getClickhouseTableSchemaById(relevant_snapshot_schema_id); + } bool operator==(const IDataLakeMetadata & other) const override { @@ -58,7 +63,6 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext const ConfigurationObserverPtr & configuration, const ContextPtr & local_context); - size_t getVersion() const { return current_metadata_version; } std::shared_ptr getInitialSchemaByPath(const String & data_path) const override { @@ -70,7 +74,7 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext { auto version_if_outdated = getSchemaVersionByFileIfOutdated(data_path); return version_if_outdated.has_value() - ? schema_processor.getSchemaTransformationDagByIds(version_if_outdated.value(), current_schema_id) + ? schema_processor.getSchemaTransformationDagByIds(version_if_outdated.value(), relevant_snapshot_schema_id) : nullptr; } @@ -99,21 +103,30 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext mutable Iceberg::ManifestListsStorage manifest_lists_by_name; mutable ManifestEntryByDataFile manifest_file_by_data_file; - Int32 current_metadata_version; + std::tuple getVersion() const { return std::make_tuple(relevant_snapshot_id, relevant_snapshot_schema_id); } + + Int32 last_metadata_version; + Poco::JSON::Object::Ptr last_metadata_object; Int32 format_version; - Int32 current_schema_id; - std::optional current_snapshot; + + + Int32 relevant_snapshot_schema_id; + std::optional relevant_snapshot; + Int64 relevant_snapshot_id{-1}; String table_location; - mutable std::optional cached_unprunned_files_for_current_snapshot; + mutable std::optional cached_unprunned_files_for_last_processed_snapshot; - mutable std::vector positional_delete_files_for_current_query; + void updateState(const ContextPtr & local_context); + + void updateSnapshot(); Iceberg::ManifestList initializeManifestList(const String & filename) const; + mutable std::vector positional_delete_files_for_current_query; - Iceberg::ManifestListIterator getManifestList(const String & filename) const; + void addTableSchemaById(Int32 schema_id); - Iceberg::IcebergSnapshot getSnapshot(const String & filename) const; + Iceberg::ManifestListIterator getManifestList(const String & filename) const; std::optional getSchemaVersionByFileIfOutdated(String data_path) const; @@ -129,7 +142,6 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext std::optional tryGetManifestFile(const String & filename) const; }; - } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.cpp index 5165765ce5ad..961d129f71ca 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.cpp @@ -423,4 +423,8 @@ std::shared_ptr IcebergSchemaProcessor::getClickhouseTableSch return it->second; } +bool IcebergSchemaProcessor::hasClickhouseTableSchemaById(Int32 id) const +{ + return clickhouse_table_schemas_by_ids.find(id) != clickhouse_table_schemas_by_ids.end(); +} } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h index 3fbf2b1a5f9e..de003456cef2 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h @@ -83,6 +83,8 @@ class IcebergSchemaProcessor std::optional tryGetFieldCharacteristics(Int32 schema_version, Int32 source_id) const; NamesAndTypesList tryGetFieldsCharacteristics(Int32 schema_id, const std::vector & source_ids) const; + bool hasClickhouseTableSchemaById(Int32 id) const; + static DataTypePtr getSimpleType(const String & type_name); private: std::unordered_map iceberg_table_schemas_by_ids; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h index ba9fac631385..e6efb1eb7c00 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h @@ -22,6 +22,7 @@ using ManifestListIterator = IteratorWrapper; struct IcebergSnapshot { ManifestListIterator manifest_list_iterator; + Int64 snapshot_id; }; } diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 3da62c4e39b9..ec7a048ae982 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -3187,40 +3187,337 @@ def check_validity_and_get_prunned_files(select_expression): ) @pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) -def test_explicit_metadata_file(started_cluster, storage_type): +def test_schema_evolution_with_time_travel( + started_cluster, format_version, storage_type +): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session TABLE_NAME = ( - "test_explicit_metadata_file_" + "test_schema_evolution_with_time_travel_" + + format_version + + "_" + storage_type + "_" + get_uuid_str() ) - spark.sql( - f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" + def execute_spark_query(query: str): + spark.sql(query) + default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", + ) + return + + execute_spark_query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}; + """ ) - for i in range(50): - spark.sql( - f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)" - ) + execute_spark_query( + f""" + CREATE TABLE IF NOT EXISTS {TABLE_NAME} ( + a int NOT NULL + ) + USING iceberg + OPTIONS ('format-version'='{format_version}') + """ + ) + + table_creation_expression = get_creation_expression( + storage_type, + TABLE_NAME, + started_cluster, + table_function=True, + allow_dynamic_metadata_for_data_lakes=True, + ) + + table_select_expression = table_creation_expression + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"] + ], + [], + ) + + first_timestamp_ms = int(datetime.now().timestamp() * 1000) + + time.sleep(0.5) + + execute_spark_query( + f""" + INSERT INTO {TABLE_NAME} VALUES (4); + """ + ) + + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"], + ], + [["4"]], + ) + + error_message = instance.query_and_get_error(f"SELECT * FROM {table_select_expression} ORDER BY ALL SETTINGS iceberg_timestamp_ms = {first_timestamp_ms}") + assert "No snapshot found in snapshot log before requested timestamp" in error_message + + + second_timestamp_ms = int(datetime.now().timestamp() * 1000) + + time.sleep(0.5) + + execute_spark_query( + f""" + ALTER TABLE {TABLE_NAME} ADD COLUMNS ( + b double + ); + """ + ) + + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"], + ["b", "Nullable(Float64)"] + ], + [["4", "\\N"]], + ) + + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"], + ], + [["4"]], + timestamp_ms=second_timestamp_ms, + ) + + third_timestamp_ms = int(datetime.now().timestamp() * 1000) + + time.sleep(0.5) + + + execute_spark_query( + f""" + INSERT INTO {TABLE_NAME} VALUES (7, 5.0); + """ + ) + + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"], + ["b", "Nullable(Float64)"] + ], + [["4", "\\N"], ["7", "5"]], + ) + + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"], + ], + [["4"]], + timestamp_ms=second_timestamp_ms, + ) + + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"], ], + [["4"]], + timestamp_ms=third_timestamp_ms, + ) + + execute_spark_query( + f""" + ALTER TABLE {TABLE_NAME} ADD COLUMNS ( + c double + ); + """ + ) + + time.sleep(0.5) + fourth_timestamp_ms = int(datetime.now().timestamp() * 1000) + + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"], + ["b", "Nullable(Float64)"] + ], + [["4", "\\N"], ["7", "5"]], + timestamp_ms=fourth_timestamp_ms, + ) + + check_schema_and_data( + instance, + table_select_expression, + [ + ["a", "Int32"], + ["b", "Nullable(Float64)"], + ["c", "Nullable(Float64)"] + ], + [["4", "\\N", "\\N"], ["7", "5", "\\N"]], + ) + +def get_last_snapshot(path_to_table): + import json + import os + + metadata_dir = f"{path_to_table}/metadata/" + last_timestamp = 0 + last_snapshot_id = -1 + for filename in os.listdir(metadata_dir): + if filename.endswith('.json'): + filepath = os.path.join(metadata_dir, filename) + with open(filepath, 'r') as f: + data = json.load(f) + print(data) + timestamp = data.get('last-updated-ms') + if (timestamp > last_timestamp): + last_timestamp = timestamp + last_snapshot_id = data.get('current-snapshot-id') + return last_snapshot_id + + +@pytest.mark.parametrize("format_version", ["1", "2"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) +def test_iceberg_snapshot_reads(started_cluster, format_version, storage_type): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + TABLE_NAME = ( + "test_iceberg_snapshot_reads" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) + + write_iceberg_from_df( + spark, + generate_data(spark, 0, 100), + TABLE_NAME, + mode="overwrite", + format_version=format_version, + ) + default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", + ) + + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + snapshot1_timestamp = datetime.now(timezone.utc) + snapshot1_id = get_last_snapshot(f"/iceberg_data/default/{TABLE_NAME}/") + time.sleep(0.1) + + write_iceberg_from_df( + spark, + generate_data(spark, 100, 200), + TABLE_NAME, + mode="append", + format_version=format_version, + ) default_upload_directory( started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", + "", + ) + snapshot2_timestamp = datetime.now(timezone.utc) + snapshot2_id = get_last_snapshot(f"/iceberg_data/default/{TABLE_NAME}/") + time.sleep(0.1) + + write_iceberg_from_df( + spark, + generate_data(spark, 200, 300), + TABLE_NAME, + mode="append", + format_version=format_version, + ) + default_upload_directory( + started_cluster, + storage_type, f"/iceberg_data/default/{TABLE_NAME}/", + "", + ) + snapshot3_timestamp = datetime.now(timezone.utc) + snapshot3_id = get_last_snapshot(f"/iceberg_data/default/{TABLE_NAME}/") + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 300 + assert instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY 1") == instance.query( + "SELECT number, toString(number + 1) FROM numbers(300)" ) - create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, explicit_metadata_path="") + # Validate that each snapshot timestamp only sees the data inserted by that time. + assert ( + instance.query( + f""" + SELECT * FROM {TABLE_NAME} ORDER BY 1 + SETTINGS iceberg_timestamp_ms = {int(snapshot1_timestamp.timestamp() * 1000)}""" + ) + == instance.query("SELECT number, toString(number + 1) FROM numbers(100)") + ) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 + assert ( + instance.query( + f""" + SELECT * FROM {TABLE_NAME} ORDER BY 1 + SETTINGS iceberg_snapshot_id = {snapshot1_id}""" + ) + == instance.query("SELECT number, toString(number + 1) FROM numbers(100)") + ) - create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, explicit_metadata_path="metadata/v31.metadata.json") - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 300 + assert ( + instance.query( + f""" + SELECT * FROM {TABLE_NAME} ORDER BY 1 + SETTINGS iceberg_timestamp_ms = {int(snapshot2_timestamp.timestamp() * 1000)}""" + ) + == instance.query("SELECT number, toString(number + 1) FROM numbers(200)") + ) - create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, explicit_metadata_path="metadata/v11.metadata.json") + assert ( + instance.query( + f""" + SELECT * FROM {TABLE_NAME} ORDER BY 1 + SETTINGS iceberg_snapshot_id = {snapshot2_id}""" + ) + == instance.query("SELECT number, toString(number + 1) FROM numbers(200)") + ) + + + assert ( + instance.query( + f"""SELECT * FROM {TABLE_NAME} ORDER BY 1 + SETTINGS iceberg_timestamp_ms = {int(snapshot3_timestamp.timestamp() * 1000)}""" + ) + == instance.query("SELECT number, toString(number + 1) FROM numbers(300)") + ) + + assert ( + instance.query( + f""" + SELECT * FROM {TABLE_NAME} ORDER BY 1 + SETTINGS iceberg_snapshot_id = {snapshot3_id}""" + ) + == instance.query("SELECT number, toString(number + 1) FROM numbers(300)") + ) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From 1ac1132e5f9953c3a2b0c5f73cbc658acc26a3c6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 Mar 2025 10:15:09 +0000 Subject: [PATCH 02/14] Merge pull request #77916 from ClickHouse/add_trivial_count Refactor operations with Avro files in Iceberg-related code. --- .../Formats/Impl/AvroRowInputFormat.cpp | 25 +++ .../Formats/Impl/AvroRowInputFormat.h | 1 + .../Iceberg/AvroForIcebergDeserializer.cpp | 88 ++++++++ .../Iceberg/AvroForIcebergDeserializer.h | 57 +++++ .../DataLakes/Iceberg/IcebergMetadata.cpp | 106 ++-------- .../DataLakes/Iceberg/IcebergMetadata.h | 6 +- .../DataLakes/Iceberg/ManifestFile.cpp | 196 +++++------------- .../DataLakes/Iceberg/ManifestFile.h | 48 ++++- .../DataLakes/Iceberg/ManifestFileImpl.h | 66 ------ .../ObjectStorage/DataLakes/Iceberg/Utils.cpp | 88 +------- .../ObjectStorage/DataLakes/Iceberg/Utils.h | 15 +- 11 files changed, 297 insertions(+), 399 deletions(-) create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/AvroForIcebergDeserializer.cpp create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/AvroForIcebergDeserializer.h delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 0119bf95b34a..45dcde422a24 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -940,6 +940,31 @@ AvroDeserializer::Action AvroDeserializer::createAction(const Block & header, co } } + +AvroDeserializer::AvroDeserializer(DataTypePtr data_type, const std::string & column_name, avro::ValidSchema schema, bool allow_missing_fields, bool null_as_default_, const FormatSettings & settings_) + : null_as_default(null_as_default_), settings(settings_) +{ + const auto & schema_root = schema.root(); + if (schema_root->type() != avro::AVRO_RECORD) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Root schema must be a record"); + + Block header; + header.insert({data_type->createColumn(), data_type, column_name}); + + column_found.resize(header.columns()); + row_action = createAction(header, schema_root, column_name); + // fail on missing fields when allow_missing_fields = false + if (!allow_missing_fields) + { + for (size_t i = 0; i < header.columns(); ++i) + { + if (!column_found[i]) + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Field {} not found in Avro schema", header.getByPosition(i).name); + } + } + +} + AvroDeserializer::AvroDeserializer(const Block & header, avro::ValidSchema schema, bool allow_missing_fields, bool null_as_default_, const FormatSettings & settings_) : null_as_default(null_as_default_), settings(settings_) { diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index 88cec2b6d060..6a2a95f1cef3 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -50,6 +50,7 @@ class AvroDeserializer { public: AvroDeserializer(const Block & header, avro::ValidSchema schema, bool allow_missing_fields, bool null_as_default_, const FormatSettings & settings_); + AvroDeserializer(DataTypePtr data_type, const std::string & column_name, avro::ValidSchema schema, bool allow_missing_fields, bool null_as_default_, const FormatSettings & settings_); void deserializeRow(MutableColumns & columns, avro::Decoder & decoder, RowReadExtension & ext) const; using DeserializeFn = std::function; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroForIcebergDeserializer.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroForIcebergDeserializer.cpp new file mode 100644 index 000000000000..ee990f76e44b --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroForIcebergDeserializer.cpp @@ -0,0 +1,88 @@ +#include + +#if USE_AVRO + +#include +#include +#include + +namespace DB::ErrorCodes +{ + extern const int ICEBERG_SPECIFICATION_VIOLATION; +} + +namespace Iceberg +{ + +using namespace DB; + +AvroForIcebergDeserializer::AvroForIcebergDeserializer( + std::unique_ptr buffer_, + const std::string & manifest_file_path_, + const DB::FormatSettings & format_settings) + : buffer(std::move(buffer_)) + , manifest_file_path(manifest_file_path_) +{ + auto manifest_file_reader + = std::make_unique(std::make_unique(*buffer)); + + avro::NodePtr root_node = manifest_file_reader->dataSchema().root(); + auto data_type = AvroSchemaReader::avroNodeToDataType(root_node); + + MutableColumns columns; + columns.push_back(data_type->createColumn()); + AvroDeserializer deserializer(data_type, root_node->name(), manifest_file_reader->dataSchema(), true, true, format_settings); + manifest_file_reader->init(); + RowReadExtension ext; + while (manifest_file_reader->hasMore()) + { + manifest_file_reader->decr(); + deserializer.deserializeRow(columns, manifest_file_reader->decoder(), ext); + } + + metadata = manifest_file_reader->metadata(); + parsed_column = std::move(columns[0]); + parsed_column_data_type = std::dynamic_pointer_cast(data_type); +} + +size_t AvroForIcebergDeserializer::rows() const +{ + return parsed_column->size(); +} + +bool AvroForIcebergDeserializer::hasPath(const std::string & path) const +{ + return parsed_column_data_type->hasSubcolumn(path); +} + +TypeIndex AvroForIcebergDeserializer::getTypeForPath(const std::string & path) const +{ + return WhichDataType(parsed_column_data_type->getSubcolumnType(path)).idx; +} + +Field AvroForIcebergDeserializer::getValueFromRowByName(size_t row_num, const std::string & path, std::optional expected_type) const +{ + auto current_column = parsed_column_data_type->getSubcolumn(path, parsed_column); + auto current_data_type = parsed_column_data_type->getSubcolumnType(path); + + if (expected_type && WhichDataType(current_data_type).idx != *expected_type) + throw Exception(ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, + "Got wrong data type for key {} in manifest file {}, expected {}, got {}", + path, manifest_file_path, *expected_type, WhichDataType(current_data_type).idx); + Field result; + current_column->get(row_num, result); + return result; +} + +std::optional AvroForIcebergDeserializer::tryGetAvroMetadataValue(std::string metadata_key) const +{ + auto it = metadata.find(metadata_key); + if (it == metadata.end()) + return std::nullopt; + + return std::string{it->second.begin(), it->second.end()}; +} + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroForIcebergDeserializer.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroForIcebergDeserializer.h new file mode 100644 index 000000000000..cbef423c14ee --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroForIcebergDeserializer.h @@ -0,0 +1,57 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO + +#include +#include +#include +#include + +#include + +namespace Iceberg +{ + +/// In Iceberg manifest files and manifest lists are store in Avro format: https://avro.apache.org/ +/// This format is some kind of mix between JSON and binary schemaful format like protobuf. +/// It has rich types system, with it's own binary representation and it's really tricky +/// to parse some of them. +/// +/// In ClickHouse we already support avro as input format, so we can parse it. The main complexity +/// comes from the fact that we parse Avro files into nested Tuple column which are really hard +/// to operate in key-value fashion. That is why this class is written on top of our avro parser. +/// It allows to access files in avro files using syntax like "data_file.partition.XXXX" and return +/// Field values back. Also manages avro file metadata which is basically just mapping string -> string. +class AvroForIcebergDeserializer +{ +private: + std::unique_ptr buffer; + std::string manifest_file_path; + DB::ColumnPtr parsed_column; + std::shared_ptr parsed_column_data_type; + + std::map> metadata; +public: + + AvroForIcebergDeserializer( + std::unique_ptr buffer_, + const std::string & manifest_file_path_, + const DB::FormatSettings & format_settings); + + size_t rows() const; + + /// Allow to access avro paths like "a.b.c" + bool hasPath(const std::string & path) const; + DB::TypeIndex getTypeForPath(const std::string & path) const; + /// Allow to access avro paths like "a.b.c". + /// If expected type is provided will throw an exception if types don't match + DB::Field getValueFromRowByName(size_t row_num, const std::string & path, std::optional expected_type = std::nullopt) const; + + std::optional tryGetAvroMetadataValue(std::string metadata_key) const; +}; + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index 6e9d37a67b25..cef983cd123a 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -1,53 +1,46 @@ -#include -#include "base/types.h" -#include "Core/NamesAndTypes.h" -#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h" #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 namespace ProfileEvents { -extern const Event IcebergPartitionPrunnedFiles; + extern const Event IcebergPartitionPrunnedFiles; } + namespace DB { namespace StorageObjectStorageSetting { -extern const StorageObjectStorageSettingsString iceberg_metadata_file_path; + extern const StorageObjectStorageSettingsString iceberg_metadata_file_path; } namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; -extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int ICEBERG_SPECIFICATION_VIOLATION; @@ -76,19 +69,16 @@ constexpr const char * SNAPSHOTS_FIELD = "snapshots"; std::pair -parseTableSchemaFromManifestFile(const avro::DataFileReaderBase & manifest_file_reader, const String & manifest_file_name) +parseTableSchemaFromManifestFile(const AvroForIcebergDeserializer & deserializer, const String & manifest_file_name) { - auto avro_metadata = manifest_file_reader.metadata(); - auto avro_schema_it = avro_metadata.find("schema"); - if (avro_schema_it == avro_metadata.end()) + auto schema_json_string = deserializer.tryGetAvroMetadataValue("schema"); + if (!schema_json_string.has_value()) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Cannot read Iceberg table: manifest file '{}' doesn't have table schema in its metadata", manifest_file_name); - std::vector schema_json = avro_schema_it->second; - String schema_json_string = String(reinterpret_cast(schema_json.data()), schema_json.size()); Poco::JSON::Parser parser; - Poco::Dynamic::Var json = parser.parse(schema_json_string); + Poco::Dynamic::Var json = parser.parse(*schema_json_string); const Poco::JSON::Object::Ptr & schema_object = json.extract(); Int32 schema_object_id = schema_object->getValue("schema-id"); return {schema_object_id, schema_object}; @@ -486,69 +476,20 @@ ManifestList IcebergMetadata::initializeManifestList(const String & filename) co if (configuration_ptr == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration is expired"); - auto context = getContext(); StorageObjectStorage::ObjectInfo object_info(filename); - auto manifest_list_buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, context, log); - - auto manifest_list_file_reader - = std::make_unique(std::make_unique(*manifest_list_buf)); - - auto [name_to_index, name_to_data_type, header] = getColumnsAndTypesFromAvroByNames( - manifest_list_file_reader->dataSchema().root(), - {MANIFEST_FILE_PATH_COLUMN, SEQUENCE_NUMBER_COLUMN}, - {avro::Type::AVRO_STRING, avro::Type::AVRO_LONG}); - - if (name_to_index.find(MANIFEST_FILE_PATH_COLUMN) == name_to_index.end()) - throw Exception( - DB::ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, - "Required columns are not found in manifest file: {}", - MANIFEST_FILE_PATH_COLUMN); - if (format_version > 1 && name_to_index.find(SEQUENCE_NUMBER_COLUMN) == name_to_index.end()) - throw Exception( - DB::ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, - "Required columns are not found in manifest file: `{}`", - SEQUENCE_NUMBER_COLUMN); - - - auto columns = parseAvro(*manifest_list_file_reader, header, getFormatSettings(context)); - const auto & manifest_path_col = columns.at(name_to_index.at(MANIFEST_FILE_PATH_COLUMN)); - - std::optional sequence_number_column = std::nullopt; - if (format_version > 1) - { - if (columns.at(name_to_index.at(SEQUENCE_NUMBER_COLUMN))->getDataType() != TypeIndex::Int64) - { - throw Exception( - DB::ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be Int64 type, got `{}`", - SEQUENCE_NUMBER_COLUMN, - columns.at(name_to_index.at(SEQUENCE_NUMBER_COLUMN))->getFamilyName()); - } - sequence_number_column = assert_cast(columns.at(name_to_index.at(SEQUENCE_NUMBER_COLUMN)).get()); - } + auto manifest_list_buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, getContext(), log); + AvroForIcebergDeserializer manifest_list_deserializer(std::move(manifest_list_buf), filename, getFormatSettings(getContext())); - if (manifest_path_col->getDataType() != TypeIndex::String) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be String type, got `{}`", - MANIFEST_FILE_PATH_COLUMN, - manifest_path_col->getFamilyName()); - } - - const auto * manifest_path_col_str = typeid_cast(manifest_path_col.get()); ManifestList manifest_list; - - for (size_t i = 0; i < manifest_path_col_str->size(); ++i) + for (size_t i = 0; i < manifest_list_deserializer.rows(); ++i) { - const std::string_view file_path = manifest_path_col_str->getDataAt(i).toView(); + const std::string file_path = manifest_list_deserializer.getValueFromRowByName(i, MANIFEST_FILE_PATH_COLUMN, TypeIndex::String).safeGet(); const auto manifest_file_name = getProperFilePathFromMetadataInfo(file_path, configuration_ptr->getPath(), table_location); Int64 added_sequence_number = 0; if (format_version > 1) - { - added_sequence_number = sequence_number_column.value()->getInt(i); - } + added_sequence_number = manifest_list_deserializer.getValueFromRowByName(i, SEQUENCE_NUMBER_COLUMN, TypeIndex::Int64).safeGet(); + /// We can't encapsulate this logic in getManifestFile because we need not only the name of the file, but also an inherited sequence number which is known only during the parsing of ManifestList auto manifest_file_content = initializeManifestFile(manifest_file_name, added_sequence_number); auto [iterator, _inserted] = manifest_files_by_name.emplace(manifest_file_name, std::move(manifest_file_content)); @@ -570,21 +511,18 @@ ManifestFileContent IcebergMetadata::initializeManifestFile(const String & filen ObjectInfo manifest_object_info(filename); auto buffer = StorageObjectStorageSource::createReadBuffer(manifest_object_info, object_storage, getContext(), log); - auto manifest_file_reader = std::make_unique(std::make_unique(*buffer)); - auto [schema_id, schema_object] = parseTableSchemaFromManifestFile(*manifest_file_reader, filename); + AvroForIcebergDeserializer manifest_file_deserializer(std::move(buffer), filename, getFormatSettings(getContext())); + auto [schema_id, schema_object] = parseTableSchemaFromManifestFile(manifest_file_deserializer, filename); schema_processor.addIcebergTableSchema(schema_object); - auto manifest_file_impl = std::make_unique( - std::move(manifest_file_reader), + return ManifestFileContent( + manifest_file_deserializer, format_version, configuration_ptr->getPath(), - getFormatSettings(getContext()), schema_id, schema_processor, inherited_sequence_number, table_location, getContext()); - - return ManifestFileContent(std::move(manifest_file_impl)); } ManifestFileIterator IcebergMetadata::getManifestFile(const String & filename) const diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h index 6dff1e8301e9..f853c85d9b5c 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h @@ -13,9 +13,9 @@ #include #include -#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h" -#include "Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h" -#include "Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h" +#include +#include +#include #include diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp index 46953627c92c..19e6cbb49f0d 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp @@ -2,25 +2,21 @@ #if USE_AVRO -#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h" -#include "Storages/ObjectStorage/DataLakes/Iceberg/Utils.h" +#include +#include +#include -#include -#include -#include #include -#include "DataTypes/DataTypeTuple.h" #include #include -#include #include +#include namespace DB::ErrorCodes { -extern const int ILLEGAL_COLUMN; -extern const int UNSUPPORTED_METHOD; -extern const int ICEBERG_SPECIFICATION_VIOLATION; -extern const int LOGICAL_ERROR; + extern const int UNSUPPORTED_METHOD; + extern const int ICEBERG_SPECIFICATION_VIOLATION; + extern const int LOGICAL_ERROR; } namespace Iceberg @@ -30,34 +26,27 @@ constexpr const char * COLUMN_STATUS_NAME = "status"; constexpr const char * COLUMN_TUPLE_DATA_FILE_NAME = "data_file"; constexpr const char * COLUMN_SEQ_NUMBER_NAME = "sequence_number"; -constexpr const char * SUBCOLUMN_FILE_PATH_NAME = "file_path"; -constexpr const char * SUBCOLUMN_CONTENT_NAME = "content"; -constexpr const char * SUBCOLUMN_PARTITION_NAME = "partition"; +constexpr const char * SUBCOLUMN_FILE_PATH_NAME = "data_file.file_path"; +constexpr const char * SUBCOLUMN_CONTENT_NAME = "data_file.content"; +constexpr const char * SUBCOLUMN_PARTITION_NAME = "data_file.partition"; const std::vector & ManifestFileContent::getFiles() const { - return impl->files; + return files; } Int32 ManifestFileContent::getSchemaId() const { - return impl->schema_id; -} - - -ManifestFileContent::ManifestFileContent(std::unique_ptr impl_) : impl(std::move(impl_)) -{ + return schema_id; } using namespace DB; - -ManifestFileContentImpl::ManifestFileContentImpl( - std::unique_ptr manifest_file_reader_, +ManifestFileContent::ManifestFileContent( + const AvroForIcebergDeserializer & manifest_file_deserializer, Int32 format_version_, const String & common_path, - const DB::FormatSettings & format_settings, Int32 schema_id_, const IcebergSchemaProcessor & schema_processor, Int64 inherited_sequence_number, @@ -66,101 +55,26 @@ ManifestFileContentImpl::ManifestFileContentImpl( { this->schema_id = schema_id_; - avro::NodePtr root_node = manifest_file_reader_->dataSchema().root(); - - auto [name_to_index, name_to_data_type, manifest_file_header] = getColumnsAndTypesFromAvroByNames( - root_node, - {COLUMN_STATUS_NAME, COLUMN_TUPLE_DATA_FILE_NAME, COLUMN_SEQ_NUMBER_NAME}, - {avro::Type::AVRO_INT, avro::Type::AVRO_RECORD, avro::Type::AVRO_UNION}); - for (const auto & column_name : {COLUMN_STATUS_NAME, COLUMN_TUPLE_DATA_FILE_NAME}) { - if (name_to_index.find(column_name) == name_to_index.end()) + if (!manifest_file_deserializer.hasPath(column_name)) throw Exception( DB::ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "Required columns are not found in manifest file: {}", column_name); } - if (format_version_ > 1 && name_to_index.find(COLUMN_SEQ_NUMBER_NAME) == name_to_index.end()) + if (format_version_ > 1 && !manifest_file_deserializer.hasPath(COLUMN_SEQ_NUMBER_NAME)) throw Exception( ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "Required columns are not found in manifest file: {}", COLUMN_SEQ_NUMBER_NAME); - auto columns = parseAvro(*manifest_file_reader_, manifest_file_header, format_settings); - if (columns.at(name_to_index.at(COLUMN_STATUS_NAME))->getDataType() != TypeIndex::Int32) - { - throw Exception( - DB::ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be Int32 type, got {}", - COLUMN_STATUS_NAME, - columns.at(name_to_index.at(COLUMN_STATUS_NAME))->getFamilyName()); - } - if (columns.at(name_to_index.at(COLUMN_TUPLE_DATA_FILE_NAME))->getDataType() != TypeIndex::Tuple) - { - throw Exception( - DB::ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be Tuple type, got {}", - COLUMN_TUPLE_DATA_FILE_NAME, - magic_enum::enum_name(columns.at(name_to_index.at(COLUMN_TUPLE_DATA_FILE_NAME))->getDataType())); - } - - const auto * status_int_column = assert_cast(columns.at(name_to_index.at(COLUMN_STATUS_NAME)).get()); - - const auto & data_file_tuple_type = assert_cast(*name_to_data_type.at(COLUMN_TUPLE_DATA_FILE_NAME).get()); - const auto * data_file_tuple_column = assert_cast(columns.at(name_to_index.at(COLUMN_TUPLE_DATA_FILE_NAME)).get()); - - ColumnPtr file_path_column = data_file_tuple_column->getColumnPtr(data_file_tuple_type.getPositionByName(SUBCOLUMN_FILE_PATH_NAME)); - - if (file_path_column->getDataType() != TypeIndex::String) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be String type, got {}", - SUBCOLUMN_FILE_PATH_NAME, - magic_enum::enum_name(file_path_column->getDataType())); - } - - const auto * file_path_string_column = assert_cast(file_path_column.get()); - - ColumnPtr content_column; - const ColumnInt32 * content_int_column = nullptr; - if (format_version_ > 1) - { - content_column = data_file_tuple_column->getColumnPtr(data_file_tuple_type.getPositionByName(SUBCOLUMN_CONTENT_NAME)); - if (content_column->getDataType() != TypeIndex::Int32) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be Int type, got {}", - SUBCOLUMN_CONTENT_NAME, - magic_enum::enum_name(content_column->getDataType())); - } - - content_int_column = assert_cast(content_column.get()); - } - - Poco::JSON::Parser parser; - ColumnPtr big_partition_column = data_file_tuple_column->getColumnPtr(data_file_tuple_type.getPositionByName(SUBCOLUMN_PARTITION_NAME)); - if (big_partition_column->getDataType() != TypeIndex::Tuple) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be Tuple type, got {}", - SUBCOLUMN_PARTITION_NAME, - magic_enum::enum_name(big_partition_column->getDataType())); - } - const auto * big_partition_tuple = assert_cast(big_partition_column.get()); + auto partition_spec_json_string = manifest_file_deserializer.tryGetAvroMetadataValue("partition-spec"); + if (!partition_spec_json_string.has_value()) + throw Exception(ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "No partition-spec in iceberg manifest file"); - auto avro_metadata = manifest_file_reader_->metadata(); - - std::vector partition_spec_json_bytes = avro_metadata["partition-spec"]; - String partition_spec_json_string - = String(reinterpret_cast(partition_spec_json_bytes.data()), partition_spec_json_bytes.size()); - - Poco::Dynamic::Var partition_spec_json = parser.parse(partition_spec_json_string); + Poco::Dynamic::Var partition_spec_json = parser.parse(*partition_spec_json_string); const Poco::JSON::Array::Ptr & partition_specification = partition_spec_json.extract(); - std::vector partition_columns; DB::NamesAndTypesList partition_columns_description; std::shared_ptr partition_key_ast = std::make_shared(); partition_key_ast->name = "tuple"; @@ -183,56 +97,45 @@ ManifestFileContentImpl::ManifestFileContentImpl( partition_key_ast->arguments->children.emplace_back(std::move(partition_ast)); partition_columns_description.emplace_back(numeric_column_name, removeNullable(manifest_file_column_characteristics.type)); - partition_columns.push_back(removeNullable(big_partition_tuple->getColumnPtr(i))); this->partition_column_ids.push_back(source_id); } if (!partition_column_ids.empty()) this->partition_key_description.emplace(DB::KeyDescription::getKeyFromAST(std::move(partition_key_ast), ColumnsDescription(partition_columns_description), context)); - std::optional sequence_number_column = std::nullopt; - if (format_version_ > 1) - { - if (columns.at(name_to_index.at(COLUMN_SEQ_NUMBER_NAME))->getDataType() != TypeIndex::Nullable) - { - throw Exception( - DB::ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be Nullable type, got {}", - COLUMN_SEQ_NUMBER_NAME, - magic_enum::enum_name(columns.at(name_to_index.at(COLUMN_SEQ_NUMBER_NAME))->getDataType())); - } - sequence_number_column = assert_cast(columns.at(name_to_index.at(COLUMN_SEQ_NUMBER_NAME)).get()); - if (sequence_number_column.value()->getNestedColumnPtr()->getDataType() != TypeIndex::Int64) - { - throw Exception( - DB::ErrorCodes::ILLEGAL_COLUMN, - "The parsed column from Avro file of `{}` field should be Int64 type, got {}", - COLUMN_SEQ_NUMBER_NAME, - magic_enum::enum_name(sequence_number_column.value()->getNestedColumnPtr()->getDataType())); - } - } - - for (size_t i = 0; i < data_file_tuple_column->size(); ++i) + for (size_t i = 0; i < manifest_file_deserializer.rows(); ++i) { FileContentType content_type = FileContentType::DATA; if (format_version_ > 1) { - content_type = FileContentType(content_int_column->getElement(i)); + content_type = FileContentType(manifest_file_deserializer.getValueFromRowByName(i, SUBCOLUMN_CONTENT_NAME, TypeIndex::Int32).safeGet()); if (content_type != FileContentType::DATA) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: positional and equality deletes are not supported"); } - const auto status = ManifestEntryStatus(status_int_column->getInt(i)); - - const auto file_path = getProperFilePathFromMetadataInfo(file_path_string_column->getDataAt(i).toView(), common_path, table_location); + const auto status = ManifestEntryStatus(manifest_file_deserializer.getValueFromRowByName(i, COLUMN_STATUS_NAME, TypeIndex::Int32).safeGet()); + + const auto file_path = getProperFilePathFromMetadataInfo(manifest_file_deserializer.getValueFromRowByName(i, SUBCOLUMN_FILE_PATH_NAME, TypeIndex::String).safeGet(), common_path, table_location); + + /// NOTE: This is weird, because in manifest file partition looks like this: + /// { + /// ... + /// "data_file": { + /// "partition": { + /// "total_amount_trunc": { + /// "decimal_10_2": "\u0000\u0000\u0000\u0013" + /// } + /// }, + /// .... + /// However, somehow parser ignores all these nested keys like "total_amount_trunc" or "decimal_10_2" and + /// directly returns tuple of partition values. However it's exactly what we need. + Field partition_value = manifest_file_deserializer.getValueFromRowByName(i, SUBCOLUMN_PARTITION_NAME); + auto tuple = partition_value.safeGet(); DB::Row partition_key_value; - for (const auto & partition_column : partition_columns) - { - Field partition_value; - partition_column->get(i, partition_value); - partition_key_value.emplace_back(partition_value); - } + for (const auto & value : tuple) + partition_key_value.emplace_back(value); + FileEntry file = FileEntry{DataFileEntry{file_path}}; Int64 added_sequence_number = 0; @@ -244,13 +147,16 @@ ManifestFileContentImpl::ManifestFileContentImpl( added_sequence_number = inherited_sequence_number; break; case ManifestEntryStatus::EXISTING: - if (sequence_number_column.value()->isNullAt(i)) + { + auto value = manifest_file_deserializer.getValueFromRowByName(i, COLUMN_SEQ_NUMBER_NAME); + if (value.isNull()) throw Exception( DB::ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "Data sequence number is null for the file added in another snapshot"); else - added_sequence_number = sequence_number_column.value()->getInt(i); + added_sequence_number = value.safeGet(); break; + } case ManifestEntryStatus::DELETED: added_sequence_number = inherited_sequence_number; break; @@ -262,19 +168,19 @@ ManifestFileContentImpl::ManifestFileContentImpl( bool ManifestFileContent::hasPartitionKey() const { - return !impl->partition_column_ids.empty(); + return !partition_column_ids.empty(); } const DB::KeyDescription & ManifestFileContent::getPartitionKeyDescription() const { if (!hasPartitionKey()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table has no partition key, but it was requested"); - return *(impl->partition_key_description); + return *(partition_key_description); } const std::vector & ManifestFileContent::getPartitionKeyColumnIDs() const { - return impl->partition_column_ids; + return partition_column_ids; } } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h index 97a3db58518a..1c3b6035a24e 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h @@ -5,6 +5,8 @@ #if USE_AVRO #include +#include +#include #include #include @@ -47,10 +49,44 @@ struct ManifestFileEntry DB::Row partition_key_value; }; +/** + * Manifest file has the following format: '/iceberg_data/db/table_name/metadata/c87bfec7-d36c-4075-ad04-600b6b0f2020-m0.avro' + * + * `manifest file` is different in format version V1 and V2 and has the following contents: + * v1 v2 + * status req req + * snapshot_id req opt + * sequence_number opt + * file_sequence_number opt + * data_file req req + * Example format version V1: + * ┌─status─┬─────────snapshot_id─┬─data_file───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ + * │ 1 │ 2819310504515118887 │ ('/iceberg_data/db/table_name/data/00000-1-3edca534-15a0-4f74-8a28-4733e0bf1270-00001.parquet','PARQUET',(),100,1070,67108864,[(1,233),(2,210)],[(1,100),(2,100)],[(1,0),(2,0)],[],[(1,'\0'),(2,'0')],[(1,'c'),(2,'99')],NULL,[4],0) │ + * └────────┴─────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + * Example format version V2: + * ┌─status─┬─────────snapshot_id─┬─sequence_number─┬─file_sequence_number─┬─data_file───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ + * │ 1 │ 5887006101709926452 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ (0,'/iceberg_data/db/table_name/data/00000-1-c8045c90-8799-4eac-b957-79a0484e223c-00001.parquet','PARQUET',(),100,1070,[(1,233),(2,210)],[(1,100),(2,100)],[(1,0),(2,0)],[],[(1,'\0'),(2,'0')],[(1,'c'),(2,'99')],NULL,[4],[],0) │ + * └────────┴─────────────────────┴─────────────────┴──────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + * In case of partitioned data we'll have extra directory partition=value: + * ─status─┬─────────snapshot_id─┬─data_file──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ + * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=0/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00001.parquet','PARQUET',(0),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0\0'),(2,'1')],[(1,'\0\0\0\0\0\0\0\0'),(2,'1')],NULL,[4],0) │ + * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=1/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00002.parquet','PARQUET',(1),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'2')],[(1,'\0\0\0\0\0\0\0'),(2,'2')],NULL,[4],0) │ + * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=2/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00003.parquet','PARQUET',(2),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'3')],[(1,'\0\0\0\0\0\0\0'),(2,'3')],NULL,[4],0) │ + * └────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + */ + class ManifestFileContent { public: - explicit ManifestFileContent(std::unique_ptr impl_); + explicit ManifestFileContent( + const AvroForIcebergDeserializer & manifest_file_deserializer, + Int32 format_version_, + const String & common_path, + Int32 schema_id_, + const DB::IcebergSchemaProcessor & schema_processor, + Int64 inherited_sequence_number, + const std::string & table_location, + DB::ContextPtr context); const std::vector & getFiles() const; Int32 getSchemaId() const; @@ -59,9 +95,15 @@ class ManifestFileContent const DB::KeyDescription & getPartitionKeyDescription() const; const std::vector & getPartitionKeyColumnIDs() const; private: - std::unique_ptr impl; -}; + Int32 schema_id; + + std::optional partition_key_description; + std::vector partition_column_ids; + // Size - number of files + std::vector files; + +}; using ManifestFilesStorage = std::map; using ManifestFileIterator = IteratorWrapper; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h deleted file mode 100644 index f96ea285483e..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFileImpl.h +++ /dev/null @@ -1,66 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AVRO - -#include -#include -#include -#include - -#include "SchemaProcessor.h" - -namespace Iceberg -{ - -/** - * Manifest file has the following format: '/iceberg_data/db/table_name/metadata/c87bfec7-d36c-4075-ad04-600b6b0f2020-m0.avro' - * - * `manifest file` is different in format version V1 and V2 and has the following contents: - * v1 v2 - * status req req - * snapshot_id req opt - * sequence_number opt - * file_sequence_number opt - * data_file req req - * Example format version V1: - * ┌─status─┬─────────snapshot_id─┬─data_file───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ - * │ 1 │ 2819310504515118887 │ ('/iceberg_data/db/table_name/data/00000-1-3edca534-15a0-4f74-8a28-4733e0bf1270-00001.parquet','PARQUET',(),100,1070,67108864,[(1,233),(2,210)],[(1,100),(2,100)],[(1,0),(2,0)],[],[(1,'\0'),(2,'0')],[(1,'c'),(2,'99')],NULL,[4],0) │ - * └────────┴─────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - * Example format version V2: - * ┌─status─┬─────────snapshot_id─┬─sequence_number─┬─file_sequence_number─┬─data_file───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ - * │ 1 │ 5887006101709926452 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ (0,'/iceberg_data/db/table_name/data/00000-1-c8045c90-8799-4eac-b957-79a0484e223c-00001.parquet','PARQUET',(),100,1070,[(1,233),(2,210)],[(1,100),(2,100)],[(1,0),(2,0)],[],[(1,'\0'),(2,'0')],[(1,'c'),(2,'99')],NULL,[4],[],0) │ - * └────────┴─────────────────────┴─────────────────┴──────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - * In case of partitioned data we'll have extra directory partition=value: - * ─status─┬─────────snapshot_id─┬─data_file──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ - * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=0/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00001.parquet','PARQUET',(0),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0\0'),(2,'1')],[(1,'\0\0\0\0\0\0\0\0'),(2,'1')],NULL,[4],0) │ - * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=1/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00002.parquet','PARQUET',(1),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'2')],[(1,'\0\0\0\0\0\0\0'),(2,'2')],NULL,[4],0) │ - * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=2/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00003.parquet','PARQUET',(2),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'3')],[(1,'\0\0\0\0\0\0\0'),(2,'3')],NULL,[4],0) │ - * └────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - */ -struct ManifestFileContentImpl -{ -public: - explicit ManifestFileContentImpl( - std::unique_ptr manifest_file_reader_, - Int32 format_version_, - const String & common_path, - const DB::FormatSettings & format_settings, - Int32 schema_id_, - const DB::IcebergSchemaProcessor & schema_processor, - Int64 inherited_sequence_number, - const std::string & table_location, - DB::ContextPtr context); - - Int32 schema_id; - - std::optional partition_key_description; - std::vector partition_column_ids; - // Size - number of files - std::vector files; -}; - -} - -#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp index 375af274b52a..56d7b9565fb1 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp @@ -15,9 +15,9 @@ using namespace DB; namespace DB::ErrorCodes { -extern const int ICEBERG_SPECIFICATION_VIOLATION; -extern const int BAD_TYPE_OF_FIELD; + extern const int BAD_ARGUMENTS; + } namespace Iceberg @@ -25,31 +25,6 @@ namespace Iceberg using namespace DB; - -MutableColumns parseAvro(avro::DataFileReaderBase & file_reader, const Block & header, const FormatSettings & settings) -{ - auto deserializer = std::make_unique(header, file_reader.dataSchema(), true, true, settings); - MutableColumns columns = header.cloneEmptyColumns(); - - file_reader.init(); - RowReadExtension ext; - while (file_reader.hasMore()) - { - file_reader.decr(); - deserializer->deserializeRow(columns, file_reader.decoder(), ext); - } - - for (size_t i = 0; i < columns.size(); ++i) - { - if (columns[0]->size() != columns[i]->size()) - { - throw Exception(DB::ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, "All columns should have the same size"); - } - } - return columns; -} - - // 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 "/". @@ -113,65 +88,6 @@ std::string getProperFilePathFromMetadataInfo(std::string_view data_path, std::s } } -std::tuple getColumnsAndTypesFromAvroByNames( - avro::NodePtr root_node, const std::vector & names, const std::vector & expected_types) -{ - NameToIndex name_to_index; - NameToDataType name_to_data_type; - - std::unordered_map> initial_index_by_name; - for (const auto & name : names) - { - initial_index_by_name.insert({name, std::nullopt}); - } - - size_t leaves_num = root_node->leaves(); - for (size_t i = 0; i < leaves_num; ++i) - { - const auto & name = root_node->nameAt(static_cast(i)); - - if (initial_index_by_name.find(name) != initial_index_by_name.end()) - initial_index_by_name[name] = i; - } - - - size_t current_new_index = 0; - ColumnsWithTypeAndName columns_to_add = {}; - for (size_t i = 0; i < names.size(); ++i) - { - const auto & name = names[i]; - if (initial_index_by_name.at(name).has_value()) - { - name_to_index.insert({name, current_new_index++}); - const auto node = root_node->leafAt(static_cast(initial_index_by_name.at(name).value())); - const size_t initial_index = initial_index_by_name.at(name).value(); - if (node->type() != expected_types.at(i)) - { - throw Exception( - ErrorCodes::BAD_TYPE_OF_FIELD, - "The parsed column from Avro file of `{}` field should be {} type, got {}", - name, - magic_enum::enum_name(expected_types[initial_index]), - magic_enum::enum_name(node->type())); - } - name_to_data_type.insert({name, AvroSchemaReader::avroNodeToDataType(node)}); - columns_to_add.push_back(ColumnWithTypeAndName{name_to_data_type.at(name)->createColumn(), name_to_data_type.at(name), name}); - } - } - - return std::make_tuple(name_to_index, name_to_data_type, Block{columns_to_add}); -} - -void checkColumnType(const DB::ColumnPtr & column, DB::TypeIndex expected_type_index) -{ - if (column->getDataType() != expected_type_index) - throw Exception( - ErrorCodes::BAD_TYPE_OF_FIELD, - "The parsed column from Avro file should be {} type, got {}", - magic_enum::enum_name(expected_type_index), - column->getFamilyName()); -} - } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h index 3166a7fb6ab5..432751be8832 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h @@ -2,25 +2,16 @@ #include "config.h" -#if USE_AVRO +#include +#include -#include +#if USE_AVRO namespace Iceberg { -using NameToIndex = std::unordered_map; -using NameToDataType = std::unordered_map; - -DB::MutableColumns parseAvro(avro::DataFileReaderBase & file_reader, const DB::Block & header, const DB::FormatSettings & settings); - - std::string getProperFilePathFromMetadataInfo(std::string_view data_path, std::string_view common_path, std::string_view table_location); -std::tuple getColumnsAndTypesFromAvroByNames( - avro::NodePtr root_node, const std::vector & names, const std::vector & expected_types); } -void checkColumnType(const DB::ColumnPtr & column, DB::TypeIndex expected_type_index); - #endif From bead76bf8b60a08c9edaecdc2ae4fa08053d1f42 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Mar 2025 15:51:34 +0000 Subject: [PATCH 03/14] Merge pull request #78090 from ClickHouse/trivial_count_for_iceberg Trivial count optimization for iceberg --- src/Common/ProfileEvents.cpp | 1 + src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../ServerAsynchronousMetrics.cpp | 6 +- src/Planner/PlannerJoinTree.cpp | 2 +- .../QueryPlan/Optimizations/optimizeJoin.cpp | 2 +- src/Storages/IStorage.h | 4 +- .../DataLakes/DataLakeConfiguration.h | 10 +- .../DataLakes/IDataLakeMetadata.h | 2 + .../DataLakes/Iceberg/IcebergMetadata.cpp | 81 +++++++++++++++- .../DataLakes/Iceberg/IcebergMetadata.h | 4 +- .../DataLakes/Iceberg/ManifestFile.cpp | 97 ++++++++++++++++++- .../DataLakes/Iceberg/ManifestFile.h | 15 +++ .../DataLakes/Iceberg/Snapshot.h | 2 + .../ObjectStorage/StorageObjectStorage.cpp | 13 +++ .../ObjectStorage/StorageObjectStorage.h | 5 + .../RocksDB/StorageEmbeddedRocksDB.cpp | 6 +- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 4 +- src/Storages/StorageBuffer.cpp | 6 +- src/Storages/StorageBuffer.h | 4 +- src/Storages/StorageDistributed.cpp | 8 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageJoin.cpp | 6 +- src/Storages/StorageJoin.h | 4 +- src/Storages/StorageLog.cpp | 4 +- src/Storages/StorageLog.h | 4 +- src/Storages/StorageMaterializedView.cpp | 8 +- src/Storages/StorageMaterializedView.h | 4 +- src/Storages/StorageMemory.cpp | 4 +- src/Storages/StorageMemory.h | 4 +- src/Storages/StorageMerge.cpp | 8 +- src/Storages/StorageMerge.h | 4 +- src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageMergeTree.h | 4 +- src/Storages/StorageNull.h | 4 +- src/Storages/StorageProxy.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.h | 4 +- src/Storages/StorageSet.cpp | 4 +- src/Storages/StorageSet.h | 4 +- src/Storages/StorageStripeLog.cpp | 4 +- src/Storages/StorageStripeLog.h | 4 +- src/Storages/StorageTimeSeries.cpp | 8 +- src/Storages/StorageTimeSeries.h | 4 +- src/Storages/System/StorageSystemTables.cpp | 18 ++-- 45 files changed, 312 insertions(+), 88 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 22c818fac55b..4e7622232cf4 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -214,6 +214,7 @@ M(ExternalJoinUncompressedBytes, "Amount of data (uncompressed, before compression) written for JOIN in external memory.", ValueType::Bytes) \ \ M(IcebergPartitionPrunnedFiles, "Number of skipped files during Iceberg partition pruning", ValueType::Number) \ + M(IcebergTrivialCountOptimizationApplied, "Trivial count optimization applied while reading from Iceberg", ValueType::Number) \ M(JoinBuildTableRowCount, "Total number of rows in the build table for a JOIN operation.", ValueType::Number) \ M(JoinProbeTableRowCount, "Total number of rows in the probe table for a JOIN operation.", ValueType::Number) \ M(JoinResultRowCount, "Total number of rows in the result of a JOIN operation.", ValueType::Number) \ diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index a38ded75f83f..975ddcd18bf6 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -150,7 +150,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ContextPtr & context_, AS const auto & settings = getContext()->getSettingsRef(); if (query.if_empty) { - if (auto rows = table->totalRows(settings); rows > 0) + if (auto rows = table->totalRows(getContext()); rows > 0) throw Exception(ErrorCodes::TABLE_NOT_EMPTY, "Table {} is not empty", backQuoteIfNeed(table_id.table_name)); } checkStorageSupportsTransactionsIfNeeded(table, context_); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 32f05f89a9d6..9c05a168352c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2463,7 +2463,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 allow_exper /// require reading some data (but much faster than reading columns). /// Set a special flag in query info so the storage will see it and optimize count in read() method. query_info.optimize_trivial_count = optimize_trivial_count; - return storage->totalRows(settings); + return storage->totalRows(context); } // It's possible to optimize count() given only partition predicates diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 17294be81f37..6a92e7f6fb6c 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -327,12 +327,10 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr if (MergeTreeData * table_merge_tree = dynamic_cast(table.get())) { - const auto & settings = getContext()->getSettingsRef(); - calculateMax(max_part_count_for_partition, table_merge_tree->getMaxPartsCountAndSizeForPartition().first); - size_t bytes = table_merge_tree->totalBytes(settings).value(); - size_t rows = table_merge_tree->totalRows(settings).value(); + size_t bytes = table_merge_tree->totalBytes(getContext()).value(); + size_t rows = table_merge_tree->totalRows(getContext()).value(); size_t parts = table_merge_tree->getActivePartsCount(); total_number_of_bytes += bytes; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1b95b200ab9c..7eb96c06d6a0 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -333,7 +333,7 @@ bool applyTrivialCountIfPossible( select_query_info.optimize_trivial_count = true; /// Get number of rows - std::optional num_rows = storage->totalRows(settings); + std::optional num_rows = storage->totalRows(query_context); if (!num_rows) return false; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp index 2077fe6104c9..8b327b1ec1a0 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -87,7 +87,7 @@ static std::optional estimateReadRowsCount(QueryPlan::Node & node, bool } if (const auto * reading = typeid_cast(step)) - return reading->getStorage()->totalRows(Settings{}); + return reading->getStorage()->totalRows({}); if (node.children.size() != 1) return {}; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index fd1548bdaa11..f1113d28a365 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -655,7 +655,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo /// - For total_rows column in system.tables /// /// Does takes underlying Storage (if any) into account. - virtual std::optional totalRows(const Settings &) const { return {}; } + virtual std::optional totalRows(ContextPtr) const { return {}; } /// Same as above but also take partition predicate into account. virtual std::optional totalRowsByPartitionPredicate(const ActionsDAG &, ContextPtr) const { return {}; } @@ -673,7 +673,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo /// Memory part should be estimated as a resident memory size. /// In particular, alloctedBytes() is preferable over bytes() /// when considering in-memory blocks. - virtual std::optional totalBytes(const Settings &) const { return {}; } + virtual std::optional totalBytes(ContextPtr) const { return {}; } /// If it is possible to quickly determine exact number of uncompressed bytes for the table on storage: /// - disk (uncompressed) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 395d690cb2dc..7c1d82a2a10b 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -17,7 +17,6 @@ #include #include -#include #include @@ -89,6 +88,15 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl BaseStorageConfiguration::setPaths(current_metadata->makePartitionPruning(filter_dag)); } + + std::optional totalRows() override + { + if (!current_metadata) + return {}; + + return current_metadata->totalRows(); + } + std::shared_ptr getInitialSchemaByPath(const String & data_path) const override { if (!current_metadata) diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h index 4160220a4679..f2110c7a7b2e 100644 --- a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h @@ -50,6 +50,8 @@ class IDataLakeMetadata : boost::noncopyable /// Whether schema evolution is supported. virtual bool supportsExternalMetadataChange() const { return false; } + virtual std::optional totalRows() const { return {}; } + virtual std::optional totalBytes() const { return {}; } protected: [[noreturn]] void throwNotImplemented(std::string_view method) const { diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index cef983cd123a..92a742fa4f93 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -24,10 +24,10 @@ #include #include - namespace ProfileEvents { extern const Event IcebergPartitionPrunnedFiles; + extern const Event IcebergTrivialCountOptimizationApplied; } namespace DB @@ -347,10 +347,24 @@ void IcebergMetadata::updateSnapshot() "No manifest list found for snapshot id `{}` for iceberg table `{}`", relevant_snapshot_id, configuration_ptr->getPath()); + std::optional total_rows; + std::optional total_bytes; + + if (snapshot->has("summary")) + { + auto summary_object = snapshot->get("summary").extract(); + if (summary_object->has("total-records")) + total_rows = summary_object->getValue("total-records"); + + if (summary_object->has("total-files-size")) + total_bytes = summary_object->getValue("total-files-size"); + } + relevant_snapshot = IcebergSnapshot{ getManifestList(getProperFilePathFromMetadataInfo( snapshot->getValue(MANIFEST_LIST_PATH_FIELD), configuration_ptr->getPath(), table_location)), - relevant_snapshot_id}; + relevant_snapshot_id, total_rows, total_bytes}; + if (!snapshot->has("schema-id")) throw Exception( ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, @@ -599,6 +613,69 @@ Strings IcebergMetadata::makePartitionPruning(const ActionsDAG & filter_dag) } return getDataFilesImpl(&filter_dag); } + +std::optional IcebergMetadata::totalRows() const +{ + auto configuration_ptr = configuration.lock(); + if (!configuration_ptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration is expired"); + + if (!relevant_snapshot) + { + ProfileEvents::increment(ProfileEvents::IcebergTrivialCountOptimizationApplied); + return 0; + } + + /// All these "hints" with total rows or bytes are optional both in + /// metadata files and in manifest files, so we try all of them one by one + if (relevant_snapshot->total_rows.has_value()) + { + ProfileEvents::increment(ProfileEvents::IcebergTrivialCountOptimizationApplied); + return relevant_snapshot->total_rows; + } + + Int64 result = 0; + for (const auto & manifest_list_entry : *(relevant_snapshot->manifest_list)) + { + auto count = manifest_list_entry->getRowsCountInAllDataFilesExcludingDeleted(); + if (!count.has_value()) + return {}; + + result += count.value(); + } + + ProfileEvents::increment(ProfileEvents::IcebergTrivialCountOptimizationApplied); + return result; +} + + +std::optional IcebergMetadata::totalBytes() const +{ + auto configuration_ptr = configuration.lock(); + if (!configuration_ptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration is expired"); + + if (!relevant_snapshot) + return 0; + + /// All these "hints" with total rows or bytes are optional both in + /// metadata files and in manifest files, so we try all of them one by one + if (relevant_snapshot->total_bytes.has_value()) + return relevant_snapshot->total_bytes; + + Int64 result = 0; + for (const auto & manifest_list_entry : *(relevant_snapshot->manifest_list)) + { + auto count = manifest_list_entry->getBytesCountInAllDataFiles(); + if (!count.has_value()) + return {}; + + result += count.value(); + } + + return result; +} + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h index f853c85d9b5c..ff35b4c5b0d2 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h @@ -63,7 +63,6 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext const ConfigurationObserverPtr & configuration, const ContextPtr & local_context); - std::shared_ptr getInitialSchemaByPath(const String & data_path) const override { auto version_if_outdated = getSchemaVersionByFileIfOutdated(data_path); @@ -91,6 +90,9 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext bool supportsPartitionPruning() override { return true; } + std::optional totalRows() const override; + std::optional totalBytes() const override; + private: using ManifestEntryByDataFile = std::unordered_map; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp index 19e6cbb49f0d..b9a968f06db6 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB::ErrorCodes { @@ -30,6 +31,12 @@ constexpr const char * SUBCOLUMN_FILE_PATH_NAME = "data_file.file_path"; constexpr const char * SUBCOLUMN_CONTENT_NAME = "data_file.content"; constexpr const char * SUBCOLUMN_PARTITION_NAME = "data_file.partition"; +constexpr const char * SUBCOLUMN_VALUES_COUNT_NAME = "data_file.value_counts"; +constexpr const char * SUBCOLUMN_COLUMN_SIZES_NAME = "data_file.column_sizes"; +constexpr const char * SUBCOLUMN_NULL_VALUE_COUNTS_NAME = "data_file.null_value_counts"; +constexpr const char * SUBCOLUMN_LOWER_BOUNDS_NAME = "data_file.lower_bounds"; +constexpr const char * SUBCOLUMN_UPPER_BOUNDS_NAME = "data_file.upper_bounds"; + const std::vector & ManifestFileContent::getFiles() const { @@ -136,6 +143,46 @@ ManifestFileContent::ManifestFileContent( for (const auto & value : tuple) partition_key_value.emplace_back(value); + std::unordered_map columns_infos; + + for (const auto & path : {SUBCOLUMN_VALUES_COUNT_NAME, SUBCOLUMN_COLUMN_SIZES_NAME, SUBCOLUMN_NULL_VALUE_COUNTS_NAME}) + { + if (manifest_file_deserializer.hasPath(path)) + { + Field values_count = manifest_file_deserializer.getValueFromRowByName(i, path); + for (const auto & column_stats : values_count.safeGet()) + { + const auto & column_number_and_count = column_stats.safeGet(); + Int32 number = column_number_and_count[0].safeGet(); + Int64 count = column_number_and_count[1].safeGet(); + if (path == SUBCOLUMN_VALUES_COUNT_NAME) + columns_infos[number].rows_count = count; + else if (path == SUBCOLUMN_COLUMN_SIZES_NAME) + columns_infos[number].bytes_size = count; + else + columns_infos[number].nulls_count = count; + } + } + } + + for (const auto & path : {SUBCOLUMN_LOWER_BOUNDS_NAME, SUBCOLUMN_UPPER_BOUNDS_NAME}) + { + if (manifest_file_deserializer.hasPath(path)) + { + Field bounds = manifest_file_deserializer.getValueFromRowByName(i, path); + for (const auto & column_stats : bounds.safeGet()) + { + const auto & column_number_and_bound = column_stats.safeGet(); + Int32 number = column_number_and_bound[0].safeGet(); + const Field & bound_value = column_number_and_bound[1]; + if (path == SUBCOLUMN_LOWER_BOUNDS_NAME) + columns_infos[number].lower_bound = bound_value; + else + columns_infos[number].upper_bound = bound_value; + } + } + } + FileEntry file = FileEntry{DataFileEntry{file_path}}; Int64 added_sequence_number = 0; @@ -162,7 +209,7 @@ ManifestFileContent::ManifestFileContent( break; } } - this->files.emplace_back(status, added_sequence_number, file, partition_key_value); + this->files.emplace_back(status, added_sequence_number, file, partition_key_value, columns_infos); } } @@ -183,6 +230,54 @@ const std::vector & ManifestFileContent::getPartitionKeyColumnIDs() const return partition_column_ids; } +std::optional ManifestFileContent::getRowsCountInAllDataFilesExcludingDeleted() const +{ + Int64 result = 0; + for (const auto & file : files) + { + /// Have at least one column with rows count + bool found = false; + for (const auto & [column, column_info] : file.columns_infos) + { + if (column_info.rows_count.has_value()) + { + if (file.status != ManifestEntryStatus::DELETED) + result += *column_info.rows_count; + found = true; + break; + } + } + + if (!found) + return std::nullopt; + } + return result; +} + +std::optional ManifestFileContent::getBytesCountInAllDataFiles() const +{ + Int64 result = 0; + for (const auto & file : files) + { + /// Have at least one column with bytes count + bool found = false; + for (const auto & [column, column_info] : file.columns_infos) + { + if (column_info.bytes_size.has_value()) + { + result += *column_info.bytes_size; + found = true; + break; + } + } + + if (!found) + return std::nullopt; + } + return result; + +} + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h index 1c3b6035a24e..4e2f13fb3cbf 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h @@ -37,6 +37,15 @@ struct DataFileEntry String file_name; }; +struct ColumnInfo +{ + std::optional rows_count; + std::optional bytes_size; + std::optional nulls_count; + std::optional lower_bound; + std::optional upper_bound; +}; + using FileEntry = std::variant; // In the future we will add PositionalDeleteFileEntry and EqualityDeleteFileEntry here /// Description of Data file in manifest file @@ -47,6 +56,7 @@ struct ManifestFileEntry FileEntry file; DB::Row partition_key_value; + std::unordered_map columns_infos; }; /** @@ -94,6 +104,11 @@ class ManifestFileContent bool hasPartitionKey() const; const DB::KeyDescription & getPartitionKeyDescription() const; const std::vector & getPartitionKeyColumnIDs() const; + + /// Fields with rows count in manifest files are optional + /// they can be absent. + std::optional getRowsCountInAllDataFilesExcludingDeleted() const; + std::optional getBytesCountInAllDataFiles() const; private: Int32 schema_id; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h index e6efb1eb7c00..ed5ba39b2e7a 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h @@ -23,6 +23,8 @@ struct IcebergSnapshot { ManifestListIterator manifest_list_iterator; Int64 snapshot_id; + std::optional total_rows; + std::optional total_bytes; }; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 229a6e47b545..aa0ab6dd2c67 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -202,6 +202,18 @@ void StorageObjectStorage::updateExternalDynamicMetadata(ContextPtr context_ptr) setInMemoryMetadata(metadata); } +std::optional StorageObjectStorage::totalRows(ContextPtr query_context) const +{ + configuration->update(object_storage, query_context); + return configuration->totalRows(); +} + +std::optional StorageObjectStorage::totalBytes(ContextPtr query_context) const +{ + configuration->update(object_storage, query_context); + return configuration->totalBytes(); +} + namespace { class ReadFromObjectStorageStep : public SourceStepWithFilter @@ -674,4 +686,5 @@ void StorageObjectStorage::Configuration::assertInitialized() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration was not initialized before usage"); } } + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 440774d30d5c..0fbeb2df431b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -139,6 +139,8 @@ class StorageObjectStorage : public IStorage void updateExternalDynamicMetadata(ContextPtr) override; + std::optional totalRows(ContextPtr query_context) const override; + std::optional totalBytes(ContextPtr query_context) const override; protected: String getPathSample(ContextPtr context); @@ -222,6 +224,9 @@ class StorageObjectStorage::Configuration virtual void implementPartitionPruning(const ActionsDAG &) { } + virtual std::optional totalRows() { return {}; } + virtual std::optional totalBytes() { return {}; } + virtual bool hasExternalDynamicMetadata() { return false; } virtual std::shared_ptr getInitialSchemaByPath(const String&) const { return {}; } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index dcb9f86e130f..f18f7ba53e70 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -823,9 +823,9 @@ Chunk StorageEmbeddedRocksDB::getBySerializedKeys( return Chunk(std::move(columns), num_rows); } -std::optional StorageEmbeddedRocksDB::totalRows(const Settings & query_settings) const +std::optional StorageEmbeddedRocksDB::totalRows(ContextPtr query_context) const { - if (!query_settings[Setting::optimize_trivial_approximate_count_query]) + if (!query_context->getSettingsRef()[Setting::optimize_trivial_approximate_count_query]) return {}; std::shared_lock lock(rocksdb_ptr_mx); if (!rocksdb_ptr) @@ -836,7 +836,7 @@ std::optional StorageEmbeddedRocksDB::totalRows(const Settings & query_s return estimated_rows; } -std::optional StorageEmbeddedRocksDB::totalBytes(const Settings & /*settings*/) const +std::optional StorageEmbeddedRocksDB::totalBytes(ContextPtr) const { std::shared_lock lock(rocksdb_ptr_mx); if (!rocksdb_ptr) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 5b2cef648e40..51a6e907b7d6 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -102,9 +102,9 @@ class StorageEmbeddedRocksDB final : public IStorage, public IKeyValueEntity, Wi /// To turn on the optimization optimize_trivial_approximate_count_query=1 should be set for a query. bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } - std::optional totalRows(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalBytes(ContextPtr query_context) const override; void checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const override; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 4345ad8aaf69..3c77fe41b21b 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1167,16 +1167,16 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, Context } } -std::optional StorageBuffer::totalRows(const Settings & settings) const +std::optional StorageBuffer::totalRows(ContextPtr query_context) const { std::optional underlying_rows; if (auto destination = getDestinationTable()) - underlying_rows = destination->totalRows(settings); + underlying_rows = destination->totalRows(query_context); return total_writes.rows + underlying_rows.value_or(0); } -std::optional StorageBuffer::totalBytes(const Settings & /*settings*/) const +std::optional StorageBuffer::totalBytes(ContextPtr) const { return total_writes.bytes; } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 3341271db4a5..d68fb1ee1a95 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -116,8 +116,8 @@ friend class BufferSink; /// The structure of the subordinate table is not checked and does not change. void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & table_lock_holder) override; - std::optional totalRows(const Settings & settings) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; + std::optional totalBytes(ContextPtr query_context) const override; std::optional lifetimeRows() const override { return lifetime_writes.rows; } std::optional lifetimeBytes() const override { return lifetime_writes.bytes; } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ca4b7206755d..5934b7875418 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1513,7 +1513,7 @@ Cluster::Addresses StorageDistributed::parseAddresses(const std::string & name) return addresses; } -std::optional StorageDistributed::totalBytes(const Settings &) const +std::optional StorageDistributed::totalBytes(ContextPtr) const { UInt64 total_bytes = 0; for (const auto & status : getDirectoryQueueStatuses()) @@ -1837,7 +1837,7 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const !(*distributed_settings)[DistributedSetting::bytes_to_delay_insert]) return; - UInt64 total_bytes = *totalBytes(getContext()->getSettingsRef()); + UInt64 total_bytes = *totalBytes(getContext()); if ((*distributed_settings)[DistributedSetting::bytes_to_throw_insert] && total_bytes > (*distributed_settings)[DistributedSetting::bytes_to_throw_insert]) { @@ -1858,12 +1858,12 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const do { delayed_ms += step_ms; std::this_thread::sleep_for(std::chrono::milliseconds(step_ms)); - } while (*totalBytes(getContext()->getSettingsRef()) > (*distributed_settings)[DistributedSetting::bytes_to_delay_insert] && delayed_ms < (*distributed_settings)[DistributedSetting::max_delay_to_insert]*1000); + } while (*totalBytes(getContext()) > (*distributed_settings)[DistributedSetting::bytes_to_delay_insert] && delayed_ms < (*distributed_settings)[DistributedSetting::max_delay_to_insert]*1000); ProfileEvents::increment(ProfileEvents::DistributedDelayedInserts); ProfileEvents::increment(ProfileEvents::DistributedDelayedInsertsMilliseconds, delayed_ms); - UInt64 new_total_bytes = *totalBytes(getContext()->getSettingsRef()); + UInt64 new_total_bytes = *totalBytes(getContext()); LOG_INFO(log, "Too many bytes pending for async INSERT: was {}, now {}, INSERT was delayed to {} ms", formatReadableSizeWithBinarySuffix(total_bytes), formatReadableSizeWithBinarySuffix(new_total_bytes), diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 8bdd5be837f8..58a683e1dee3 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -109,7 +109,7 @@ class StorageDistributed final : public IStorage, WithContext size_t /*num_streams*/) override; bool supportsParallelInsert() const override { return true; } - std::optional totalBytes(const Settings &) const override; + std::optional totalBytes(ContextPtr) const override; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool /*async_insert*/) override; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 74c1e0ee4bc7..8f52417543c6 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -337,14 +337,16 @@ size_t StorageJoin::getSize(ContextPtr context) const return join->getTotalRowCount(); } -std::optional StorageJoin::totalRows(const Settings &settings) const +std::optional StorageJoin::totalRows(ContextPtr query_context) const { + const auto & settings = query_context->getSettingsRef(); TableLockHolder holder = tryLockTimed(rwlock, RWLockImpl::Read, RWLockImpl::NO_QUERY, settings[Setting::lock_acquire_timeout]); return join->getTotalRowCount(); } -std::optional StorageJoin::totalBytes(const Settings &settings) const +std::optional StorageJoin::totalBytes(ContextPtr query_context) const { + const auto & settings = query_context->getSettingsRef(); TableLockHolder holder = tryLockTimed(rwlock, RWLockImpl::Read, RWLockImpl::NO_QUERY, settings[Setting::lock_acquire_timeout]); return join->getTotalByteCount(); } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index dd8ef97f8cc5..dab7a21605bc 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -83,8 +83,8 @@ class StorageJoin final : public StorageSetOrJoinBase size_t max_block_size, size_t num_streams) override; - std::optional totalRows(const Settings & settings) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; + std::optional totalBytes(ContextPtr query_context) const override; Block getRightSampleBlock() const { diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 468856938423..5a3247a63c1e 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -987,7 +987,7 @@ void StorageLog::updateTotalRows(const WriteLock &) total_rows = 0; } -std::optional StorageLog::totalRows(const Settings &) const +std::optional StorageLog::totalRows(ContextPtr) const { if (use_marks_file && marks_loaded) return total_rows; @@ -998,7 +998,7 @@ std::optional StorageLog::totalRows(const Settings &) const return {}; } -std::optional StorageLog::totalBytes(const Settings &) const +std::optional StorageLog::totalBytes(ContextPtr) const { return total_bytes; } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 882e9cfaa75b..71dd78543d20 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -69,8 +69,8 @@ class StorageLog final : public IStorage, public WithMutableContext bool supportsSubcolumns() const override { return true; } ColumnSizeByName getColumnSizes() const override; - std::optional totalRows(const Settings & settings) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalRows(ContextPtr) const override; + std::optional totalBytes(ContextPtr) const override; void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 1ebf1893aa38..a14dde4aab8a 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -821,22 +821,22 @@ bool StorageMaterializedView::supportsBackupPartition() const return false; } -std::optional StorageMaterializedView::totalRows(const Settings & settings) const +std::optional StorageMaterializedView::totalRows(ContextPtr query_context) const { if (hasInnerTable()) { if (auto table = tryGetTargetTable()) - return table->totalRows(settings); + return table->totalRows(query_context); } return {}; } -std::optional StorageMaterializedView::totalBytes(const Settings & settings) const +std::optional StorageMaterializedView::totalBytes(ContextPtr query_context) const { if (hasInnerTable()) { if (auto table = tryGetTargetTable()) - return table->totalBytes(settings); + return table->totalBytes(query_context); } return {}; } diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index b995731b5dae..2eff219fc155 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -98,8 +98,8 @@ class StorageMaterializedView final : public IStorage, WithMutableContext void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; bool supportsBackupPartition() const override; - std::optional totalRows(const Settings & settings) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; + std::optional totalBytes(ContextPtr query_context) const override; std::optional totalBytesUncompressed(const Settings & settings) const override; private: diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index da7fb483ef24..2980c7a82156 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -612,14 +612,14 @@ void StorageMemory::checkAlterIsPossible(const AlterCommands & commands, Context } } -std::optional StorageMemory::totalRows(const Settings &) const +std::optional StorageMemory::totalRows(ContextPtr) const { /// All modifications of these counters are done under mutex which automatically guarantees synchronization/consistency /// When run concurrently we are fine with any value: "before" or "after" return total_size_rows.load(std::memory_order_relaxed); } -std::optional StorageMemory::totalBytes(const Settings &) const +std::optional StorageMemory::totalBytes(ContextPtr) const { return total_size_bytes.load(std::memory_order_relaxed); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 07cf86a29737..6b2ae22c8728 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -86,8 +86,8 @@ friend class MemorySink; void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & alter_lock_holder) override; - std::optional totalRows(const Settings &) const override; - std::optional totalBytes(const Settings &) const override; + std::optional totalRows(ContextPtr) const override; + std::optional totalBytes(ContextPtr) const override; /** Delays initialization of StorageMemory::read() until the first read is actually happen. * Usually, fore code like this: diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index a527296a7267..cb8a0c29cced 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1675,14 +1675,14 @@ bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr &, return traverseTablesUntil([&](const auto & table) { return !table->supportsTrivialCountOptimization(nullptr, ctx); }) == nullptr; } -std::optional StorageMerge::totalRows(const Settings & settings) const +std::optional StorageMerge::totalRows(ContextPtr query_context) const { - return totalRowsOrBytes([&](const auto & table) { return table->totalRows(settings); }); + return totalRowsOrBytes([&](const auto & table) { return table->totalRows(query_context); }); } -std::optional StorageMerge::totalBytes(const Settings & settings) const +std::optional StorageMerge::totalBytes(ContextPtr query_context) const { - return totalRowsOrBytes([&](const auto & table) { return table->totalBytes(settings); }); + return totalRowsOrBytes([&](const auto & table) { return table->totalBytes(query_context); }); } template diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index ac4ae17e006f..c15164eb5445 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -81,8 +81,8 @@ class StorageMerge final : public IStorage, WithContext bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; - std::optional totalRows(const Settings & settings) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; + std::optional totalBytes(ContextPtr query_context) const override; using DatabaseTablesIterators = std::vector; DatabaseTablesIterators getDatabaseIterators(ContextPtr context) const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0d394a3ffc8a..e277b9400087 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -310,7 +310,7 @@ void StorageMergeTree::read( query_plan = std::move(*plan); } -std::optional StorageMergeTree::totalRows(const Settings &) const +std::optional StorageMergeTree::totalRows(ContextPtr) const { return getTotalActiveSizeInRows(); } @@ -321,7 +321,7 @@ std::optional StorageMergeTree::totalRowsByPartitionPredicate(const Acti return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts); } -std::optional StorageMergeTree::totalBytes(const Settings &) const +std::optional StorageMergeTree::totalBytes(ContextPtr) const { return getTotalActiveSizeInBytes(); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index eed6d1945acd..e0f512b54f9d 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -64,9 +64,9 @@ class StorageMergeTree final : public MergeTreeData size_t max_block_size, size_t num_streams) override; - std::optional totalRows(const Settings &) const override; + std::optional totalRows(ContextPtr) const override; std::optional totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr) const override; - std::optional totalBytes(const Settings &) const override; + std::optional totalBytes(ContextPtr) const override; std::optional totalBytesUncompressed(const Settings &) const override; UInt64 getNumberOnFlyDataMutations() const override; diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 74abf931f8f1..5507b183a834 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -59,11 +59,11 @@ class StorageNull final : public IStorage void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & table_lock_holder) override; - std::optional totalRows(const Settings &) const override + std::optional totalRows(ContextPtr) const override { return {0}; } - std::optional totalBytes(const Settings &) const override + std::optional totalBytes(ContextPtr) const override { return {0}; } diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 5cd86f7ad2c1..ee68b4d4be05 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -155,8 +155,8 @@ class StorageProxy : public IStorage bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } StoragePolicyPtr getStoragePolicy() const override { return getNested()->getStoragePolicy(); } - std::optional totalRows(const Settings & settings) const override { return getNested()->totalRows(settings); } - std::optional totalBytes(const Settings & settings) const override { return getNested()->totalBytes(settings); } + std::optional totalRows(ContextPtr query_context) const override { return getNested()->totalRows(query_context); } + std::optional totalBytes(ContextPtr query_context) const override { return getNested()->totalBytes(query_context); } std::optional lifetimeRows() const override { return getNested()->lifetimeRows(); } std::optional lifetimeBytes() const override { return getNested()->lifetimeBytes(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1e466d2fc699..809c965ab627 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5901,8 +5901,9 @@ void StorageReplicatedMergeTree::foreachActiveParts(Func && func, bool select_se } } -std::optional StorageReplicatedMergeTree::totalRows(const Settings & settings) const +std::optional StorageReplicatedMergeTree::totalRows(ContextPtr query_context) const { + const auto & settings = query_context->getSettingsRef(); UInt64 res = 0; foreachActiveParts([&res](auto & part) { res += part->rows_count; }, settings[Setting::select_sequential_consistency]); return res; @@ -5915,8 +5916,9 @@ std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate( return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts); } -std::optional StorageReplicatedMergeTree::totalBytes(const Settings & settings) const +std::optional StorageReplicatedMergeTree::totalBytes(ContextPtr query_context) const { + const auto & settings = query_context->getSettingsRef(); UInt64 res = 0; foreachActiveParts([&res](auto & part) { res += part->getBytesOnDisk(); }, settings[Setting::select_sequential_consistency]); return res; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e09f0cd373f2..14f3f4127d07 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -163,9 +163,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData size_t max_block_size, size_t num_streams) override; - std::optional totalRows(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; std::optional totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalBytes(ContextPtr query_context) const override; std::optional totalBytesUncompressed(const Settings & settings) const override; UInt64 getNumberOnFlyDataMutations() const override; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 498b09fd490c..e10bd2b5717c 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -213,7 +213,7 @@ size_t StorageSet::getSize(ContextPtr) const return current_set->getTotalRowCount(); } -std::optional StorageSet::totalRows(const Settings &) const +std::optional StorageSet::totalRows(ContextPtr) const { SetPtr current_set; { @@ -223,7 +223,7 @@ std::optional StorageSet::totalRows(const Settings &) const return current_set->getTotalRowCount(); } -std::optional StorageSet::totalBytes(const Settings &) const +std::optional StorageSet::totalBytes(ContextPtr) const { SetPtr current_set; { diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index 5b6d899b48db..fb4cc81ffb5e 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -82,8 +82,8 @@ class StorageSet final : public StorageSetOrJoinBase void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; - std::optional totalRows(const Settings & settings) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; + std::optional totalBytes(ContextPtr query_context) const override; private: /// Allows to concurrently truncate the set and work (read/fill) the existing set. diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 0db0f67084fc..522833f1f848 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -538,7 +538,7 @@ void StorageStripeLog::updateTotalRows(const WriteLock &) total_rows = new_total_rows; } -std::optional StorageStripeLog::totalRows(const Settings &) const +std::optional StorageStripeLog::totalRows(ContextPtr) const { if (indices_loaded) return total_rows; @@ -549,7 +549,7 @@ std::optional StorageStripeLog::totalRows(const Settings &) const return {}; } -std::optional StorageStripeLog::totalBytes(const Settings &) const +std::optional StorageStripeLog::totalBytes(ContextPtr) const { return total_bytes; } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index dc2f8d8be4a1..66a686670187 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -61,8 +61,8 @@ friend class StripeLogSink; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; - std::optional totalRows(const Settings & settings) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; + std::optional totalBytes(ContextPtr query_context) const override; void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; diff --git a/src/Storages/StorageTimeSeries.cpp b/src/Storages/StorageTimeSeries.cpp index b5b0f2fdb952..9ed782a96e11 100644 --- a/src/Storages/StorageTimeSeries.cpp +++ b/src/Storages/StorageTimeSeries.cpp @@ -255,7 +255,7 @@ StoragePtr StorageTimeSeries::tryGetTargetTable(ViewTarget::Kind target_kind, co } -std::optional StorageTimeSeries::totalRows(const Settings & settings) const +std::optional StorageTimeSeries::totalRows(ContextPtr query_context) const { UInt64 total_rows = 0; if (has_inner_tables) @@ -268,7 +268,7 @@ std::optional StorageTimeSeries::totalRows(const Settings & settings) co if (!inner_table) return std::nullopt; - auto total_rows_in_inner_table = inner_table->totalRows(settings); + auto total_rows_in_inner_table = inner_table->totalRows(query_context); if (!total_rows_in_inner_table) return std::nullopt; @@ -279,7 +279,7 @@ std::optional StorageTimeSeries::totalRows(const Settings & settings) co return total_rows; } -std::optional StorageTimeSeries::totalBytes(const Settings & settings) const +std::optional StorageTimeSeries::totalBytes(ContextPtr query_context) const { UInt64 total_bytes = 0; if (has_inner_tables) @@ -292,7 +292,7 @@ std::optional StorageTimeSeries::totalBytes(const Settings & settings) c if (!inner_table) return std::nullopt; - auto total_bytes_in_inner_table = inner_table->totalBytes(settings); + auto total_bytes_in_inner_table = inner_table->totalBytes(query_context); if (!total_bytes_in_inner_table) return std::nullopt; diff --git a/src/Storages/StorageTimeSeries.h b/src/Storages/StorageTimeSeries.h index 263b2c42bc2e..444f27cdd014 100644 --- a/src/Storages/StorageTimeSeries.h +++ b/src/Storages/StorageTimeSeries.h @@ -85,8 +85,8 @@ class StorageTimeSeries final : public IStorage, WithContext void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; - std::optional totalRows(const Settings & settings) const override; - std::optional totalBytes(const Settings & settings) const override; + std::optional totalRows(ContextPtr query_context) const override; + std::optional totalBytes(ContextPtr query_context) const override; std::optional totalBytesUncompressed(const Settings & settings) const override; Strings getDataPaths() const override; diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index bf4243764a5f..be37c195c158 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -341,7 +341,6 @@ class TablesBlockSource : public ISource if (columns_mask[src_index++]) res_columns[res_index++]->insert(table.second->getName()); - const auto & settings = context->getSettingsRef(); while (src_index < columns_mask.size()) { // total_rows @@ -349,7 +348,7 @@ class TablesBlockSource : public ISource { try { - if (auto total_rows = table.second->totalRows(settings)) + if (auto total_rows = table.second->totalRows(context)) res_columns[res_index++]->insert(*total_rows); else res_columns[res_index++]->insertDefault(); @@ -366,7 +365,7 @@ class TablesBlockSource : public ISource { try { - if (auto total_bytes = table.second->totalBytes(settings)) + if (auto total_bytes = table.second->totalBytes(context)) res_columns[res_index++]->insert(*total_bytes); else res_columns[res_index++]->insertDefault(); @@ -588,13 +587,16 @@ class TablesBlockSource : public ISource res_columns[res_index++]->insertDefault(); } - auto settings = context->getSettingsRef(); - settings[Setting::select_sequential_consistency] = 0; + ContextMutablePtr context_copy = Context::createCopy(context); + Settings settings_copy = context_copy->getSettingsCopy(); + settings_copy[Setting::select_sequential_consistency] = 0; + context_copy->setSettings(settings_copy); + if (columns_mask[src_index++]) { try { - auto total_rows = table ? table->totalRows(settings) : std::nullopt; + auto total_rows = table ? table->totalRows(context) : std::nullopt; if (total_rows) res_columns[res_index++]->insert(*total_rows); else @@ -612,7 +614,7 @@ class TablesBlockSource : public ISource { try { - auto total_bytes = table->totalBytes(settings); + auto total_bytes = table->totalBytes(context_copy); if (total_bytes) res_columns[res_index++]->insert(*total_bytes); else @@ -630,7 +632,7 @@ class TablesBlockSource : public ISource { try { - auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); + auto total_bytes_uncompressed = table->totalBytesUncompressed(context_copy->getSettingsRef()); if (total_bytes_uncompressed) res_columns[res_index++]->insert(*total_bytes_uncompressed); else From d793a111169cc8f9dde6ba67d191554d117b18f0 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 21 Mar 2025 18:39:12 +0000 Subject: [PATCH 04/14] Merge pull request #78021 from hanfei1991/hanfei/refactor-iceberg Refactor some code in Iceberg Storage --- .../DataLakes/Iceberg/IcebergMetadata.cpp | 41 ++++++++----------- .../DataLakes/Iceberg/IcebergMetadata.h | 16 ++++---- .../DataLakes/Iceberg/IteratorWrapper.h | 29 ------------- .../DataLakes/Iceberg/ManifestFile.h | 6 +-- .../DataLakes/Iceberg/Snapshot.h | 18 +++----- 5 files changed, 33 insertions(+), 77 deletions(-) delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/IteratorWrapper.h diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index 92a742fa4f93..e5f45afc3530 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -506,20 +506,19 @@ ManifestList IcebergMetadata::initializeManifestList(const String & filename) co /// We can't encapsulate this logic in getManifestFile because we need not only the name of the file, but also an inherited sequence number which is known only during the parsing of ManifestList auto manifest_file_content = initializeManifestFile(manifest_file_name, added_sequence_number); - auto [iterator, _inserted] = manifest_files_by_name.emplace(manifest_file_name, std::move(manifest_file_content)); - auto manifest_file_iterator = ManifestFileIterator{iterator}; - for (const auto & data_file_path : manifest_file_iterator->getFiles()) + manifest_files_by_name.emplace(manifest_file_name, manifest_file_content); + for (const auto & data_file_path : manifest_file_content->getFiles()) { if (std::holds_alternative(data_file_path.file)) - manifest_file_by_data_file.emplace(std::get(data_file_path.file).file_name, manifest_file_iterator); + manifest_file_by_data_file.emplace(std::get(data_file_path.file).file_name, manifest_file_content); } - manifest_list.push_back(ManifestListFileEntry{manifest_file_iterator, added_sequence_number}); + manifest_list.push_back(manifest_file_content); } return manifest_list; } -ManifestFileContent IcebergMetadata::initializeManifestFile(const String & filename, Int64 inherited_sequence_number) const +ManifestFilePtr IcebergMetadata::initializeManifestFile(const String & filename, Int64 inherited_sequence_number) const { auto configuration_ptr = configuration.lock(); @@ -528,7 +527,7 @@ ManifestFileContent IcebergMetadata::initializeManifestFile(const String & filen AvroForIcebergDeserializer manifest_file_deserializer(std::move(buffer), filename, getFormatSettings(getContext())); auto [schema_id, schema_object] = parseTableSchemaFromManifestFile(manifest_file_deserializer, filename); schema_processor.addIcebergTableSchema(schema_object); - return ManifestFileContent( + return std::make_shared( manifest_file_deserializer, format_version, configuration_ptr->getPath(), @@ -537,32 +536,26 @@ ManifestFileContent IcebergMetadata::initializeManifestFile(const String & filen inherited_sequence_number, table_location, getContext()); -} -ManifestFileIterator IcebergMetadata::getManifestFile(const String & filename) const -{ - auto manifest_file_it = manifest_files_by_name.find(filename); - if (manifest_file_it != manifest_files_by_name.end()) - return ManifestFileIterator{manifest_file_it}; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot find manifest file: {}", filename); } -std::optional IcebergMetadata::tryGetManifestFile(const String & filename) const +ManifestFilePtr IcebergMetadata::tryGetManifestFile(const String & filename) const { auto manifest_file_it = manifest_files_by_name.find(filename); if (manifest_file_it != manifest_files_by_name.end()) - return ManifestFileIterator{manifest_file_it}; - return std::nullopt; + return manifest_file_it->second; + return nullptr; } -ManifestListIterator IcebergMetadata::getManifestList(const String & filename) const +ManifestListPtr IcebergMetadata::getManifestList(const String & filename) const { auto manifest_file_it = manifest_lists_by_name.find(filename); if (manifest_file_it != manifest_lists_by_name.end()) - return ManifestListIterator{manifest_file_it}; + return manifest_file_it->second; auto configuration_ptr = configuration.lock(); - auto [manifest_file_iterator, _inserted] = manifest_lists_by_name.emplace(filename, initializeManifestList(filename)); - return ManifestListIterator{manifest_file_iterator}; + auto manifest_list_ptr = std::make_shared(initializeManifestList(filename)); + manifest_lists_by_name.emplace(filename, manifest_list_ptr); + return manifest_list_ptr; } Strings IcebergMetadata::getDataFilesImpl(const ActionsDAG * filter_dag) const @@ -574,10 +567,10 @@ Strings IcebergMetadata::getDataFilesImpl(const ActionsDAG * filter_dag) const return cached_unprunned_files_for_last_processed_snapshot.value(); Strings data_files; - for (const auto & manifest_list_entry : *(relevant_snapshot->manifest_list_iterator)) + for (const auto & manifest_file_ptr : *(relevant_snapshot->manifest_list)) { - PartitionPruner pruner(schema_processor, relevant_snapshot_schema_id, filter_dag, *manifest_list_entry.manifest_file, getContext()); - const auto & data_files_in_manifest = manifest_list_entry.manifest_file->getFiles(); + PartitionPruner pruner(schema_processor, relevant_snapshot_schema_id, filter_dag, *manifest_file_ptr, getContext()); + const auto & data_files_in_manifest = manifest_file_ptr->getFiles(); for (const auto & manifest_file_entry : data_files_in_manifest) { if (manifest_file_entry.status != ManifestEntryStatus::DELETED) diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h index ff35b4c5b0d2..0826b86ca035 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h @@ -94,15 +94,17 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext std::optional totalBytes() const override; private: - using ManifestEntryByDataFile = std::unordered_map; + using ManifestEntryByDataFile = std::unordered_map; + using ManifestFilesStorage = std::unordered_map; + using ManifestListsStorage = std::unordered_map; const ObjectStoragePtr object_storage; const ConfigurationObserverPtr configuration; mutable IcebergSchemaProcessor schema_processor; LoggerPtr log; - mutable Iceberg::ManifestFilesStorage manifest_files_by_name; - mutable Iceberg::ManifestListsStorage manifest_lists_by_name; + mutable ManifestFilesStorage manifest_files_by_name; + mutable ManifestListsStorage manifest_lists_by_name; mutable ManifestEntryByDataFile manifest_file_by_data_file; std::tuple getVersion() const { return std::make_tuple(relevant_snapshot_id, relevant_snapshot_schema_id); } @@ -128,13 +130,11 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext void addTableSchemaById(Int32 schema_id); - Iceberg::ManifestListIterator getManifestList(const String & filename) const; + Iceberg::ManifestListPtr getManifestList(const String & filename) const; std::optional getSchemaVersionByFileIfOutdated(String data_path) const; - Iceberg::ManifestFileContent initializeManifestFile(const String & filename, Int64 inherited_sequence_number) const; - - Iceberg::ManifestFileIterator getManifestFile(const String & filename) const; + Iceberg::ManifestFilePtr initializeManifestFile(const String & filename, Int64 inherited_sequence_number) const; std::optional getRelevantManifestList(const Poco::JSON::Object::Ptr & metadata); @@ -142,7 +142,7 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext Strings getDataFilesImpl(const ActionsDAG * filter_dag) const; - std::optional tryGetManifestFile(const String & filename) const; + Iceberg::ManifestFilePtr tryGetManifestFile(const String & filename) const; }; } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IteratorWrapper.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IteratorWrapper.h deleted file mode 100644 index e64a6e95fb81..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IteratorWrapper.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include -#include -namespace Iceberg -{ - -template -class IteratorWrapper -{ -private: - using StorageType = std::map; - using StorageConstIterator = StorageType::const_iterator; - using StorageIterator = StorageType::iterator; - -public: - explicit IteratorWrapper(StorageConstIterator iterator_) : iterator(iterator_) { } - explicit IteratorWrapper(StorageIterator iterator_) : iterator(iterator_) { } - - String getName() const { return iterator->first; } - - const T * operator->() const { return &iterator->second; } - const T & operator*() const { return iterator->second; } - -private: - StorageIterator iterator; -}; - -} diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h index 4e2f13fb3cbf..46d4fae5bc77 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h @@ -4,7 +4,6 @@ #if USE_AVRO -#include #include #include #include @@ -120,8 +119,9 @@ class ManifestFileContent }; -using ManifestFilesStorage = std::map; -using ManifestFileIterator = IteratorWrapper; +/// Once manifest file is constructed. It's unchangeable. +using ManifestFilePtr = std::shared_ptr; + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h index ed5ba39b2e7a..21fc25063bc0 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Snapshot.h @@ -3,25 +3,17 @@ #if USE_AVRO -#include "Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h" -namespace Iceberg -{ +#include -struct ManifestListFileEntry +namespace Iceberg { - ManifestFileIterator manifest_file; - Int64 added_sequence_number; -}; - -using ManifestList = std::vector; - -using ManifestListsStorage = std::map; -using ManifestListIterator = IteratorWrapper; +using ManifestList = std::vector; +using ManifestListPtr = std::shared_ptr; struct IcebergSnapshot { - ManifestListIterator manifest_list_iterator; + ManifestListPtr manifest_list; Int64 snapshot_id; std::optional total_rows; std::optional total_bytes; From c32eb471f8050bd7063064c6e4bfbd13b81982ea Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Apr 2025 16:51:41 +0000 Subject: [PATCH 05/14] Merge pull request #78242 from ClickHouse/minmax_iceberg Minmax iceberg --- src/Common/ProfileEvents.cpp | 1 + .../DataLakes/Iceberg/IcebergMetadata.cpp | 11 +- .../DataLakes/Iceberg/ManifestFile.cpp | 119 +++++++++- .../DataLakes/Iceberg/ManifestFile.h | 11 +- .../Iceberg/ManifestFilesPruning.cpp | 217 ++++++++++++++++++ ...titionPruning.h => ManifestFilesPruning.h} | 15 +- .../DataLakes/Iceberg/PartitionPruning.cpp | 144 ------------ .../DataLakes/Iceberg/SchemaProcessor.cpp | 11 + .../DataLakes/Iceberg/SchemaProcessor.h | 2 + 9 files changed, 360 insertions(+), 171 deletions(-) create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFilesPruning.cpp rename src/Storages/ObjectStorage/DataLakes/Iceberg/{PartitionPruning.h => ManifestFilesPruning.h} (72%) delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/PartitionPruning.cpp diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 4e7622232cf4..a873aab3579d 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -215,6 +215,7 @@ \ M(IcebergPartitionPrunnedFiles, "Number of skipped files during Iceberg partition pruning", ValueType::Number) \ M(IcebergTrivialCountOptimizationApplied, "Trivial count optimization applied while reading from Iceberg", ValueType::Number) \ + M(IcebergMinMaxIndexPrunnedFiles, "Number of skipped files by using MinMax index in Iceberg", ValueType::Number) \ M(JoinBuildTableRowCount, "Total number of rows in the build table for a JOIN operation.", ValueType::Number) \ M(JoinProbeTableRowCount, "Total number of rows in the probe table for a JOIN operation.", ValueType::Number) \ M(JoinResultRowCount, "Total number of rows in the result of a JOIN operation.", ValueType::Number) \ diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index e5f45afc3530..aedba7369ecd 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include @@ -26,7 +26,6 @@ namespace ProfileEvents { - extern const Event IcebergPartitionPrunnedFiles; extern const Event IcebergTrivialCountOptimizationApplied; } @@ -569,17 +568,13 @@ Strings IcebergMetadata::getDataFilesImpl(const ActionsDAG * filter_dag) const Strings data_files; for (const auto & manifest_file_ptr : *(relevant_snapshot->manifest_list)) { - PartitionPruner pruner(schema_processor, relevant_snapshot_schema_id, filter_dag, *manifest_file_ptr, getContext()); + ManifestFilesPruner pruner(schema_processor, relevant_snapshot_schema_id, filter_dag, *manifest_file_ptr, getContext()); const auto & data_files_in_manifest = manifest_file_ptr->getFiles(); for (const auto & manifest_file_entry : data_files_in_manifest) { if (manifest_file_entry.status != ManifestEntryStatus::DELETED) { - if (pruner.canBePruned(manifest_file_entry)) - { - ProfileEvents::increment(ProfileEvents::IcebergPartitionPrunnedFiles); - } - else + if (!pruner.canBePruned(manifest_file_entry)) { if (std::holds_alternative(manifest_file_entry.file)) data_files.push_back(std::get(manifest_file_entry.file).file_name); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp index b9a968f06db6..650a5c4af033 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp @@ -4,14 +4,16 @@ #include #include -#include +#include +#include #include #include #include #include #include -#include +#include +#include namespace DB::ErrorCodes { @@ -23,6 +25,86 @@ namespace DB::ErrorCodes namespace Iceberg { +namespace +{ + /// Iceberg stores lower_bounds and upper_bounds serialized with some custom deserialization as bytes array + /// https://iceberg.apache.org/spec/#appendix-d-single-value-serialization + std::optional deserializeFieldFromBinaryRepr(std::string str, DB::DataTypePtr expected_type, bool lower_bound) + { + auto non_nullable_type = DB::removeNullable(expected_type); + auto column = non_nullable_type->createColumn(); + if (DB::WhichDataType(non_nullable_type).isDecimal()) + { + /// Iceberg store decimal values as unscaled value with two’s-complement big-endian binary + /// using the minimum number of bytes for the value + /// Our decimal binary representation is little endian + /// so we cannot reuse our default code for parsing it. + int64_t unscaled_value = 0; + + // Convert from big-endian to signed int + for (const auto byte : str) + unscaled_value = (unscaled_value << 8) | static_cast(byte); + + /// Add sign + if (str[0] & 0x80) + { + int64_t sign_extension = -1; + sign_extension <<= (str.size() * 8); + unscaled_value |= sign_extension; + } + + /// NOTE: It's very weird, but Decimal values for lower bound and upper bound + /// are stored rounded, without fractional part. What is more strange + /// the integer part is rounded mathematically correctly according to fractional part. + /// Example: 17.22 -> 17, 8888.999 -> 8889, 1423.77 -> 1424. + /// I've checked two implementations: Spark and Amazon Athena and both of them + /// do this. + /// + /// The problem is -- we cannot use rounded values for lower bounds and upper bounds. + /// Example: upper_bound(x) = 17.22, but it's rounded 17.00, now condition WHERE x >= 17.21 will + /// check rounded value and say: "Oh largest value is 17, so values bigger than 17.21 cannot be in this file, + /// let's skip it". But it will produce incorrect result since actual value (17.22 >= 17.21) is stored in this file. + /// + /// To handle this issue we subtract 1 from the integral part for lower_bound and add 1 to integral + /// part of upper_bound. This produces: 17.22 -> [16.0, 18.0]. So this is more rough boundary, + /// but at least it doesn't lead to incorrect results. + { + int64_t scaler = lower_bound ? -10 : 10; + int32_t scale = DB::getDecimalScale(*non_nullable_type); + while (--scale) + scaler *= 10; + + unscaled_value += scaler; + } + + if (const auto * decimal_type = DB::checkDecimal(*non_nullable_type)) + { + DB::DecimalField result(unscaled_value, decimal_type->getScale()); + return result; + } + if (const auto * decimal_type = DB::checkDecimal(*non_nullable_type)) + { + DB::DecimalField result(unscaled_value, decimal_type->getScale()); + return result; + } + else + { + return std::nullopt; + } + } + else + { + /// For all other types except decimal binary representation + /// matches our internal representation + column->insertData(str.data(), str.length()); + DB::Field result; + column->get(0, result); + return result; + } + } + +} + constexpr const char * COLUMN_STATUS_NAME = "status"; constexpr const char * COLUMN_TUPLE_DATA_FILE_NAME = "data_file"; constexpr const char * COLUMN_SEQ_NUMBER_NAME = "sequence_number"; @@ -104,10 +186,9 @@ ManifestFileContent::ManifestFileContent( partition_key_ast->arguments->children.emplace_back(std::move(partition_ast)); partition_columns_description.emplace_back(numeric_column_name, removeNullable(manifest_file_column_characteristics.type)); - this->partition_column_ids.push_back(source_id); } - if (!partition_column_ids.empty()) + if (!partition_columns_description.empty()) this->partition_key_description.emplace(DB::KeyDescription::getKeyFromAST(std::move(partition_key_ast), ColumnsDescription(partition_columns_description), context)); for (size_t i = 0; i < manifest_file_deserializer.rows(); ++i) @@ -165,6 +246,7 @@ ManifestFileContent::ManifestFileContent( } } + std::unordered_map> value_for_bounds; for (const auto & path : {SUBCOLUMN_LOWER_BOUNDS_NAME, SUBCOLUMN_UPPER_BOUNDS_NAME}) { if (manifest_file_deserializer.hasPath(path)) @@ -175,14 +257,28 @@ ManifestFileContent::ManifestFileContent( const auto & column_number_and_bound = column_stats.safeGet(); Int32 number = column_number_and_bound[0].safeGet(); const Field & bound_value = column_number_and_bound[1]; + if (path == SUBCOLUMN_LOWER_BOUNDS_NAME) - columns_infos[number].lower_bound = bound_value; + value_for_bounds[number].first = bound_value; else - columns_infos[number].upper_bound = bound_value; + value_for_bounds[number].second = bound_value; + + column_ids_which_have_bounds.insert(number); } } } + for (const auto & [column_id, bounds] : value_for_bounds) + { + DB::NameAndTypePair name_and_type = schema_processor.getFieldCharacteristics(schema_id, column_id); + auto left = deserializeFieldFromBinaryRepr(bounds.first.safeGet(), name_and_type.type, true); + auto right = deserializeFieldFromBinaryRepr(bounds.second.safeGet(), name_and_type.type, false); + if (!left || !right) + continue; + + columns_infos[column_id].hyperrectangle.emplace(*left, true, *right, true); + } + FileEntry file = FileEntry{DataFileEntry{file_path}}; Int64 added_sequence_number = 0; @@ -215,7 +311,7 @@ ManifestFileContent::ManifestFileContent( bool ManifestFileContent::hasPartitionKey() const { - return !partition_column_ids.empty(); + return partition_key_description.has_value(); } const DB::KeyDescription & ManifestFileContent::getPartitionKeyDescription() const @@ -225,9 +321,14 @@ const DB::KeyDescription & ManifestFileContent::getPartitionKeyDescription() con return *(partition_key_description); } -const std::vector & ManifestFileContent::getPartitionKeyColumnIDs() const +bool ManifestFileContent::hasBoundsInfoInManifests() const +{ + return !column_ids_which_have_bounds.empty(); +} + +const std::set & ManifestFileContent::getColumnsIDsWithBounds() const { - return partition_column_ids; + return column_ids_which_have_bounds; } std::optional ManifestFileContent::getRowsCountInAllDataFilesExcludingDeleted() const diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h index 46d4fae5bc77..83b531400326 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -41,8 +42,7 @@ struct ColumnInfo std::optional rows_count; std::optional bytes_size; std::optional nulls_count; - std::optional lower_bound; - std::optional upper_bound; + std::optional hyperrectangle; }; using FileEntry = std::variant; // In the future we will add PositionalDeleteFileEntry and EqualityDeleteFileEntry here @@ -102,21 +102,24 @@ class ManifestFileContent bool hasPartitionKey() const; const DB::KeyDescription & getPartitionKeyDescription() const; - const std::vector & getPartitionKeyColumnIDs() const; /// Fields with rows count in manifest files are optional /// they can be absent. std::optional getRowsCountInAllDataFilesExcludingDeleted() const; std::optional getBytesCountInAllDataFiles() const; + + bool hasBoundsInfoInManifests() const; + const std::set & getColumnsIDsWithBounds() const; private: Int32 schema_id; std::optional partition_key_description; - std::vector partition_column_ids; // Size - number of files std::vector files; + std::set column_ids_which_have_bounds; + }; /// Once manifest file is constructed. It's unchangeable. diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFilesPruning.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFilesPruning.cpp new file mode 100644 index 000000000000..246af3f2d457 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFilesPruning.cpp @@ -0,0 +1,217 @@ +#include "config.h" + +#if USE_AVRO + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +using namespace DB; + +namespace ProfileEvents +{ + extern const Event IcebergPartitionPrunnedFiles; + extern const Event IcebergMinMaxIndexPrunnedFiles; +} + + +namespace Iceberg +{ + +DB::ASTPtr getASTFromTransform(const String & transform_name_src, const String & column_name) +{ + std::string transform_name = Poco::toLower(transform_name_src); + + if (transform_name == "year" || transform_name == "years") + return makeASTFunction("toYearNumSinceEpoch", std::make_shared(column_name)); + + if (transform_name == "month" || transform_name == "months") + return makeASTFunction("toMonthNumSinceEpoch", std::make_shared(column_name)); + + if (transform_name == "day" || transform_name == "date" || transform_name == "days" || transform_name == "dates") + return makeASTFunction("toRelativeDayNum", std::make_shared(column_name)); + + if (transform_name == "hour" || transform_name == "hours") + return makeASTFunction("toRelativeHourNum", std::make_shared(column_name)); + + if (transform_name == "identity") + return std::make_shared(column_name); + + if (transform_name == "void") + return makeASTFunction("tuple"); + + if (transform_name.starts_with("truncate")) + { + /// should look like transform[N] + + if (transform_name.back() != ']') + return nullptr; + + auto argument_start = transform_name.find('['); + + if (argument_start == std::string::npos) + return nullptr; + + auto argument_width = transform_name.length() - 2 - argument_start; + std::string width = transform_name.substr(argument_start + 1, argument_width); + size_t truncate_width; + bool parsed = DB::tryParse(truncate_width, width); + + if (!parsed) + return nullptr; + + return makeASTFunction("icebergTruncate", std::make_shared(truncate_width), std::make_shared(column_name)); + } + else + { + return nullptr; + } +} + +std::unique_ptr ManifestFilesPruner::transformFilterDagForManifest(const DB::ActionsDAG * source_dag, std::vector & used_columns_in_filter) const +{ + if (source_dag == nullptr) + return nullptr; + + const auto & inputs = source_dag->getInputs(); + + for (const auto & input : inputs) + { + if (input->type == ActionsDAG::ActionType::INPUT) + { + std::string input_name = input->result_name; + std::optional input_id = schema_processor.tryGetColumnIDByName(current_schema_id, input_name); + if (input_id) + used_columns_in_filter.push_back(*input_id); + } + } + + ActionsDAG dag_with_renames; + for (const auto column_id : used_columns_in_filter) + { + auto column = schema_processor.tryGetFieldCharacteristics(current_schema_id, column_id); + + /// Columns which we dropped and don't exist in current schema + /// cannot be queried in WHERE expression. + if (!column.has_value()) + continue; + + /// We take data type from manifest schema, not latest type + auto column_from_manifest = schema_processor.tryGetFieldCharacteristics(manifest_schema_id, column_id); + if (!column_from_manifest.has_value()) + continue; + + auto numeric_column_name = DB::backQuote(DB::toString(column_id)); + const auto * node = &dag_with_renames.addInput(numeric_column_name, column_from_manifest->type); + node = &dag_with_renames.addAlias(*node, column->name); + dag_with_renames.getOutputs().push_back(node); + } + auto result = std::make_unique(DB::ActionsDAG::merge(std::move(dag_with_renames), source_dag->clone())); + result->removeUnusedActions(); + return result; + +} + + +ManifestFilesPruner::ManifestFilesPruner( + const DB::IcebergSchemaProcessor & schema_processor_, + Int32 current_schema_id_, + const DB::ActionsDAG * filter_dag, + const ManifestFileContent & manifest_file, + DB::ContextPtr context) + : schema_processor(schema_processor_) + , current_schema_id(current_schema_id_) + , manifest_schema_id(manifest_file.getSchemaId()) +{ + std::unique_ptr transformed_dag; + std::vector used_columns_in_filter; + if (manifest_file.hasPartitionKey() || manifest_file.hasBoundsInfoInManifests()) + transformed_dag = transformFilterDagForManifest(filter_dag, used_columns_in_filter); + + if (manifest_file.hasPartitionKey()) + { + partition_key = &manifest_file.getPartitionKeyDescription(); + if (transformed_dag != nullptr) + partition_key_condition.emplace(transformed_dag.get(), context, partition_key->column_names, partition_key->expression, true /* single_point */); + } + + if (manifest_file.hasBoundsInfoInManifests() && transformed_dag != nullptr) + { + { + const auto & bounded_colums = manifest_file.getColumnsIDsWithBounds(); + for (Int32 used_column_id : used_columns_in_filter) + { + if (!bounded_colums.contains(used_column_id)) + continue; + + NameAndTypePair name_and_type = schema_processor.getFieldCharacteristics(manifest_schema_id, used_column_id); + name_and_type.name = DB::backQuote(DB::toString(used_column_id)); + + ExpressionActionsPtr expression = std::make_shared( + ActionsDAG({name_and_type}), ExpressionActionsSettings(context)); + + min_max_key_conditions.emplace(used_column_id, KeyCondition(transformed_dag.get(), context, {name_and_type.name}, expression)); + } + } + } +} + +bool ManifestFilesPruner::canBePruned(const ManifestFileEntry & entry) const +{ + if (partition_key_condition.has_value()) + { + const auto & partition_value = entry.partition_key_value; + std::vector index_value(partition_value.begin(), partition_value.end()); + for (auto & field : index_value) + { + // NULL_LAST + if (field.isNull()) + field = POSITIVE_INFINITY; + } + + bool can_be_true = partition_key_condition->mayBeTrueInRange( + partition_value.size(), index_value.data(), index_value.data(), partition_key->data_types); + + if (!can_be_true) + { + ProfileEvents::increment(ProfileEvents::IcebergPartitionPrunnedFiles); + return true; + } + } + + for (const auto & [column_id, key_condition] : min_max_key_conditions) + { + std::optional name_and_type = schema_processor.tryGetFieldCharacteristics(manifest_schema_id, column_id); + + /// There is no such column in this manifest file + if (!name_and_type.has_value()) + continue; + + auto hyperrectangle = entry.columns_infos.at(column_id).hyperrectangle; + if (hyperrectangle.has_value() && !key_condition.mayBeTrueInRange(1, &hyperrectangle->left, &hyperrectangle->right, {name_and_type->type})) + { + ProfileEvents::increment(ProfileEvents::IcebergMinMaxIndexPrunnedFiles); + return true; + } + } + + return false; +} + + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/PartitionPruning.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFilesPruning.h similarity index 72% rename from src/Storages/ObjectStorage/DataLakes/Iceberg/PartitionPruning.h rename to src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFilesPruning.h index 8b9fd80c612f..89a761265985 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/PartitionPruning.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFilesPruning.h @@ -20,20 +20,23 @@ class ManifestFileContent; DB::ASTPtr getASTFromTransform(const String & transform_name_src, const String & column_name); /// Prune specific data files based on manifest content -class PartitionPruner +class ManifestFilesPruner { private: const DB::IcebergSchemaProcessor & schema_processor; Int32 current_schema_id; + Int32 manifest_schema_id; const DB::KeyDescription * partition_key; + std::optional partition_key_condition; - std::optional key_condition; - /// NOTE: tricky part to support RENAME column in partition key. + std::unordered_map min_max_key_conditions; + /// NOTE: tricky part to support RENAME column. /// Takes ActionDAG representation of user's WHERE expression and - /// rename columns to the their origina numeric id's in iceberg - std::unique_ptr transformFilterDagForManifest(const DB::ActionsDAG * source_dag, Int32 manifest_schema_id, const std::vector & partition_column_ids) const; + /// rename columns to the their origina numeric ID's in iceberg + std::unique_ptr transformFilterDagForManifest(const DB::ActionsDAG * source_dag, std::vector & used_columns_in_filter) const; public: - PartitionPruner( + + ManifestFilesPruner( const DB::IcebergSchemaProcessor & schema_processor_, Int32 current_schema_id_, const DB::ActionsDAG * filter_dag, diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/PartitionPruning.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/PartitionPruning.cpp deleted file mode 100644 index 9e24494e071e..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/PartitionPruning.cpp +++ /dev/null @@ -1,144 +0,0 @@ -#include "config.h" - -#if USE_AVRO - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -using namespace DB; - -namespace Iceberg -{ - -DB::ASTPtr getASTFromTransform(const String & transform_name_src, const String & column_name) -{ - std::string transform_name = Poco::toLower(transform_name_src); - - if (transform_name == "year" || transform_name == "years") - return makeASTFunction("toYearNumSinceEpoch", std::make_shared(column_name)); - - if (transform_name == "month" || transform_name == "months") - return makeASTFunction("toMonthNumSinceEpoch", std::make_shared(column_name)); - - if (transform_name == "day" || transform_name == "date" || transform_name == "days" || transform_name == "dates") - return makeASTFunction("toRelativeDayNum", std::make_shared(column_name)); - - if (transform_name == "hour" || transform_name == "hours") - return makeASTFunction("toRelativeHourNum", std::make_shared(column_name)); - - if (transform_name == "identity") - return std::make_shared(column_name); - - if (transform_name == "void") - return makeASTFunction("tuple"); - - if (transform_name.starts_with("truncate")) - { - /// should look like transform[N] - - if (transform_name.back() != ']') - return nullptr; - - auto argument_start = transform_name.find('['); - - if (argument_start == std::string::npos) - return nullptr; - - auto argument_width = transform_name.length() - 2 - argument_start; - std::string width = transform_name.substr(argument_start + 1, argument_width); - size_t truncate_width; - bool parsed = DB::tryParse(truncate_width, width); - - if (!parsed) - return nullptr; - - return makeASTFunction("icebergTruncate", std::make_shared(truncate_width), std::make_shared(column_name)); - } - else - { - return nullptr; - } -} - -std::unique_ptr PartitionPruner::transformFilterDagForManifest(const DB::ActionsDAG * source_dag, Int32 manifest_schema_id, const std::vector & partition_column_ids) const -{ - if (source_dag == nullptr) - return nullptr; - - ActionsDAG dag_with_renames; - for (const auto column_id : partition_column_ids) - { - auto column = schema_processor.tryGetFieldCharacteristics(current_schema_id, column_id); - - /// Columns which we dropped and doesn't exist in current schema - /// cannot be queried in WHERE expression. - if (!column.has_value()) - continue; - - /// We take data type from manifest schema, not latest type - auto column_type = schema_processor.getFieldCharacteristics(manifest_schema_id, column_id).type; - auto numeric_column_name = DB::backQuote(DB::toString(column_id)); - const auto * node = &dag_with_renames.addInput(numeric_column_name, column_type); - node = &dag_with_renames.addAlias(*node, column->name); - dag_with_renames.getOutputs().push_back(node); - } - auto result = std::make_unique(DB::ActionsDAG::merge(std::move(dag_with_renames), source_dag->clone())); - result->removeUnusedActions(); - return result; - -} - -PartitionPruner::PartitionPruner( - const DB::IcebergSchemaProcessor & schema_processor_, - Int32 current_schema_id_, - const DB::ActionsDAG * filter_dag, - const ManifestFileContent & manifest_file, - DB::ContextPtr context) - : schema_processor(schema_processor_) - , current_schema_id(current_schema_id_) -{ - if (manifest_file.hasPartitionKey()) - { - partition_key = &manifest_file.getPartitionKeyDescription(); - auto transformed_dag = transformFilterDagForManifest(filter_dag, manifest_file.getSchemaId(), manifest_file.getPartitionKeyColumnIDs()); - if (transformed_dag != nullptr) - key_condition.emplace(transformed_dag.get(), context, partition_key->column_names, partition_key->expression, true /* single_point */); - } -} - -bool PartitionPruner::canBePruned(const ManifestFileEntry & entry) const -{ - if (!key_condition.has_value()) - return false; - - const auto & partition_value = entry.partition_key_value; - std::vector index_value(partition_value.begin(), partition_value.end()); - for (auto & field : index_value) - { - // NULL_LAST - if (field.isNull()) - field = POSITIVE_INFINITY; - } - - bool can_be_true = key_condition->mayBeTrueInRange( - partition_value.size(), index_value.data(), index_value.data(), partition_key->data_types); - - return !can_be_true; -} - - -} - -#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.cpp index 961d129f71ca..acd057ff4a97 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.cpp @@ -117,6 +117,7 @@ void IcebergSchemaProcessor::addIcebergTableSchema(Poco::JSON::Object::Ptr schem auto type = getFieldType(field, "type", required, current_full_name, true); clickhouse_schema->push_back(NameAndTypePair{name, type}); clickhouse_types_by_source_ids[{schema_id, field->getValue("id")}] = NameAndTypePair{current_full_name, type}; + clickhouse_ids_by_source_names[{schema_id, current_full_name}] = field->getValue("id"); } clickhouse_table_schemas_by_ids[schema_id] = clickhouse_schema; } @@ -139,6 +140,14 @@ std::optional IcebergSchemaProcessor::tryGetFieldCharacteristic return it->second; } +std::optional IcebergSchemaProcessor::tryGetColumnIDByName(Int32 schema_id, const std::string & name) const +{ + auto it = clickhouse_ids_by_source_names.find({schema_id, name}); + if (it == clickhouse_ids_by_source_names.end()) + return {}; + return it->second; +} + NamesAndTypesList IcebergSchemaProcessor::tryGetFieldsCharacteristics(Int32 schema_id, const std::vector & source_ids) const { NamesAndTypesList fields; @@ -232,6 +241,8 @@ IcebergSchemaProcessor::getComplexTypeFromObject(const Poco::JSON::Object::Ptr & element_types.push_back(getFieldType(field, "type", required, current_full_name, true)); clickhouse_types_by_source_ids[{current_schema_id.value(), field->getValue("id")}] = NameAndTypePair{current_full_name, element_types.back()}; + + clickhouse_ids_by_source_names[{current_schema_id.value(), current_full_name}] = field->getValue("id"); } else { diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h index de003456cef2..c0e9b57909b8 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h @@ -82,6 +82,7 @@ class IcebergSchemaProcessor NameAndTypePair getFieldCharacteristics(Int32 schema_version, Int32 source_id) const; std::optional tryGetFieldCharacteristics(Int32 schema_version, Int32 source_id) const; NamesAndTypesList tryGetFieldsCharacteristics(Int32 schema_id, const std::vector & source_ids) const; + std::optional tryGetColumnIDByName(Int32 schema_id, const std::string & name) const; bool hasClickhouseTableSchemaById(Int32 id) const; @@ -91,6 +92,7 @@ class IcebergSchemaProcessor std::unordered_map> clickhouse_table_schemas_by_ids; std::map, std::shared_ptr> transform_dags_by_ids; mutable std::map, NameAndTypePair> clickhouse_types_by_source_ids; + mutable std::map, Int32> clickhouse_ids_by_source_names; NamesAndTypesList getSchemaType(const Poco::JSON::Object::Ptr & schema); DataTypePtr getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type, String & current_full_name, bool is_subfield_of_root); From 3a57eb6b22fdf7c59e55e86c50e84839d158fc26 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 1 Apr 2025 20:54:54 +0000 Subject: [PATCH 06/14] Merge pull request #78368 from ClickHouse/delta-kernel-perf-issue delta-kernel: fix progress bar, fix performance --- src/Databases/DataLake/DatabaseDataLake.cpp | 1 + .../DataLakes/DataLakeConfiguration.h | 4 +- .../DataLakes/DeltaLake/TableSnapshot.cpp | 167 +++++++++++++++--- .../DataLakes/DeltaLake/TableSnapshot.h | 5 +- .../DeltaLake/getSchemaFromSnapshot.cpp | 7 + .../DeltaLake/getSchemaFromSnapshot.h | 2 + .../DataLakes/DeltaLakeMetadata.h | 9 +- .../DeltaLakeMetadataDeltaKernel.cpp | 8 +- .../DataLakes/DeltaLakeMetadataDeltaKernel.h | 20 ++- .../DataLakes/IDataLakeMetadata.h | 3 +- .../ObjectStorage/StorageObjectStorage.cpp | 8 +- .../ObjectStorage/StorageObjectStorage.h | 4 +- .../StorageObjectStorageSettings.cpp | 3 + .../StorageObjectStorageSource.cpp | 2 +- .../TableFunctionObjectStorage.cpp | 3 +- 15 files changed, 203 insertions(+), 43 deletions(-) diff --git a/src/Databases/DataLake/DatabaseDataLake.cpp b/src/Databases/DataLake/DatabaseDataLake.cpp index 687d59ac3884..0ebf0bc38c9a 100644 --- a/src/Databases/DataLake/DatabaseDataLake.cpp +++ b/src/Databases/DataLake/DatabaseDataLake.cpp @@ -417,6 +417,7 @@ StoragePtr DatabaseDataLake::tryGetTableImpl(const String & name, ContextPtr con LoadingStrictnessLevel::CREATE, /* distributed_processing */false, /* partition_by */nullptr, + /* is_table_function */false, /* lazy_init */true); } diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 7c1d82a2a10b..12c0319c291d 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -138,10 +138,10 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl return current_metadata->supportsFileIterator(); } - ObjectIterator iterate() override + ObjectIterator iterate(IDataLakeMetadata::FileProgressCallback callback, size_t list_batch_size) override { chassert(current_metadata); - return current_metadata->iterate(); + return current_metadata->iterate(callback, list_batch_size); } /// This is an awful temporary crutch, diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.cpp index 87cc421e5050..a47ab8fb0421 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.cpp @@ -10,6 +10,8 @@ #include #include #include +#include +#include #include #include "getSchemaFromSnapshot.h" #include "KernelUtils.h" @@ -61,19 +63,72 @@ class TableSnapshot::Iterator final : public DB::IObjectIterator const DB::NamesAndTypesList & schema_, const DB::Names & partition_columns_, DB::ObjectStoragePtr object_storage_, + DB::IDataLakeMetadata::FileProgressCallback callback_, + size_t list_batch_size_, LoggerPtr log_) - : scan(KernelUtils::unwrapResult(ffi::scan(snapshot_.get(), engine_.get(), /* predicate */{}), "scan")) - , scan_data_iterator(KernelUtils::unwrapResult( - ffi::kernel_scan_data_init(engine_.get(), scan.get()), - "kernel_scan_data_init")) + : engine(engine_) + , snapshot(snapshot_) , data_prefix(data_prefix_) , schema(schema_) , partition_columns(partition_columns_) , object_storage(object_storage_) + , callback(callback_) + , list_batch_size(list_batch_size_) , log(log_) + , thread([&, thread_group = DB::CurrentThread::getGroup()] { + /// Attach to current query thread group, to be able to + /// have query id in logs and metrics from scanDataFunc. + DB::ThreadGroupSwitcher switcher(thread_group, "TableSnapshot"); + scanDataFunc(); + }) { } + ~Iterator() override + { + shutdown.store(true); + schedule_next_batch_cv.notify_one(); + if (thread.joinable()) + thread.join(); + } + + void initScanState() + { + scan = KernelUtils::unwrapResult(ffi::scan(snapshot.get(), engine.get(), /* predicate */{}), "scan"); + scan_data_iterator = KernelUtils::unwrapResult( + ffi::kernel_scan_data_init(engine.get(), scan.get()), + "kernel_scan_data_init"); + } + + void scanDataFunc() + { + initScanState(); + while (!shutdown.load()) + { + bool have_scan_data_res = KernelUtils::unwrapResult( + ffi::kernel_scan_data_next(scan_data_iterator.get(), this, visitData), + "kernel_scan_data_next"); + + if (have_scan_data_res) + { + std::unique_lock lock(next_mutex); + if (!shutdown.load() && data_files.size() >= list_batch_size) + { + schedule_next_batch_cv.wait(lock, [&]() { return (data_files.size() < list_batch_size) || shutdown.load(); }); + } + } + else + { + { + std::lock_guard lock(next_mutex); + iterator_finished = true; + } + data_files_cv.notify_all(); + return; + } + } + } + size_t estimatedKeysCount() override { /// For now do the same as StorageObjectStorageSource::GlobIterator. @@ -83,23 +138,35 @@ class TableSnapshot::Iterator final : public DB::IObjectIterator DB::ObjectInfoPtr next(size_t) override { - std::lock_guard lock(next_mutex); - while (data_files.empty()) + DB::ObjectInfoPtr object; { - bool have_scan_data_res = KernelUtils::unwrapResult( - ffi::kernel_scan_data_next(scan_data_iterator.get(), this, visitData), - "kernel_scan_data_next"); + std::unique_lock lock(next_mutex); + if (!iterator_finished && data_files.empty()) + { + LOG_TEST(log, "Waiting for next data file"); + schedule_next_batch_cv.notify_one(); + data_files_cv.wait(lock, [&]() { return !data_files.empty() || iterator_finished; }); + } - if (!have_scan_data_res) + if (data_files.empty()) return nullptr; - } - chassert(!data_files.empty()); + LOG_TEST(log, "Current data files: {}", data_files.size()); - auto object = data_files.front(); - data_files.pop_front(); + object = data_files.front(); + data_files.pop_front(); + if (data_files.empty()) + schedule_next_batch_cv.notify_one(); + } chassert(object); + object->metadata = object_storage->getObjectMetadata(object->getPath()); + + if (callback) + { + chassert(object->metadata); + callback(DB::FileProgress(0, object->metadata->size_bytes)); + } return object; } @@ -161,39 +228,66 @@ class TableSnapshot::Iterator final : public DB::IObjectIterator "Scanned file: {}, size: {}, num records: {}, partition columns: {}", full_path, size, stats->num_records, partitions_info.size()); - auto metadata = context->object_storage->getObjectMetadata(full_path); DB::ObjectInfoPtr object; if (partitions_info.empty()) - object = std::make_shared(std::move(full_path), metadata); + object = std::make_shared(std::move(full_path)); else - object = std::make_shared(std::move(partitions_info), std::move(full_path), metadata); + object = std::make_shared(std::move(partitions_info), std::move(full_path)); - context->data_files.push_back(std::move(object)); + { + std::lock_guard lock(context->next_mutex); + context->data_files.push_back(std::move(object)); + } + context->data_files_cv.notify_one(); } private: using KernelScan = KernelPointerWrapper; using KernelScanDataIterator = KernelPointerWrapper; - const KernelScan scan; - const KernelScanDataIterator scan_data_iterator; + + const KernelExternEngine & engine; + const KernelSnapshot & snapshot; + KernelScan scan; + KernelScanDataIterator scan_data_iterator; + const std::string data_prefix; const DB::NamesAndTypesList & schema; const DB::Names & partition_columns; const DB::ObjectStoragePtr object_storage; + const DB::IDataLakeMetadata::FileProgressCallback callback; + const size_t list_batch_size; const LoggerPtr log; + /// Whether scanDataFunc should stop scanning. + /// Set in destructor. + std::atomic shutdown = false; + /// A CV to notify that new data_files are available. + std::condition_variable data_files_cv; + /// A flag meaning that all data files were scanned + /// and data scanning thread is finished. + bool iterator_finished = false; + + /// A CV to notify data scanning thread to continue, + /// as current data batch is fully read. + std::condition_variable schedule_next_batch_cv; + std::deque data_files; std::mutex next_mutex; + + /// A thread for async data scanning. + ThreadFromGlobalPool thread; }; TableSnapshot::TableSnapshot( KernelHelperPtr helper_, DB::ObjectStoragePtr object_storage_, + bool read_schema_same_as_table_schema_, LoggerPtr log_) : helper(helper_) , object_storage(object_storage_) + , read_schema_same_as_table_schema(read_schema_same_as_table_schema_) , log(log_) { } @@ -225,13 +319,15 @@ void TableSnapshot::initSnapshot() const void TableSnapshot::initSnapshotImpl() const { + LOG_TEST(log, "Initializing snapshot"); + auto * engine_builder = helper->createBuilder(); engine = KernelUtils::unwrapResult(ffi::builder_build(engine_builder), "builder_build"); snapshot = KernelUtils::unwrapResult( ffi::snapshot(KernelUtils::toDeltaString(helper->getTableLocation()), engine.get()), "snapshot"); snapshot_version = ffi::version(snapshot.get()); - LOG_TEST(log, "Snapshot version: {}", snapshot_version); + LOG_TRACE(log, "Snapshot version: {}", snapshot_version); } ffi::SharedSnapshot * TableSnapshot::getSnapshot() @@ -241,7 +337,7 @@ ffi::SharedSnapshot * TableSnapshot::getSnapshot() return snapshot.get(); } -DB::ObjectIterator TableSnapshot::iterate() +DB::ObjectIterator TableSnapshot::iterate(DB::IDataLakeMetadata::FileProgressCallback callback, size_t list_batch_size) { initSnapshot(); return std::make_shared( @@ -251,6 +347,8 @@ DB::ObjectIterator TableSnapshot::iterate() getTableSchema(), getPartitionColumns(), object_storage, + callback, + list_batch_size, log); } @@ -259,13 +357,16 @@ const DB::NamesAndTypesList & TableSnapshot::getTableSchema() if (!table_schema.has_value()) { table_schema = getTableSchemaFromSnapshot(getSnapshot()); - LOG_TEST(log, "Fetched table schema: {}", table_schema->toString()); + LOG_TRACE(log, "Fetched table schema"); + LOG_TEST(log, "Table schema: {}", table_schema->toString()); } return table_schema.value(); } const DB::NamesAndTypesList & TableSnapshot::getReadSchema() { + if (read_schema_same_as_table_schema) + return getTableSchema(); if (!read_schema.has_value()) loadReadSchemaAndPartitionColumns(); return read_schema.value(); @@ -282,11 +383,23 @@ void TableSnapshot::loadReadSchemaAndPartitionColumns() { auto * current_snapshot = getSnapshot(); chassert(engine.get()); - std::tie(read_schema, partition_columns) = getReadSchemaAndPartitionColumnsFromSnapshot(current_snapshot, engine.get()); + if (read_schema_same_as_table_schema) + { + partition_columns = getPartitionColumnsFromSnapshot(current_snapshot, engine.get()); + LOG_TRACE( + log, "Fetched partition columns: {}", + fmt::join(partition_columns.value(), ", ")); + } + else + { + std::tie(read_schema, partition_columns) = getReadSchemaAndPartitionColumnsFromSnapshot(current_snapshot, engine.get()); + LOG_TRACE( + log, "Fetched read schema and partition columns: {}", + fmt::join(partition_columns.value(), ", ")); + + LOG_TEST(log, "Read schema: {}", read_schema->toString()); + } - LOG_TEST( - log, "Fetched read schema: {}, partition columns: {}", - read_schema->toString(), fmt::join(partition_columns.value(), ", ")); } } diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h index dde7159a9e48..b7f4d7831d55 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h @@ -9,6 +9,7 @@ #include #include #include +#include #include "KernelPointerWrapper.h" #include "KernelHelper.h" #include @@ -29,6 +30,7 @@ class TableSnapshot explicit TableSnapshot( KernelHelperPtr helper_, DB::ObjectStoragePtr object_storage_, + bool read_schema_same_as_table_schema_, LoggerPtr log_); /// Get snapshot version. @@ -38,7 +40,7 @@ class TableSnapshot bool update(); /// Iterate over DeltaLake data files. - DB::ObjectIterator iterate(); + DB::ObjectIterator iterate(DB::IDataLakeMetadata::FileProgressCallback callback, size_t list_batch_size); /// Get schema from DeltaLake table metadata. const DB::NamesAndTypesList & getTableSchema(); @@ -59,6 +61,7 @@ class TableSnapshot const KernelHelperPtr helper; const DB::ObjectStoragePtr object_storage; + const bool read_schema_same_as_table_schema; const LoggerPtr log; mutable KernelExternEngine engine; diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/getSchemaFromSnapshot.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLake/getSchemaFromSnapshot.cpp index 6b74eeb932f8..53745f4b0a54 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/getSchemaFromSnapshot.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/getSchemaFromSnapshot.cpp @@ -462,6 +462,13 @@ getReadSchemaAndPartitionColumnsFromSnapshot(ffi::SharedSnapshot * snapshot, ffi return {data.getSchemaResult(), data.getPartitionColumns()}; } +DB::Names getPartitionColumnsFromSnapshot(ffi::SharedSnapshot * snapshot, ffi::SharedExternEngine * engine) +{ + SchemaVisitorData data; + SchemaVisitor::visitPartitionColumns(snapshot, engine, data); + return data.getPartitionColumns(); +} + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/getSchemaFromSnapshot.h b/src/Storages/ObjectStorage/DataLakes/DeltaLake/getSchemaFromSnapshot.h index d9c7f081657b..c7d511baa951 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/getSchemaFromSnapshot.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/getSchemaFromSnapshot.h @@ -27,6 +27,8 @@ DB::NamesAndTypesList getTableSchemaFromSnapshot(ffi::SharedSnapshot * snapshot) std::pair getReadSchemaAndPartitionColumnsFromSnapshot(ffi::SharedSnapshot * snapshot, ffi::SharedExternEngine * engine); +DB::Names getPartitionColumnsFromSnapshot(ffi::SharedSnapshot * snapshot, ffi::SharedExternEngine * engine); + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h index 3f3f43047a77..79db89b77c4e 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -18,6 +18,7 @@ namespace DB namespace StorageObjectStorageSetting { extern const StorageObjectStorageSettingsBool allow_experimental_delta_kernel_rs; +extern const StorageObjectStorageSettingsBool delta_lake_read_schema_same_as_table_schema; } struct DeltaLakePartitionColumn @@ -61,8 +62,12 @@ class DeltaLakeMetadata final : public IDataLakeMetadata { #if USE_DELTA_KERNEL_RS auto configuration_ptr = configuration.lock(); - if (configuration_ptr->getSettingsRef()[StorageObjectStorageSetting::allow_experimental_delta_kernel_rs]) - return std::make_unique(object_storage, configuration); + const auto & settings_ref = configuration_ptr->getSettingsRef(); + if (settings_ref[StorageObjectStorageSetting::allow_experimental_delta_kernel_rs]) + return std::make_unique( + object_storage, + configuration, + settings_ref[StorageObjectStorageSetting::delta_lake_read_schema_same_as_table_schema]); else return std::make_unique(object_storage, configuration, local_context); #else diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp index 9eee15c3646d..0492877c1ced 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp @@ -9,12 +9,14 @@ namespace DB DeltaLakeMetadataDeltaKernel::DeltaLakeMetadataDeltaKernel( ObjectStoragePtr object_storage, - ConfigurationObserverPtr configuration_) + ConfigurationObserverPtr configuration_, + bool read_schema_same_as_table_schema_) : log(getLogger("DeltaLakeMetadata")) , table_snapshot( std::make_shared( getKernelHelper(configuration_.lock(), object_storage), object_storage, + read_schema_same_as_table_schema_, log)) { } @@ -35,9 +37,9 @@ Strings DeltaLakeMetadataDeltaKernel::getDataFiles() const throwNotImplemented("getDataFiles()"); } -ObjectIterator DeltaLakeMetadataDeltaKernel::iterate() const +ObjectIterator DeltaLakeMetadataDeltaKernel::iterate(FileProgressCallback callback, size_t list_batch_size) const { - return table_snapshot->iterate(); + return table_snapshot->iterate(callback, list_batch_size); } NamesAndTypesList DeltaLakeMetadataDeltaKernel::getTableSchema() const diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h index 054bb194440c..708b822e3b35 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -18,6 +19,11 @@ class TableSnapshot; namespace DB { +namespace StorageObjectStorageSetting +{ +extern const StorageObjectStorageSettingsBool allow_experimental_delta_kernel_rs; +extern const StorageObjectStorageSettingsBool delta_lake_read_schema_same_as_table_schema; +} class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata { @@ -25,7 +31,10 @@ class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata using ConfigurationObserverPtr = StorageObjectStorage::ConfigurationObserverPtr; static constexpr auto name = "DeltaLake"; - DeltaLakeMetadataDeltaKernel(ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_); + DeltaLakeMetadataDeltaKernel( + ObjectStoragePtr object_storage_, + ConfigurationObserverPtr configuration_, + bool read_schema_same_as_table_schema_); bool supportsUpdate() const override { return true; } @@ -44,12 +53,17 @@ class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata ConfigurationObserverPtr configuration, ContextPtr, bool) { - return std::make_unique(object_storage, configuration); + auto configuration_ptr = configuration.lock(); + const auto & settings_ref = configuration_ptr->getSettingsRef(); + return std::make_unique( + object_storage, + configuration, + settings_ref[StorageObjectStorageSetting::delta_lake_read_schema_same_as_table_schema]); } bool supportsFileIterator() const override { return true; } - ObjectIterator iterate() const override; + ObjectIterator iterate(FileProgressCallback callback, size_t list_batch_size) const override; private: const LoggerPtr log; diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h index f2110c7a7b2e..c9cdd7ea5967 100644 --- a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h @@ -26,7 +26,8 @@ class IDataLakeMetadata : boost::noncopyable /// Whether `iterate()` method is supported for the data lake. virtual bool supportsFileIterator() const { return false; } /// Return iterator to `data files`. - virtual ObjectIterator iterate() const { throwNotImplemented("iterate()"); } + using FileProgressCallback = std::function; + virtual ObjectIterator iterate(FileProgressCallback /* callback */, size_t /* list_batch_size */) const { throwNotImplemented("iterate()"); } /// Table schema from data lake metadata. virtual NamesAndTypesList getTableSchema() const = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index aa0ab6dd2c67..c2ad873ebecb 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -90,6 +90,7 @@ StorageObjectStorage::StorageObjectStorage( LoadingStrictnessLevel mode, bool distributed_processing_, ASTPtr partition_by_, + bool is_table_function_, bool lazy_init) : IStorage(table_id_) , configuration(configuration_) @@ -100,6 +101,7 @@ StorageObjectStorage::StorageObjectStorage( , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { bool do_lazy_init = lazy_init && !columns_.empty() && !configuration->format.empty(); + update_configuration_on_read = !is_table_function_ || do_lazy_init; bool failed_init = false; auto do_init = [&]() { @@ -358,7 +360,11 @@ void StorageObjectStorage::read( size_t max_block_size, size_t num_streams) { - configuration->update(object_storage, local_context); + /// We did configuration->update() in constructor, + /// so in case of table function there is no need to do the same here again. + if (update_configuration_on_read) + configuration->update(object_storage, local_context); + if (partition_by && configuration->withPartitionWildcard()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 0fbeb2df431b..2383b20fdfe1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -70,6 +70,7 @@ class StorageObjectStorage : public IStorage LoadingStrictnessLevel mode, bool distributed_processing_ = false, ASTPtr partition_by_ = nullptr, + bool is_table_function_ = false, bool lazy_init = false); String getName() const override; @@ -156,6 +157,7 @@ class StorageObjectStorage : public IStorage const std::optional format_settings; const ASTPtr partition_by; const bool distributed_processing; + bool update_configuration_on_read; LoggerPtr log; }; @@ -248,7 +250,7 @@ class StorageObjectStorage::Configuration virtual std::optional tryGetTableStructureFromMetadata() const; virtual bool supportsFileIterator() const { return false; } - virtual ObjectIterator iterate() + virtual ObjectIterator iterate(std::function /* callback */, size_t /* list_batch_size */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method iterate() is not implemented for configuration type {}", getTypeName()); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSettings.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSettings.cpp index f59914cae3f3..84314fde2338 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSettings.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSettings.cpp @@ -19,6 +19,9 @@ If enabled, indicates that metadata is taken from iceberg specification that is )", 0) \ DECLARE(Bool, allow_experimental_delta_kernel_rs, false, R"( If enabled, the engine would use delta-kernel-rs for DeltaLake metadata parsing +)", 0) \ + DECLARE(Bool, delta_lake_read_schema_same_as_table_schema, false, R"( +Whether delta-lake read schema is the same as table schema. )", 0) \ DECLARE(String, iceberg_metadata_file_path, "", R"( Explicit path to desired Iceberg metadata file, should be relative to path in object storage. Make sense for table function use case only. diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index e398170493f6..33f877aae1b4 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -171,7 +171,7 @@ std::shared_ptr StorageObjectStorageSource::createFileIterator( } else if (configuration->supportsFileIterator()) { - return configuration->iterate(); + return configuration->iterate(file_progress_callback, query_settings.list_object_keys_size); } else { diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 27f84dd30142..5da24e93edeb 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -176,7 +176,8 @@ StoragePtr TableFunctionObjectStorage::executeImpl( /* format_settings */ std::nullopt, /* mode */ LoadingStrictnessLevel::CREATE, /* distributed_processing */ is_secondary_query, - /* partition_by */ nullptr); + /* partition_by */ nullptr, + /* is_table_function */true); storage->startup(); return storage; From 770b410efdc482d37e776f9100dab884e9c51140 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 3 Apr 2025 16:20:43 +0000 Subject: [PATCH 07/14] Merge pull request #78486 from ClickHouse/support-partition-pruning-in-delta-kernel Support partition pruning in DeltaLake engine --- src/Common/ProfileEvents.cpp | 2 + .../DataLakes/DataLakeConfiguration.h | 7 +- .../DataLakes/DeltaLake/PartitionPruner.cpp | 109 ++++++++++++++++++ .../DataLakes/DeltaLake/PartitionPruner.h | 36 ++++++ .../DataLakes/DeltaLake/TableSnapshot.cpp | 86 +++++++++----- .../DataLakes/DeltaLake/TableSnapshot.h | 21 ++-- .../DeltaLakeMetadataDeltaKernel.cpp | 7 +- .../DataLakes/DeltaLakeMetadataDeltaKernel.h | 5 +- .../DataLakes/IDataLakeMetadata.h | 5 +- .../ObjectStorage/StorageObjectStorage.cpp | 19 +-- .../ObjectStorage/StorageObjectStorage.h | 5 +- .../StorageObjectStorageCluster.cpp | 3 +- .../StorageObjectStorageSource.cpp | 6 +- .../StorageObjectStorageSource.h | 1 + 14 files changed, 257 insertions(+), 55 deletions(-) create mode 100644 src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.cpp create mode 100644 src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.h diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index a873aab3579d..df184b55f74c 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -220,6 +220,8 @@ M(JoinProbeTableRowCount, "Total number of rows in the probe table for a JOIN operation.", ValueType::Number) \ M(JoinResultRowCount, "Total number of rows in the result of a JOIN operation.", ValueType::Number) \ \ + M(DeltaLakePartitionPrunedFiles, "Number of skipped files during DeltaLake partition pruning", ValueType::Number) \ + \ M(SlowRead, "Number of reads from a file that were slow. This indicate system overload. Thresholds are controlled by read_backoff_* settings.", ValueType::Number) \ M(ReadBackoff, "Number of times the number of query processing threads was lowered due to slow reads.", ValueType::Number) \ \ diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 12c0319c291d..e15beb9cd33c 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -138,10 +138,13 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl return current_metadata->supportsFileIterator(); } - ObjectIterator iterate(IDataLakeMetadata::FileProgressCallback callback, size_t list_batch_size) override + ObjectIterator iterate( + const ActionsDAG * filter_dag, + IDataLakeMetadata::FileProgressCallback callback, + size_t list_batch_size) override { chassert(current_metadata); - return current_metadata->iterate(callback, list_batch_size); + return current_metadata->iterate(filter_dag, callback, list_batch_size); } /// This is an awful temporary crutch, diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.cpp new file mode 100644 index 000000000000..9a487b6cd617 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.cpp @@ -0,0 +1,109 @@ +#include "PartitionPruner.h" + +#if USE_DELTA_KERNEL_RS +#include + +#include +#include + +#include +#include + +#include +#include + + +namespace DB::ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DeltaLake +{ + +namespace +{ + DB::ASTPtr createPartitionKeyAST(const DB::Names & partition_columns) + { + /// DeltaLake supports only plain partition keys, + /// e.g. by column names without any functions. + + std::shared_ptr partition_key_ast = std::make_shared(); + partition_key_ast->name = "tuple"; + partition_key_ast->arguments = std::make_shared(); + partition_key_ast->children.push_back(partition_key_ast->arguments); + + for (const auto & column_name : partition_columns) + { + auto partition_ast = std::make_shared(column_name); + partition_key_ast->arguments->children.emplace_back(std::move(partition_ast)); + } + return partition_key_ast; + } + + DB::ColumnsDescription getPartitionColumnsDescription( + const DB::Names & partition_columns, + const DB::NamesAndTypesList & table_schema) + { + DB::NamesAndTypesList names_and_types; + for (const auto & column_name : partition_columns) + { + auto column = table_schema.tryGetByName(column_name); + if (!column.has_value()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Not found partition column in schema: {}", column_name); + names_and_types.emplace_back(column_name, removeNullable(column->type)); + } + return DB::ColumnsDescription(names_and_types); + } +} + +PartitionPruner::PartitionPruner( + const DB::ActionsDAG & filter_dag, + const DB::NamesAndTypesList & table_schema_, + const DB::Names & partition_columns_, + DB::ContextPtr context) +{ + if (!partition_columns_.empty()) + { + const auto partition_columns_description = getPartitionColumnsDescription(partition_columns_, table_schema_); + const auto partition_key_ast = createPartitionKeyAST(partition_columns_); + + partition_key = DB::KeyDescription::getKeyFromAST( + partition_key_ast, + partition_columns_description, + context); + + key_condition.emplace( + &filter_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */); + } +} + +bool PartitionPruner::canBePruned(const DB::ObjectInfoWithPartitionColumns & object_info) const +{ + if (!key_condition.has_value()) + return false; + + DB::Row partition_key_values; + partition_key_values.reserve(object_info.partitions_info.size()); + + for (const auto & [name_and_type, value] : object_info.partitions_info) + { + if (value.isNull()) + partition_key_values.push_back(DB::POSITIVE_INFINITY); /// NULL_LAST + else + partition_key_values.push_back(value); + } + + std::vector partition_key_values_ref(partition_key_values.begin(), partition_key_values.end()); + bool can_be_true = key_condition->mayBeTrueInRange( + partition_key_values_ref.size(), + partition_key_values_ref.data(), + partition_key_values_ref.data(), + partition_key.data_types); + + return !can_be_true; +} + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.h b/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.h new file mode 100644 index 000000000000..bb64aa64aebe --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.h @@ -0,0 +1,36 @@ +#pragma once +#include "config.h" +#include +#include + +#if USE_DELTA_KERNEL_RS + +namespace DB +{ +class ActionsDAG; +class NamesAndTypesList; +using Names = std::vector; +} + +namespace DeltaLake +{ + +class PartitionPruner +{ +public: + PartitionPruner( + const DB::ActionsDAG & filter_dag, + const DB::NamesAndTypesList & table_schema_, + const DB::Names & partition_columns_, + DB::ContextPtr context); + + bool canBePruned(const DB::ObjectInfoWithPartitionColumns & object_info) const; + +private: + std::optional key_condition; + DB::KeyDescription partition_key; +}; + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.cpp index a47ab8fb0421..fb791b83dded 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.cpp @@ -14,8 +14,8 @@ #include #include #include "getSchemaFromSnapshot.h" +#include "PartitionPruner.h" #include "KernelUtils.h" - #include namespace fs = std::filesystem; @@ -26,6 +26,11 @@ namespace DB::ErrorCodes extern const int NOT_IMPLEMENTED; } +namespace ProfileEvents +{ + extern const Event DeltaLakePartitionPrunedFiles; +} + namespace DB { @@ -63,6 +68,7 @@ class TableSnapshot::Iterator final : public DB::IObjectIterator const DB::NamesAndTypesList & schema_, const DB::Names & partition_columns_, DB::ObjectStoragePtr object_storage_, + const DB::ActionsDAG * filter_dag_, DB::IDataLakeMetadata::FileProgressCallback callback_, size_t list_batch_size_, LoggerPtr log_) @@ -82,6 +88,8 @@ class TableSnapshot::Iterator final : public DB::IObjectIterator scanDataFunc(); }) { + if (filter_dag_) + pruner.emplace(*filter_dag_, schema_, partition_columns_, DB::Context::getGlobalContextInstance()); } ~Iterator() override @@ -138,36 +146,51 @@ class TableSnapshot::Iterator final : public DB::IObjectIterator DB::ObjectInfoPtr next(size_t) override { - DB::ObjectInfoPtr object; + while (true) { - std::unique_lock lock(next_mutex); - if (!iterator_finished && data_files.empty()) + DB::ObjectInfoPtr object; { - LOG_TEST(log, "Waiting for next data file"); - schedule_next_batch_cv.notify_one(); - data_files_cv.wait(lock, [&]() { return !data_files.empty() || iterator_finished; }); - } + std::unique_lock lock(next_mutex); + if (!iterator_finished && data_files.empty()) + { + LOG_TEST(log, "Waiting for next data file"); + schedule_next_batch_cv.notify_one(); + data_files_cv.wait(lock, [&]() { return !data_files.empty() || iterator_finished; }); + } - if (data_files.empty()) - return nullptr; + if (data_files.empty()) + return nullptr; - LOG_TEST(log, "Current data files: {}", data_files.size()); + LOG_TEST(log, "Current data files: {}", data_files.size()); - object = data_files.front(); - data_files.pop_front(); - if (data_files.empty()) - schedule_next_batch_cv.notify_one(); - } + object = data_files.front(); + data_files.pop_front(); + if (data_files.empty()) + schedule_next_batch_cv.notify_one(); + } - chassert(object); - object->metadata = object_storage->getObjectMetadata(object->getPath()); + chassert(object); + if (pruner.has_value()) + { + const auto * object_with_partition_info = dynamic_cast(object.get()); + if (object_with_partition_info && pruner->canBePruned(*object_with_partition_info)) + { + ProfileEvents::increment(ProfileEvents::DeltaLakePartitionPrunedFiles); - if (callback) - { - chassert(object->metadata); - callback(DB::FileProgress(0, object->metadata->size_bytes)); + LOG_TEST(log, "Skipping file {} according to partition pruning", object->getPath()); + continue; + } + } + + object->metadata = object_storage->getObjectMetadata(object->getPath()); + + if (callback) + { + chassert(object->metadata); + callback(DB::FileProgress(0, object->metadata->size_bytes)); + } + return object; } - return object; } static void visitData( @@ -250,6 +273,7 @@ class TableSnapshot::Iterator final : public DB::IObjectIterator const KernelSnapshot & snapshot; KernelScan scan; KernelScanDataIterator scan_data_iterator; + std::optional pruner; const std::string data_prefix; const DB::NamesAndTypesList & schema; @@ -330,14 +354,17 @@ void TableSnapshot::initSnapshotImpl() const LOG_TRACE(log, "Snapshot version: {}", snapshot_version); } -ffi::SharedSnapshot * TableSnapshot::getSnapshot() +ffi::SharedSnapshot * TableSnapshot::getSnapshot() const { if (!snapshot.get()) initSnapshot(); return snapshot.get(); } -DB::ObjectIterator TableSnapshot::iterate(DB::IDataLakeMetadata::FileProgressCallback callback, size_t list_batch_size) +DB::ObjectIterator TableSnapshot::iterate( + const DB::ActionsDAG * filter_dag, + DB::IDataLakeMetadata::FileProgressCallback callback, + size_t list_batch_size) { initSnapshot(); return std::make_shared( @@ -347,12 +374,13 @@ DB::ObjectIterator TableSnapshot::iterate(DB::IDataLakeMetadata::FileProgressCal getTableSchema(), getPartitionColumns(), object_storage, + filter_dag, callback, list_batch_size, log); } -const DB::NamesAndTypesList & TableSnapshot::getTableSchema() +const DB::NamesAndTypesList & TableSnapshot::getTableSchema() const { if (!table_schema.has_value()) { @@ -363,7 +391,7 @@ const DB::NamesAndTypesList & TableSnapshot::getTableSchema() return table_schema.value(); } -const DB::NamesAndTypesList & TableSnapshot::getReadSchema() +const DB::NamesAndTypesList & TableSnapshot::getReadSchema() const { if (read_schema_same_as_table_schema) return getTableSchema(); @@ -372,14 +400,14 @@ const DB::NamesAndTypesList & TableSnapshot::getReadSchema() return read_schema.value(); } -const DB::Names & TableSnapshot::getPartitionColumns() +const DB::Names & TableSnapshot::getPartitionColumns() const { if (!partition_columns.has_value()) loadReadSchemaAndPartitionColumns(); return partition_columns.value(); } -void TableSnapshot::loadReadSchemaAndPartitionColumns() +void TableSnapshot::loadReadSchemaAndPartitionColumns() const { auto * current_snapshot = getSnapshot(); chassert(engine.get()); diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h index b7f4d7831d55..7b83c993be5d 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h @@ -40,18 +40,21 @@ class TableSnapshot bool update(); /// Iterate over DeltaLake data files. - DB::ObjectIterator iterate(DB::IDataLakeMetadata::FileProgressCallback callback, size_t list_batch_size); + DB::ObjectIterator iterate( + const DB::ActionsDAG * filter_dag, + DB::IDataLakeMetadata::FileProgressCallback callback, + size_t list_batch_size); /// Get schema from DeltaLake table metadata. - const DB::NamesAndTypesList & getTableSchema(); + const DB::NamesAndTypesList & getTableSchema() const; /// Get read schema derived from data files. /// (In most cases it would be the same as table schema). - const DB::NamesAndTypesList & getReadSchema(); + const DB::NamesAndTypesList & getReadSchema() const; /// DeltaLake stores partition columns values not in the data files, /// but in data file path directory names. /// Therefore "table schema" would contain partition columns, /// but "read schema" would not. - const DB::Names & getPartitionColumns(); + const DB::Names & getPartitionColumns() const; private: class Iterator; @@ -69,16 +72,16 @@ class TableSnapshot mutable KernelScan scan; mutable size_t snapshot_version; - std::optional table_schema; - std::optional read_schema; - std::optional partition_columns; + mutable std::optional table_schema; + mutable std::optional read_schema; + mutable std::optional partition_columns; void initSnapshot() const; void initSnapshotImpl() const; /// Both read schema and partition columns are loaded with the same data scan object, /// therefore we load them together. - void loadReadSchemaAndPartitionColumns(); - ffi::SharedSnapshot * getSnapshot(); + void loadReadSchemaAndPartitionColumns() const; + ffi::SharedSnapshot * getSnapshot() const; }; /// TODO; Enable event tracing in DeltaKernel. diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp index 0492877c1ced..1fa3f6636729 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp @@ -37,9 +37,12 @@ Strings DeltaLakeMetadataDeltaKernel::getDataFiles() const throwNotImplemented("getDataFiles()"); } -ObjectIterator DeltaLakeMetadataDeltaKernel::iterate(FileProgressCallback callback, size_t list_batch_size) const +ObjectIterator DeltaLakeMetadataDeltaKernel::iterate( + const ActionsDAG * filter_dag, + FileProgressCallback callback, + size_t list_batch_size) const { - return table_snapshot->iterate(callback, list_batch_size); + return table_snapshot->iterate(filter_dag, callback, list_batch_size); } NamesAndTypesList DeltaLakeMetadataDeltaKernel::getTableSchema() const diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h index 708b822e3b35..5b62a3da08f7 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h @@ -63,7 +63,10 @@ class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata bool supportsFileIterator() const override { return true; } - ObjectIterator iterate(FileProgressCallback callback, size_t list_batch_size) const override; + ObjectIterator iterate( + const ActionsDAG * filter_dag, + FileProgressCallback callback, + size_t list_batch_size) const override; private: const LoggerPtr log; diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h index c9cdd7ea5967..f8e6661d7e56 100644 --- a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h @@ -27,7 +27,10 @@ class IDataLakeMetadata : boost::noncopyable virtual bool supportsFileIterator() const { return false; } /// Return iterator to `data files`. using FileProgressCallback = std::function; - virtual ObjectIterator iterate(FileProgressCallback /* callback */, size_t /* list_batch_size */) const { throwNotImplemented("iterate()"); } + virtual ObjectIterator iterate( + const ActionsDAG * /* filter_dag */, + FileProgressCallback /* callback */, + size_t /* list_batch_size */) const { throwNotImplemented("iterate()"); } /// Table schema from data lake metadata. virtual NamesAndTypesList getTableSchema() const = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index c2ad873ebecb..2886049a1acf 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -65,6 +65,7 @@ String StorageObjectStorage::getPathSample(ContextPtr context) local_distributed_processing, context, {}, // predicate + {}, {}, // virtual_columns nullptr, // read_keys {} // file_progress_callback @@ -257,21 +258,17 @@ class ReadFromObjectStorageStep : public SourceStepWithFilter void applyFilters(ActionDAGNodes added_filter_nodes) override { SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag.has_value()) { - predicate = filter_actions_dag->getOutputs().at(0); if (getContext()->getSettingsRef()[Setting::use_iceberg_partition_pruning]) - { configuration->implementPartitionPruning(*filter_actions_dag); - } } - createIterator(predicate); + createIterator(); } void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override { - createIterator(nullptr); + createIterator(); Pipes pipes; auto context = getContext(); @@ -323,14 +320,19 @@ class ReadFromObjectStorageStep : public SourceStepWithFilter size_t num_streams; const bool distributed_processing; - void createIterator(const ActionsDAG::Node * predicate) + void createIterator() { if (iterator_wrapper) return; + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag.has_value()) + predicate = filter_actions_dag->getOutputs().at(0); + auto context = getContext(); iterator_wrapper = StorageObjectStorageSource::createFileIterator( configuration, configuration->getQuerySettings(context), object_storage, distributed_processing, - context, predicate, virtual_columns, nullptr, context->getFileProgressCallback()); + context, predicate, filter_actions_dag, virtual_columns, nullptr, context->getFileProgressCallback()); } }; } @@ -495,6 +497,7 @@ std::unique_ptr StorageObjectStorage::createReadBufferIterat false/* distributed_processing */, context, {}/* predicate */, + {}, {}/* virtual_columns */, &read_keys); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 2383b20fdfe1..5894d0efd394 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -250,7 +250,10 @@ class StorageObjectStorage::Configuration virtual std::optional tryGetTableStructureFromMetadata() const; virtual bool supportsFileIterator() const { return false; } - virtual ObjectIterator iterate(std::function /* callback */, size_t /* list_batch_size */) + virtual ObjectIterator iterate( + const ActionsDAG * /* filter_dag */, + std::function /* callback */, + size_t /* list_batch_size */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method iterate() is not implemented for configuration type {}", getTypeName()); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 8fabcc368a5e..0ee18533529d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -39,6 +39,7 @@ String StorageObjectStorageCluster::getPathSample(StorageInMemoryMetadata metada false, // distributed_processing context, {}, // predicate + {}, metadata.getColumns().getAll(), // virtual_columns nullptr, // read_keys {} // file_progress_callback @@ -148,7 +149,7 @@ RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExten { auto iterator = StorageObjectStorageSource::createFileIterator( configuration, configuration->getQuerySettings(local_context), object_storage, /* distributed_processing */false, - local_context, predicate, virtual_columns, nullptr, local_context->getFileProgressCallback(), /*ignore_archive_globs=*/true); + local_context, predicate, {}, virtual_columns, nullptr, local_context->getFileProgressCallback(), /*ignore_archive_globs=*/true); auto callback = std::make_shared>([iterator]() mutable -> String { diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 33f877aae1b4..cef27f90a925 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -127,6 +127,7 @@ std::shared_ptr StorageObjectStorageSource::createFileIterator( bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, + const std::optional & filter_actions_dag, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, std::function file_progress_callback, @@ -171,7 +172,10 @@ std::shared_ptr StorageObjectStorageSource::createFileIterator( } else if (configuration->supportsFileIterator()) { - return configuration->iterate(file_progress_callback, query_settings.list_object_keys_size); + return configuration->iterate( + filter_actions_dag.has_value() ? &filter_actions_dag.value() : nullptr, + file_progress_callback, + query_settings.list_object_keys_size); } else { diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index f99192caea15..40c01eea5261 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -54,6 +54,7 @@ class StorageObjectStorageSource : public SourceWithKeyCondition bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, + const std::optional & filter_actions_dag, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, std::function file_progress_callback = {}, From 5e9f4f54c84cf285ce0596aebdba382f802c5d5d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 7 Apr 2025 15:22:38 +0000 Subject: [PATCH 08/14] Merge pull request #78775 from ClickHouse/refactor-code-around-data-lakes Small refactoring around data lakes --- .../DataLakes/DataLakeConfiguration.h | 23 +------ .../DataLakes/DeltaLakeMetadata.cpp | 9 +++ .../DataLakes/DeltaLakeMetadata.h | 11 +++- .../DeltaLakeMetadataDeltaKernel.cpp | 5 -- .../DataLakes/DeltaLakeMetadataDeltaKernel.h | 4 -- .../ObjectStorage/DataLakes/HudiMetadata.cpp | 10 ++- .../ObjectStorage/DataLakes/HudiMetadata.h | 9 ++- .../DataLakes/IDataLakeMetadata.cpp | 63 +++++++++++++++++++ .../DataLakes/IDataLakeMetadata.h | 16 +++-- .../DataLakes/Iceberg/IcebergMetadata.cpp | 32 +++++----- .../DataLakes/Iceberg/IcebergMetadata.h | 20 +++--- .../ObjectStorage/StorageObjectStorage.cpp | 6 -- .../ObjectStorage/StorageObjectStorage.h | 2 - 13 files changed, 132 insertions(+), 78 deletions(-) create mode 100644 src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.cpp diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index e15beb9cd33c..20a0530d1a71 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -64,8 +64,6 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl ErrorCodes::FORMAT_VERSION_TOO_OLD, "Metadata is not consinsent with the one which was used to infer table schema. Please, retry the query."); } - if (!supportsFileIterator()) - BaseStorageConfiguration::setPaths(current_metadata->getDataFiles()); } } @@ -81,14 +79,6 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl return std::nullopt; } - void implementPartitionPruning(const ActionsDAG & filter_dag) override - { - if (!current_metadata || !current_metadata->supportsPartitionPruning()) - return; - BaseStorageConfiguration::setPaths(current_metadata->makePartitionPruning(filter_dag)); - } - - std::optional totalRows() override { if (!current_metadata) @@ -123,20 +113,11 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl ContextPtr context) override { BaseStorageConfiguration::update(object_storage, context); - if (updateMetadataObjectIfNeeded(object_storage, context)) - { - if (!supportsFileIterator()) - BaseStorageConfiguration::setPaths(current_metadata->getDataFiles()); - } - + updateMetadataObjectIfNeeded(object_storage, context); return ColumnsDescription{current_metadata->getTableSchema()}; } - bool supportsFileIterator() const override - { - chassert(current_metadata); - return current_metadata->supportsFileIterator(); - } + bool supportsFileIterator() const override { return true; } ObjectIterator iterate( const ActionsDAG * filter_dag, diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 0ac8361c94db..7e98a26ae96c 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -600,6 +600,7 @@ DeltaLakeMetadata::DeltaLakeMetadata(ObjectStoragePtr object_storage_, Configura data_files = result.data_files; schema = result.schema; partition_columns = result.partition_columns; + object_storage = object_storage_; LOG_TRACE(impl.log, "Found {} data files, {} partition files, schema: {}", data_files.size(), partition_columns.size(), schema.toString()); @@ -712,6 +713,14 @@ Field DeltaLakeMetadata::getFieldValue(const String & value, DataTypePtr data_ty throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported DeltaLake type for {}", check_type->getColumnType()); } +ObjectIterator DeltaLakeMetadata::iterate( + const ActionsDAG * filter_dag, + FileProgressCallback callback, + size_t /* list_batch_size */) const +{ + return createKeysIterator(getDataFiles(filter_dag), object_storage, callback); +} + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h index 79db89b77c4e..7835466e5490 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -41,8 +41,6 @@ class DeltaLakeMetadata final : public IDataLakeMetadata DeltaLakeMetadata(ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_, ContextPtr context_); - Strings getDataFiles() const override { return data_files; } - NamesAndTypesList getTableSchema() const override { return schema; } DeltaLakePartitionColumns getPartitionColumns() const { return partition_columns; } @@ -80,10 +78,19 @@ class DeltaLakeMetadata final : public IDataLakeMetadata static DataTypePtr getFieldValue(const Poco::JSON::Object::Ptr & field, const String & type_key, bool is_nullable); static Field getFieldValue(const String & value, DataTypePtr data_type); +protected: + ObjectIterator iterate( + const ActionsDAG * filter_dag, + FileProgressCallback callback, + size_t list_batch_size) const override; + private: mutable Strings data_files; NamesAndTypesList schema; DeltaLakePartitionColumns partition_columns; + ObjectStoragePtr object_storage; + + Strings getDataFiles(const ActionsDAG *) const { return data_files; } }; } diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp index 1fa3f6636729..dd411f9890d2 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.cpp @@ -32,11 +32,6 @@ bool DeltaLakeMetadataDeltaKernel::update(const ContextPtr &) return table_snapshot->update(); } -Strings DeltaLakeMetadataDeltaKernel::getDataFiles() const -{ - throwNotImplemented("getDataFiles()"); -} - ObjectIterator DeltaLakeMetadataDeltaKernel::iterate( const ActionsDAG * filter_dag, FileProgressCallback callback, diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h index 5b62a3da08f7..ececd3d73e7e 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h @@ -40,8 +40,6 @@ class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata bool update(const ContextPtr & context) override; - Strings getDataFiles() const override; - NamesAndTypesList getTableSchema() const override; NamesAndTypesList getReadSchema() const override; @@ -61,8 +59,6 @@ class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata settings_ref[StorageObjectStorageSetting::delta_lake_read_schema_same_as_table_schema]); } - bool supportsFileIterator() const override { return true; } - ObjectIterator iterate( const ActionsDAG * filter_dag, FileProgressCallback callback, diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp index 77ef769ed0e9..d8deea67cb12 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp @@ -91,11 +91,19 @@ HudiMetadata::HudiMetadata(ObjectStoragePtr object_storage_, ConfigurationObserv { } -Strings HudiMetadata::getDataFiles() const +Strings HudiMetadata::getDataFiles(const ActionsDAG *) const { if (data_files.empty()) data_files = getDataFilesImpl(); return data_files; } +ObjectIterator HudiMetadata::iterate( + const ActionsDAG * filter_dag, + FileProgressCallback callback, + size_t /* list_batch_size */) const +{ + return createKeysIterator(getDataFiles(filter_dag), object_storage, callback); +} + } diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h index 46291d9e6d96..a64ecfeb55dd 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h @@ -19,8 +19,6 @@ class HudiMetadata final : public IDataLakeMetadata, private WithContext HudiMetadata(ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_, ContextPtr context_); - Strings getDataFiles() const override; - NamesAndTypesList getTableSchema() const override { return {}; } bool operator ==(const IDataLakeMetadata & other) const override @@ -39,12 +37,19 @@ class HudiMetadata final : public IDataLakeMetadata, private WithContext return std::make_unique(object_storage, configuration, local_context); } +protected: + ObjectIterator iterate( + const ActionsDAG * filter_dag, + FileProgressCallback callback, + size_t list_batch_size) const override; + private: const ObjectStoragePtr object_storage; const ConfigurationObserverPtr configuration; mutable Strings data_files; Strings getDataFilesImpl() const; + Strings getDataFiles(const ActionsDAG * filter_dag) const; }; } diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.cpp new file mode 100644 index 000000000000..61f31766a455 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.cpp @@ -0,0 +1,63 @@ +#include "IDataLakeMetadata.h" +#include + +namespace DB +{ + +namespace +{ + +class KeysIterator : public IObjectIterator +{ +public: + KeysIterator( + Strings && data_files_, + ObjectStoragePtr object_storage_, + IDataLakeMetadata::FileProgressCallback callback_) + : data_files(data_files_) + , object_storage(object_storage_) + , callback(callback_) + { + } + + size_t estimatedKeysCount() override + { + return data_files.size(); + } + + ObjectInfoPtr next(size_t) override + { + while (true) + { + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= data_files.size()) + return nullptr; + + auto key = data_files[current_index]; + auto object_metadata = object_storage->getObjectMetadata(key); + + if (callback) + callback(FileProgress(0, object_metadata.size_bytes)); + + return std::make_shared(key, std::move(object_metadata)); + } + } + +private: + Strings data_files; + ObjectStoragePtr object_storage; + std::atomic index = 0; + IDataLakeMetadata::FileProgressCallback callback; +}; + +} + +ObjectIterator IDataLakeMetadata::createKeysIterator( + Strings && data_files_, + ObjectStoragePtr object_storage_, + IDataLakeMetadata::FileProgressCallback callback_) const +{ + return std::make_shared(std::move(data_files_), object_storage_, callback_); +} + +} diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h index f8e6661d7e56..fe1fa151b9a0 100644 --- a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h @@ -20,17 +20,12 @@ class IDataLakeMetadata : boost::noncopyable virtual bool operator==(const IDataLakeMetadata & other) const = 0; - /// List all data files. - /// For better parallelization, iterate() method should be used. - virtual Strings getDataFiles() const = 0; - /// Whether `iterate()` method is supported for the data lake. - virtual bool supportsFileIterator() const { return false; } /// Return iterator to `data files`. using FileProgressCallback = std::function; virtual ObjectIterator iterate( const ActionsDAG * /* filter_dag */, FileProgressCallback /* callback */, - size_t /* list_batch_size */) const { throwNotImplemented("iterate()"); } + size_t /* list_batch_size */) const = 0; /// Table schema from data lake metadata. virtual NamesAndTypesList getTableSchema() const = 0; @@ -39,9 +34,6 @@ class IDataLakeMetadata : boost::noncopyable /// Return nothing if read schema is the same as table schema. virtual NamesAndTypesList getReadSchema() const { return {}; } - virtual bool supportsPartitionPruning() { return false; } - virtual Strings makePartitionPruning(const ActionsDAG &) { throwNotImplemented("makePartitionPrunning()"); } - virtual std::shared_ptr getInitialSchemaByPath(const String &) const { return {}; } virtual std::shared_ptr getSchemaTransformer(const String &) const { return {}; } @@ -56,7 +48,13 @@ class IDataLakeMetadata : boost::noncopyable virtual std::optional totalRows() const { return {}; } virtual std::optional totalBytes() const { return {}; } + protected: + ObjectIterator createKeysIterator( + Strings && data_files_, + ObjectStoragePtr object_storage_, + IDataLakeMetadata::FileProgressCallback callback_) const; + [[noreturn]] void throwNotImplemented(std::string_view method) const { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method `{}` is not implemented", method); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index aedba7369ecd..505dd25a536a 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -49,6 +49,7 @@ namespace Setting { extern const SettingsInt64 iceberg_timestamp_ms; extern const SettingsInt64 iceberg_snapshot_id; +extern const SettingsBool use_iceberg_partition_pruning; } @@ -557,18 +558,23 @@ ManifestListPtr IcebergMetadata::getManifestList(const String & filename) const return manifest_list_ptr; } -Strings IcebergMetadata::getDataFilesImpl(const ActionsDAG * filter_dag) const +Strings IcebergMetadata::getDataFiles(const ActionsDAG * filter_dag) const { if (!relevant_snapshot) return {}; - if (!filter_dag && cached_unprunned_files_for_last_processed_snapshot.has_value()) + bool use_partition_pruning = filter_dag && getContext()->getSettingsRef()[Setting::use_iceberg_partition_pruning]; + + if (!use_partition_pruning && cached_unprunned_files_for_last_processed_snapshot.has_value()) return cached_unprunned_files_for_last_processed_snapshot.value(); Strings data_files; for (const auto & manifest_file_ptr : *(relevant_snapshot->manifest_list)) { - ManifestFilesPruner pruner(schema_processor, relevant_snapshot_schema_id, filter_dag, *manifest_file_ptr, getContext()); + ManifestFilesPruner pruner( + schema_processor, relevant_snapshot_schema_id, + use_partition_pruning ? filter_dag : nullptr, + *manifest_file_ptr, getContext()); const auto & data_files_in_manifest = manifest_file_ptr->getFiles(); for (const auto & manifest_file_entry : data_files_in_manifest) { @@ -583,7 +589,7 @@ Strings IcebergMetadata::getDataFilesImpl(const ActionsDAG * filter_dag) const } } - if (!filter_dag) + if (!use_partition_pruning) { cached_unprunned_files_for_last_processed_snapshot = data_files; return cached_unprunned_files_for_last_processed_snapshot.value(); @@ -592,16 +598,6 @@ Strings IcebergMetadata::getDataFilesImpl(const ActionsDAG * filter_dag) const return data_files; } -Strings IcebergMetadata::makePartitionPruning(const ActionsDAG & filter_dag) -{ - auto configuration_ptr = configuration.lock(); - if (!configuration_ptr) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration is expired"); - } - return getDataFilesImpl(&filter_dag); -} - std::optional IcebergMetadata::totalRows() const { auto configuration_ptr = configuration.lock(); @@ -664,6 +660,14 @@ std::optional IcebergMetadata::totalBytes() const return result; } +ObjectIterator IcebergMetadata::iterate( + const ActionsDAG * filter_dag, + FileProgressCallback callback, + size_t /* list_batch_size */) const +{ + return createKeysIterator(getDataFiles(filter_dag), object_storage, callback); +} + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h index 0826b86ca035..744a215bdbdc 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h @@ -40,12 +40,6 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext Int32 format_version_, const Poco::JSON::Object::Ptr & metadata_object); - - /// Get data files. On first request it reads manifest_list file and iterates through manifest files to find all data files. - /// All subsequent calls when the same data snapshot is relevant will return saved list of files (because it cannot be changed - /// without changing metadata file). Drops on every snapshot update. - Strings getDataFiles() const override { return getDataFilesImpl(nullptr); } - /// Get table schema parsed from metadata. NamesAndTypesList getTableSchema() const override { @@ -86,13 +80,15 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext bool update(const ContextPtr & local_context) override; - Strings makePartitionPruning(const ActionsDAG & filter_dag) override; - - bool supportsPartitionPruning() override { return true; } - std::optional totalRows() const override; std::optional totalBytes() const override; +protected: + ObjectIterator iterate( + const ActionsDAG * filter_dag, + FileProgressCallback callback, + size_t list_batch_size) const override; + private: using ManifestEntryByDataFile = std::unordered_map; using ManifestFilesStorage = std::unordered_map; @@ -121,6 +117,8 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext mutable std::optional cached_unprunned_files_for_last_processed_snapshot; + Strings getDataFiles(const ActionsDAG * filter_dag) const; + void updateState(const ContextPtr & local_context); void updateSnapshot(); @@ -140,8 +138,6 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext Poco::JSON::Object::Ptr readJSON(const String & metadata_file_path, const ContextPtr & local_context) const; - Strings getDataFilesImpl(const ActionsDAG * filter_dag) const; - Iceberg::ManifestFilePtr tryGetManifestFile(const String & filename) const; }; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 2886049a1acf..f61b557e2c23 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -37,7 +37,6 @@ namespace Setting extern const SettingsMaxThreads max_threads; extern const SettingsBool optimize_count_from_files; extern const SettingsBool use_hive_partitioning; - extern const SettingsBool use_iceberg_partition_pruning; } namespace ErrorCodes @@ -258,11 +257,6 @@ class ReadFromObjectStorageStep : public SourceStepWithFilter void applyFilters(ActionDAGNodes added_filter_nodes) override { SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - if (filter_actions_dag.has_value()) - { - if (getContext()->getSettingsRef()[Setting::use_iceberg_partition_pruning]) - configuration->implementPartitionPruning(*filter_actions_dag); - } createIterator(); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 5894d0efd394..5da7a05d1259 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -224,8 +224,6 @@ class StorageObjectStorage::Configuration virtual bool isDataLakeConfiguration() const { return false; } - virtual void implementPartitionPruning(const ActionsDAG &) { } - virtual std::optional totalRows() { return {}; } virtual std::optional totalBytes() { return {}; } From b3372b96aaba48ee974ec6ee012deac2130bacab Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 9 Apr 2025 14:12:28 +0000 Subject: [PATCH 09/14] Merge pull request #77156 from ClickHouse/hanfei/datalake_meatadata_cache Support Iceberg Metadata Files Cache --- .../table-engines/integrations/iceberg.md | 4 + docs/en/sql-reference/statements/system.md | 4 + .../sql-reference/table-functions/iceberg.md | 4 + programs/local/LocalServer.cpp | 17 ++ programs/server/Server.cpp | 17 ++ src/Access/Common/AccessType.h | 1 + src/Common/CurrentMetrics.cpp | 1 + src/Common/ProfileEvents.cpp | 3 + src/Core/Defines.h | 4 + src/Core/ServerSettings.cpp | 4 + src/Core/Settings.cpp | 4 + src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/Context.cpp | 44 ++++ src/Interpreters/Context.h | 8 + src/Interpreters/InterpreterSystemQuery.cpp | 9 + src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTSystemQuery.h | 1 + .../DataLakes/Iceberg/IcebergMetadata.cpp | 203 +++++++++++------- .../DataLakes/Iceberg/IcebergMetadata.h | 23 +- .../Iceberg/IcebergMetadataFilesCache.h | 164 ++++++++++++++ .../DataLakes/Iceberg/ManifestFile.cpp | 13 +- .../DataLakes/Iceberg/ManifestFile.h | 7 +- .../01271_show_privileges.reference | 1 + 23 files changed, 447 insertions(+), 91 deletions(-) create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 3d12bf00f29e..b46d89af4ad0 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -252,6 +252,10 @@ In Clickhouse the behavior is consistent with Spark. You can mentally replace Sp `Iceberg` table engine and table function support data caching same as `S3`, `AzureBlobStorage`, `HDFS` storages. See [here](../../../engines/table-engines/integrations/s3.md#data-cache). +## Metadata cache {#metadata-cache} + +`Iceberg` table engine and table function support metadata cache storing the information of manifest files, manifest list and metadata json. The cache is stored in memory. This feature is controlled by setting `use_iceberg_metadata_files_cache`, which is enabled by default. + ## See also {#see-also} - [iceberg table function](/sql-reference/table-functions/iceberg.md) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 5b2b2f6328fb..adfac0d106e0 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -97,6 +97,10 @@ For more convenient (automatic) cache management, see disable_internal_dns_cache Clears the mark cache. +## DROP ICEBERG METADATA CACHE {#drop-iceberg-metadata-cache} + +Clears the iceberg metadata cache. + ## DROP REPLICA {#drop-replica} Dead replicas of `ReplicatedMergeTree` tables can be dropped using following syntax: diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index a4e86981d52b..064186acf4ff 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -239,6 +239,10 @@ The second one is that while doing time travel you can't get state of table befo In Clickhouse the behavior is consistent with Spark. You can mentally replace Spark Select queries with Clickhouse Select queries and it will work the same way. +## Metadata cache {#metadata-cache} + +`Iceberg` table engine and table function support metadata cache storing the information of manifest files, manifest list and metadata json. The cache is stored in memory. This feature is controlled by setting `use_iceberg_metadata_files_cache`, which is enabled by default. + ## Aliases {#aliases} Table function `iceberg` is an alias to `icebergS3` now. diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ee0c869f788f..c0ff0620018d 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -96,6 +96,10 @@ namespace ServerSetting extern const ServerSettingsString mark_cache_policy; extern const ServerSettingsUInt64 mark_cache_size; extern const ServerSettingsDouble mark_cache_size_ratio; + extern const ServerSettingsString iceberg_metadata_files_cache_policy; + extern const ServerSettingsUInt64 iceberg_metadata_files_cache_size; + extern const ServerSettingsUInt64 iceberg_metadata_files_cache_max_entries; + extern const ServerSettingsDouble iceberg_metadata_files_cache_size_ratio; extern const ServerSettingsUInt64 max_active_parts_loading_thread_pool_size; extern const ServerSettingsUInt64 max_io_thread_pool_free_size; extern const ServerSettingsUInt64 max_io_thread_pool_size; @@ -810,6 +814,19 @@ void LocalServer::processConfig() } global_context->setMMappedFileCache(mmap_cache_size); +#if USE_AVRO + String iceberg_metadata_files_cache_policy = server_settings[ServerSetting::iceberg_metadata_files_cache_policy]; + size_t iceberg_metadata_files_cache_size = server_settings[ServerSetting::iceberg_metadata_files_cache_size]; + size_t iceberg_metadata_files_cache_max_entries = server_settings[ServerSetting::iceberg_metadata_files_cache_max_entries]; + double iceberg_metadata_files_cache_size_ratio = server_settings[ServerSetting::iceberg_metadata_files_cache_size_ratio]; + if (iceberg_metadata_files_cache_size > max_cache_size) + { + iceberg_metadata_files_cache_size = max_cache_size; + LOG_INFO(log, "Lowered Iceberg metadata cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(iceberg_metadata_files_cache_size)); + } + global_context->setIcebergMetadataFilesCache(iceberg_metadata_files_cache_policy, iceberg_metadata_files_cache_size, iceberg_metadata_files_cache_max_entries, iceberg_metadata_files_cache_size_ratio); +#endif + /// Initialize a dummy query condition cache. global_context->setQueryConditionCache(DEFAULT_QUERY_CONDITION_CACHE_POLICY, 0, 0); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a14ac5afcadf..0caca17d24f8 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -231,6 +231,10 @@ namespace ServerSetting extern const ServerSettingsString index_uncompressed_cache_policy; extern const ServerSettingsUInt64 index_uncompressed_cache_size; extern const ServerSettingsDouble index_uncompressed_cache_size_ratio; + extern const ServerSettingsString iceberg_metadata_files_cache_policy; + extern const ServerSettingsUInt64 iceberg_metadata_files_cache_size; + extern const ServerSettingsUInt64 iceberg_metadata_files_cache_max_entries; + extern const ServerSettingsDouble iceberg_metadata_files_cache_size_ratio; extern const ServerSettingsUInt64 io_thread_pool_queue_size; extern const ServerSettingsSeconds keep_alive_timeout; extern const ServerSettingsString mark_cache_policy; @@ -1761,6 +1765,19 @@ try } global_context->setMMappedFileCache(mmap_cache_size); +#if USE_AVRO + String iceberg_metadata_files_cache_policy = server_settings[ServerSetting::iceberg_metadata_files_cache_policy]; + size_t iceberg_metadata_files_cache_size = server_settings[ServerSetting::iceberg_metadata_files_cache_size]; + size_t iceberg_metadata_files_cache_max_entries = server_settings[ServerSetting::iceberg_metadata_files_cache_max_entries]; + double iceberg_metadata_files_cache_size_ratio = server_settings[ServerSetting::iceberg_metadata_files_cache_size_ratio]; + if (iceberg_metadata_files_cache_size > max_cache_size) + { + iceberg_metadata_files_cache_size = max_cache_size; + LOG_INFO(log, "Lowered Iceberg metadata cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(iceberg_metadata_files_cache_size)); + } + global_context->setIcebergMetadataFilesCache(iceberg_metadata_files_cache_policy, iceberg_metadata_files_cache_size, iceberg_metadata_files_cache_max_entries, iceberg_metadata_files_cache_size_ratio); +#endif + String query_condition_cache_policy = server_settings[ServerSetting::query_condition_cache_policy]; size_t query_condition_cache_size = server_settings[ServerSetting::query_condition_cache_size]; double query_condition_cache_size_ratio = server_settings[ServerSetting::query_condition_cache_size_ratio]; diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 1139808ff334..6a8f4112bce3 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -167,6 +167,7 @@ enum class AccessType : uint8_t M(SYSTEM_DROP_CONNECTIONS_CACHE, "SYSTEM DROP CONNECTIONS CACHE, DROP CONNECTIONS CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_PREWARM_MARK_CACHE, "SYSTEM PREWARM MARK, PREWARM MARK CACHE, PREWARM MARKS", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MARK_CACHE, "SYSTEM DROP MARK, DROP MARK CACHE, DROP MARKS", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_ICEBERG_METADATA_CACHE, "SYSTEM DROP ICEBERG_METADATA_CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_PREWARM_PRIMARY_INDEX_CACHE, "SYSTEM PREWARM PRIMARY INDEX, PREWARM PRIMARY INDEX CACHE, PREWARM PRIMARY INDEX", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_PRIMARY_INDEX_CACHE, "SYSTEM DROP PRIMARY INDEX, DROP PRIMARY INDEX CACHE, DROP PRIMARY INDEX", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_UNCOMPRESSED_CACHE, "SYSTEM DROP UNCOMPRESSED, DROP UNCOMPRESSED CACHE, DROP UNCOMPRESSED", GLOBAL, SYSTEM_DROP_CACHE) \ diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 2b7d14fe1e78..cbbe7ac2dc08 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -304,6 +304,7 @@ M(FilesystemCacheDelayedCleanupElements, "Filesystem cache elements in background cleanup queue") \ M(FilesystemCacheHoldFileSegments, "Filesystem cache file segment which are currently hold as unreleasable") \ M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \ + M(IcebergMetadataFilesCacheSize, "Size of the iceberg metadata cache in bytes") \ M(S3Requests, "S3 requests count") \ M(KeeperAliveConnections, "Number of alive connections") \ M(KeeperOutstandingRequests, "Number of outstanding requests") \ diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index df184b55f74c..0889bad56994 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -75,6 +75,9 @@ M(MarkCacheMisses, "Number of times an entry has not been found in the mark cache, so we had to load a mark file in memory, which is a costly operation, adding to query latency.", ValueType::Number) \ M(PrimaryIndexCacheHits, "Number of times an entry has been found in the primary index cache, so we didn't have to load a index file.", ValueType::Number) \ M(PrimaryIndexCacheMisses, "Number of times an entry has not been found in the primary index cache, so we had to load a index file in memory, which is a costly operation, adding to query latency.", ValueType::Number) \ + M(IcebergMetadataFilesCacheHits, "Number of times iceberg metadata files have been found in the cache.", ValueType::Number) \ + M(IcebergMetadataFilesCacheMisses, "Number of times iceberg metadata files have not been found in the iceberg metadata cache and had to be read from (remote) disk.", ValueType::Number) \ + M(IcebergMetadataFilesCacheWeightLost, "Approximate number of bytes evicted from the iceberg metadata cache.", ValueType::Number) \ M(QueryConditionCacheHits, "Number of times an entry has been found in the query condition cache (and reading of marks can be skipped). Only updated for SELECT queries with SETTING use_query_condition_cache = 1.", ValueType::Number) \ M(QueryConditionCacheMisses, "Number of times an entry has not been found in the query condition cache (and reading of mark cannot be skipped). Only updated for SELECT queries with SETTING use_query_condition_cache = 1.", ValueType::Number) \ M(QueryCacheHits, "Number of times a query result has been found in the query cache (and query computation was avoided). Only updated for SELECT queries with SETTING use_query_cache = 1.", ValueType::Number) \ diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 9c54a7d22c27..fc316435f0b1 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -99,6 +99,10 @@ static constexpr auto DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO = 0.3; static constexpr auto DEFAULT_MMAP_CACHE_MAX_SIZE = 1_KiB; /// chosen by rolling dice static constexpr auto DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE = 128_MiB; static constexpr auto DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES = 10'000; +static constexpr auto DEFAULT_ICEBERG_METADATA_CACHE_POLICY = "SLRU"; +static constexpr auto DEFAULT_ICEBERG_METADATA_CACHE_MAX_SIZE = 1_GiB; +static constexpr auto DEFAULT_ICEBERG_METADATA_CACHE_SIZE_RATIO = 0.5; +static constexpr auto DEFAULT_ICEBERG_METADATA_CACHE_MAX_ENTRIES = 1000; static constexpr auto DEFAULT_QUERY_CONDITION_CACHE_POLICY = "SLRU"; static constexpr auto DEFAULT_QUERY_CONDITION_CACHE_MAX_SIZE = 100_MiB; static constexpr auto DEFAULT_QUERY_CONDITION_CACHE_SIZE_RATIO = 0.5l; diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index a0d2105f0444..7888e491babf 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -461,6 +461,10 @@ namespace DB DECLARE(UInt64, primary_index_cache_size, DEFAULT_PRIMARY_INDEX_CACHE_MAX_SIZE, R"(Size of cache for primary index (index of MergeTree family of tables).)", 0) \ DECLARE(Double, primary_index_cache_size_ratio, DEFAULT_PRIMARY_INDEX_CACHE_SIZE_RATIO, R"(The size of the protected queue in the primary index cache relative to the cache's total size.)", 0) \ DECLARE(Double, primary_index_cache_prewarm_ratio, 0.95, R"(The ratio of total size of mark cache to fill during prewarm.)", 0) \ + DECLARE(String, iceberg_metadata_files_cache_policy, DEFAULT_ICEBERG_METADATA_CACHE_POLICY, "Iceberg metadata cache policy name.", 0) \ + DECLARE(UInt64, iceberg_metadata_files_cache_size, DEFAULT_ICEBERG_METADATA_CACHE_MAX_SIZE, "Maximum size of iceberg metadata cache in bytes. Zero means disabled.", 0) \ + DECLARE(UInt64, iceberg_metadata_files_cache_max_entries, DEFAULT_ICEBERG_METADATA_CACHE_MAX_ENTRIES, "Maximum size of iceberg metadata files cache in entries. Zero means disabled.", 0) \ + DECLARE(Double, iceberg_metadata_files_cache_size_ratio, DEFAULT_ICEBERG_METADATA_CACHE_SIZE_RATIO, "The size of the protected queue (in case of SLRU policy) in the iceberg metadata cache relative to the cache's total size.", 0) \ DECLARE(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, R"(Secondary index uncompressed cache policy name.)", 0) \ DECLARE(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, R"( Size of cache for uncompressed blocks of `MergeTree` indices. diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 98c1d5e01f2d..b2d2915db810 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -4266,6 +4266,10 @@ The maximum size of serialized literal in bytes to replace in `UPDATE` and `DELE \ DECLARE(Float, create_replicated_merge_tree_fault_injection_probability, 0.0f, R"( The probability of a fault injection during table creation after creating metadata in ZooKeeper +)", 0) \ + \ + DECLARE(Bool, use_iceberg_metadata_files_cache, true, R"( +If turned on, iceberg table function and iceberg storage may utilize the iceberg metadata files cache. )", 0) \ \ DECLARE(Bool, use_query_cache, false, R"( diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6cadfdfa7d90..3e65d2473872 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -81,6 +81,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_experimental_database_unity_catalog", false, false, "Allow experimental database engine DataLakeCatalog with catalog_type = 'unity'"}, {"allow_experimental_database_glue_catalog", false, false, "Allow experimental database engine DataLakeCatalog with catalog_type = 'glue'"}, {"use_page_cache_with_distributed_cache", false, false, "New setting"}, + {"use_iceberg_metadata_files_cache", true, true, "New setting"}, {"use_query_condition_cache", false, false, "New setting."}, {"iceberg_timestamp_ms", 0, 0, "New setting."}, {"iceberg_snapshot_id", 0, 0, "New setting."}, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fe04a07da1da..2585e602452b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include #include @@ -444,6 +445,9 @@ struct ContextSharedPart : boost::noncopyable mutable QueryResultCachePtr query_result_cache TSA_GUARDED_BY(mutex); /// Cache of query results. mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices. mutable MMappedFileCachePtr mmap_cache TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. +#if USE_AVRO + mutable IcebergMetadataFilesCachePtr iceberg_metadata_files_cache TSA_GUARDED_BY(mutex); /// Cache of deserialized iceberg metadata files. +#endif AsynchronousMetrics * asynchronous_metrics TSA_GUARDED_BY(mutex) = nullptr; /// Points to asynchronous metrics mutable PageCachePtr page_cache TSA_GUARDED_BY(mutex); /// Userspace page cache. ProcessList process_list; /// Executing queries at the moment. @@ -3582,6 +3586,46 @@ void Context::clearMMappedFileCache() const cache->clear(); } +#if USE_AVRO +void Context::setIcebergMetadataFilesCache(const String & cache_policy, size_t max_size_in_bytes, size_t max_entries, double size_ratio) +{ + std::lock_guard lock(shared->mutex); + + if (shared->iceberg_metadata_files_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Iceberg metadata cache has been already created."); + + shared->iceberg_metadata_files_cache = std::make_shared(cache_policy, max_size_in_bytes, max_entries, size_ratio); +} + +void Context::updateIcebergMetadataFilesCacheConfiguration(const Poco::Util::AbstractConfiguration & config) +{ + std::lock_guard lock(shared->mutex); + + if (!shared->iceberg_metadata_files_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Iceberg metadata cache was not created yet."); + + size_t max_size_in_bytes = config.getUInt64("iceberg_metadata_files_cache_size", DEFAULT_ICEBERG_METADATA_CACHE_MAX_SIZE); + size_t max_entries = config.getUInt64("iceberg_metadata_files_cache_max_entries", DEFAULT_ICEBERG_METADATA_CACHE_MAX_ENTRIES); + shared->iceberg_metadata_files_cache->setMaxSizeInBytes(max_size_in_bytes); + shared->iceberg_metadata_files_cache->setMaxCount(max_entries); +} + +std::shared_ptr Context::getIcebergMetadataFilesCache() const +{ + std::lock_guard lock(shared->mutex); + return shared->iceberg_metadata_files_cache; +} + +void Context::clearIcebergMetadataFilesCache() const +{ + auto cache = getIcebergMetadataFilesCache(); + + /// Clear the cache without holding context mutex to avoid blocking context for a long time + if (cache) + cache->clear(); +} +#endif + void Context::setQueryConditionCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) { std::lock_guard lock(shared->mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 9b3893fe3b84..fb690ce948dd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -97,6 +97,7 @@ class PrimaryIndexCache; class PageCache; class MMappedFileCache; class UncompressedCache; +class IcebergMetadataFilesCache; class ProcessList; class QueryStatus; using QueryStatusPtr = std::shared_ptr; @@ -1168,6 +1169,13 @@ class Context: public ContextData, public std::enable_shared_from_this std::shared_ptr getQueryResultCache() const; void clearQueryResultCache(const std::optional & tag) const; +#if USE_AVRO + void setIcebergMetadataFilesCache(const String & cache_policy, size_t max_size_in_bytes, size_t max_entries, double size_ratio); + void updateIcebergMetadataFilesCacheConfiguration(const Poco::Util::AbstractConfiguration & config); + std::shared_ptr getIcebergMetadataFilesCache() const; + void clearIcebergMetadataFilesCache() const; +#endif + void setQueryConditionCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio); void updateQueryConditionCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getQueryConditionCache() const; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 7eb3fd5218f6..0cd081637d1b 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -384,6 +384,14 @@ BlockIO InterpreterSystemQuery::execute() getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); system_context->clearMarkCache(); break; + case Type::DROP_ICEBERG_METADATA_CACHE: +#if USE_AVRO + getContext()->checkAccess(AccessType::SYSTEM_DROP_ICEBERG_METADATA_CACHE); + system_context->clearIcebergMetadataFilesCache(); + break; +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "The server was compiled without the support for AVRO"); +#endif case Type::DROP_PRIMARY_INDEX_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_PRIMARY_INDEX_CACHE); system_context->clearPrimaryIndexCache(); @@ -1448,6 +1456,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_DNS_CACHE: case Type::DROP_CONNECTIONS_CACHE: case Type::DROP_MARK_CACHE: + case Type::DROP_ICEBERG_METADATA_CACHE: case Type::DROP_PRIMARY_INDEX_CACHE: case Type::DROP_MMAP_CACHE: case Type::DROP_QUERY_CONDITION_CACHE: diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 988205dc4cf1..03557f122c08 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -456,6 +456,7 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti case Type::DROP_SKIPPING_INDEX_CACHE: case Type::DROP_COMPILED_EXPRESSION_CACHE: case Type::DROP_S3_CLIENT_CACHE: + case Type::DROP_ICEBERG_METADATA_CACHE: case Type::RESET_COVERAGE: case Type::RESTART_REPLICAS: case Type::JEMALLOC_PURGE: diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 92ae6dd8f730..ad80ca42b656 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -33,6 +33,7 @@ class ASTSystemQuery : public IAST, public ASTQueryWithOnCluster DROP_QUERY_CONDITION_CACHE, DROP_QUERY_CACHE, DROP_COMPILED_EXPRESSION_CACHE, + DROP_ICEBERG_METADATA_CACHE, DROP_FILESYSTEM_CACHE, DROP_DISTRIBUTED_CACHE, DROP_DISTRIBUTED_CACHE_CONNECTIONS, diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index 505dd25a536a..cfa834687231 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -49,6 +49,7 @@ namespace Setting { extern const SettingsInt64 iceberg_timestamp_ms; extern const SettingsInt64 iceberg_snapshot_id; +extern const SettingsBool use_iceberg_metadata_files_cache; extern const SettingsBool use_iceberg_partition_pruning; } @@ -91,19 +92,21 @@ IcebergMetadata::IcebergMetadata( const DB::ContextPtr & context_, Int32 metadata_version_, Int32 format_version_, - const Poco::JSON::Object::Ptr & metadata_object_) + const Poco::JSON::Object::Ptr & metadata_object_, + IcebergMetadataFilesCachePtr cache_ptr) : WithContext(context_) , object_storage(std::move(object_storage_)) , configuration(std::move(configuration_)) , schema_processor(IcebergSchemaProcessor()) , log(getLogger("IcebergMetadata")) + , manifest_cache(cache_ptr) , last_metadata_version(metadata_version_) , last_metadata_object(metadata_object_) , format_version(format_version_) , relevant_snapshot_schema_id(-1) , table_location(last_metadata_object->getValue(TABLE_LOCATION_FIELD)) { - updateState(context_); + updateState(context_, true); } std::pair parseTableSchemaV2Method(const Poco::JSON::Object::Ptr & metadata_object) @@ -290,15 +293,24 @@ static std::pair getLatestOrExplicitMetadataFileAndVersion(const Poco::JSON::Object::Ptr IcebergMetadata::readJSON(const String & metadata_file_path, const ContextPtr & local_context) const { - ObjectInfo object_info(metadata_file_path); - auto buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, local_context, log); + auto configuration_ptr = configuration.lock(); + auto create_fn = [&]() + { + ObjectInfo object_info(metadata_file_path); + auto buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, local_context, log); - String json_str; - readJSONObjectPossiblyInvalid(json_str, *buf); + String json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); - Poco::JSON::Parser parser; /// For some reason base/base/JSON.h can not parse this json file - Poco::Dynamic::Var json = parser.parse(json_str); - return json.extract(); + Poco::JSON::Parser parser; /// For some reason base/base/JSON.h can not parse this json file + Poco::Dynamic::Var json = parser.parse(json_str); + return std::make_pair(json.extract(), json.size()); + }; + if (manifest_cache) + { + return manifest_cache->getOrSetTableMetadata(IcebergMetadataFilesCache::getKey(configuration_ptr, metadata_file_path), create_fn); + } + return create_fn().first; } bool IcebergMetadata::update(const ContextPtr & local_context) @@ -307,16 +319,20 @@ bool IcebergMetadata::update(const ContextPtr & local_context) const auto [metadata_version, metadata_file_path] = getLatestOrExplicitMetadataFileAndVersion(object_storage, *configuration_ptr, log.get()); - last_metadata_version = metadata_version; - - last_metadata_object = readJSON(metadata_file_path, local_context); + bool metadata_file_changed = false; + if (last_metadata_version != metadata_version) + { + last_metadata_version = metadata_version; + last_metadata_object = readJSON(metadata_file_path, local_context); + metadata_file_changed = true; + } chassert(format_version == last_metadata_object->getValue(FORMAT_VERSION_FIELD)); auto previous_snapshot_id = relevant_snapshot_id; auto previous_snapshot_schema_id = relevant_snapshot_schema_id; - updateState(local_context); + updateState(local_context, metadata_file_changed); if (previous_snapshot_id != relevant_snapshot_id) { @@ -383,7 +399,7 @@ void IcebergMetadata::updateSnapshot() configuration_ptr->getPath()); } -void IcebergMetadata::updateState(const ContextPtr & local_context) +void IcebergMetadata::updateState(const ContextPtr & local_context, bool metadata_file_changed) { auto configuration_ptr = configuration.lock(); std::optional manifest_list_file; @@ -424,7 +440,7 @@ void IcebergMetadata::updateState(const ContextPtr & local_context) relevant_snapshot_id = local_context->getSettingsRef()[Setting::iceberg_snapshot_id]; updateSnapshot(); } - else + else if (metadata_file_changed) { if (!last_metadata_object->has(CURRENT_SNAPSHOT_ID_FIELD_IN_METADATA_FILE)) relevant_snapshot_id = -1; @@ -462,100 +478,135 @@ DataLakeMetadataPtr IcebergMetadata::create( auto log = getLogger("IcebergMetadata"); + Poco::JSON::Object::Ptr object = nullptr; + IcebergMetadataFilesCachePtr cache_ptr = nullptr; + if (local_context->getSettingsRef()[Setting::use_iceberg_metadata_files_cache]) + cache_ptr = local_context->getIcebergMetadataFilesCache(); + else + LOG_TRACE(log, "Not using in-memory cache for iceberg metadata files, because the setting use_iceberg_metadata_files_cache is false."); + const auto [metadata_version, metadata_file_path] = getLatestOrExplicitMetadataFileAndVersion(object_storage, *configuration_ptr, log.get()); - ObjectInfo object_info(metadata_file_path); - auto buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, local_context, log); + auto create_fn = [&]() + { + ObjectInfo object_info(metadata_file_path); // NOLINT + auto buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, local_context, log); + + String json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); - String json_str; - readJSONObjectPossiblyInvalid(json_str, *buf); + Poco::JSON::Parser parser; /// For some reason base/base/JSON.h can not parse this json file + Poco::Dynamic::Var json = parser.parse(json_str); + return std::make_pair(json.extract(), json_str.size()); + }; - Poco::JSON::Parser parser; /// For some reason base/base/JSON.h can not parse this json file - Poco::Dynamic::Var json = parser.parse(json_str); - const Poco::JSON::Object::Ptr & object = json.extract(); + if (cache_ptr) + object = cache_ptr->getOrSetTableMetadata(IcebergMetadataFilesCache::getKey(configuration_ptr, metadata_file_path), create_fn); + else + object = create_fn().first; IcebergSchemaProcessor schema_processor; auto format_version = object->getValue(FORMAT_VERSION_FIELD); auto ptr - = std::make_unique(object_storage, configuration_ptr, local_context, metadata_version, format_version, object); + = std::make_unique(object_storage, configuration_ptr, local_context, metadata_version, format_version, object, cache_ptr); return ptr; } -ManifestList IcebergMetadata::initializeManifestList(const String & filename) const +void IcebergMetadata::initializeDataFiles(ManifestListPtr manifest_list_ptr) const { - auto configuration_ptr = configuration.lock(); - if (configuration_ptr == nullptr) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration is expired"); - - StorageObjectStorage::ObjectInfo object_info(filename); - auto manifest_list_buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, getContext(), log); - AvroForIcebergDeserializer manifest_list_deserializer(std::move(manifest_list_buf), filename, getFormatSettings(getContext())); - - ManifestList manifest_list; - - for (size_t i = 0; i < manifest_list_deserializer.rows(); ++i) + for (const auto & manifest_file_content : *manifest_list_ptr) { - const std::string file_path = manifest_list_deserializer.getValueFromRowByName(i, MANIFEST_FILE_PATH_COLUMN, TypeIndex::String).safeGet(); - const auto manifest_file_name = getProperFilePathFromMetadataInfo(file_path, configuration_ptr->getPath(), table_location); - Int64 added_sequence_number = 0; - if (format_version > 1) - added_sequence_number = manifest_list_deserializer.getValueFromRowByName(i, SEQUENCE_NUMBER_COLUMN, TypeIndex::Int64).safeGet(); - - /// We can't encapsulate this logic in getManifestFile because we need not only the name of the file, but also an inherited sequence number which is known only during the parsing of ManifestList - auto manifest_file_content = initializeManifestFile(manifest_file_name, added_sequence_number); - manifest_files_by_name.emplace(manifest_file_name, manifest_file_content); for (const auto & data_file_path : manifest_file_content->getFiles()) { if (std::holds_alternative(data_file_path.file)) manifest_file_by_data_file.emplace(std::get(data_file_path.file).file_name, manifest_file_content); } - manifest_list.push_back(manifest_file_content); } - - return manifest_list; } -ManifestFilePtr IcebergMetadata::initializeManifestFile(const String & filename, Int64 inherited_sequence_number) const +ManifestListPtr IcebergMetadata::getManifestList(const String & filename) const { auto configuration_ptr = configuration.lock(); + if (configuration_ptr == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration is expired"); - ObjectInfo manifest_object_info(filename); - auto buffer = StorageObjectStorageSource::createReadBuffer(manifest_object_info, object_storage, getContext(), log); - AvroForIcebergDeserializer manifest_file_deserializer(std::move(buffer), filename, getFormatSettings(getContext())); - auto [schema_id, schema_object] = parseTableSchemaFromManifestFile(manifest_file_deserializer, filename); - schema_processor.addIcebergTableSchema(schema_object); - return std::make_shared( - manifest_file_deserializer, - format_version, - configuration_ptr->getPath(), - schema_id, - schema_processor, - inherited_sequence_number, - table_location, - getContext()); + auto create_fn = [&]() + { + ManifestList manifest_list; + StorageObjectStorage::ObjectInfo object_info(filename); + auto manifest_list_buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, getContext(), log); + AvroForIcebergDeserializer manifest_list_deserializer(std::move(manifest_list_buf), filename, getFormatSettings(getContext())); -} + ManifestFileCacheKeys manifest_file_cache_keys; -ManifestFilePtr IcebergMetadata::tryGetManifestFile(const String & filename) const -{ - auto manifest_file_it = manifest_files_by_name.find(filename); - if (manifest_file_it != manifest_files_by_name.end()) - return manifest_file_it->second; - return nullptr; + for (size_t i = 0; i < manifest_list_deserializer.rows(); ++i) + { + const std::string file_path = manifest_list_deserializer.getValueFromRowByName(i, MANIFEST_FILE_PATH_COLUMN, TypeIndex::String).safeGet(); + const auto manifest_file_name = getProperFilePathFromMetadataInfo(file_path, configuration_ptr->getPath(), table_location); + Int64 added_sequence_number = 0; + if (format_version > 1) + added_sequence_number = manifest_list_deserializer.getValueFromRowByName(i, SEQUENCE_NUMBER_COLUMN, TypeIndex::Int64).safeGet(); + manifest_file_cache_keys.emplace_back(manifest_file_name, added_sequence_number); + } + /// We only return the list of {file name, seq number} for cache. + /// Because ManifestList holds a list of ManifestFilePtr which consume much memory space. + /// ManifestFilePtr is shared pointers can be held for too much time, so we cache ManifestFile separately. + return manifest_file_cache_keys; + }; + + ManifestFileCacheKeys manifest_file_cache_keys; + ManifestList manifest_list; + if (manifest_cache) + { + manifest_file_cache_keys = manifest_cache->getOrSetManifestFileCacheKeys(IcebergMetadataFilesCache::getKey(configuration_ptr, filename), create_fn); + } + else + { + manifest_file_cache_keys = create_fn(); + } + for (const auto & entry : manifest_file_cache_keys) + { + auto manifest_file_ptr = getManifestFile(entry.manifest_file_path, entry.added_sequence_number); + manifest_list.push_back(manifest_file_ptr); + } + ManifestListPtr manifest_list_ptr = std::make_shared(std::move(manifest_list)); + initializeDataFiles(manifest_list_ptr); + return manifest_list_ptr; } -ManifestListPtr IcebergMetadata::getManifestList(const String & filename) const +ManifestFilePtr IcebergMetadata::getManifestFile(const String & filename, Int64 inherited_sequence_number) const { - auto manifest_file_it = manifest_lists_by_name.find(filename); - if (manifest_file_it != manifest_lists_by_name.end()) - return manifest_file_it->second; auto configuration_ptr = configuration.lock(); - auto manifest_list_ptr = std::make_shared(initializeManifestList(filename)); - manifest_lists_by_name.emplace(filename, manifest_list_ptr); - return manifest_list_ptr; + + auto create_fn = [&]() + { + ObjectInfo manifest_object_info(filename); + auto buffer = StorageObjectStorageSource::createReadBuffer(manifest_object_info, object_storage, getContext(), log); + AvroForIcebergDeserializer manifest_file_deserializer(std::move(buffer), filename, getFormatSettings(getContext())); + auto [schema_id, schema_object] = parseTableSchemaFromManifestFile(manifest_file_deserializer, filename); + schema_processor.addIcebergTableSchema(schema_object); + return std::make_shared( + manifest_file_deserializer, + format_version, + configuration_ptr->getPath(), + schema_id, + schema_object, + schema_processor, + inherited_sequence_number, + table_location, + getContext()); + }; + + if (manifest_cache) + { + auto manifest_file = manifest_cache->getOrSetManifestFile(IcebergMetadataFilesCache::getKey(configuration_ptr, filename), create_fn); + schema_processor.addIcebergTableSchema(manifest_file->getSchemaObject()); + return manifest_file; + } + return create_fn(); } Strings IcebergMetadata::getDataFiles(const ActionsDAG * filter_dag) const diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h index 744a215bdbdc..3f4005d153c4 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -38,7 +39,8 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext const DB::ContextPtr & context_, Int32 metadata_version_, Int32 format_version_, - const Poco::JSON::Object::Ptr & metadata_object); + const Poco::JSON::Object::Ptr & metadata_object, + IcebergMetadataFilesCachePtr cache_ptr); /// Get table schema parsed from metadata. NamesAndTypesList getTableSchema() const override @@ -91,16 +93,13 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext private: using ManifestEntryByDataFile = std::unordered_map; - using ManifestFilesStorage = std::unordered_map; - using ManifestListsStorage = std::unordered_map; const ObjectStoragePtr object_storage; const ConfigurationObserverPtr configuration; mutable IcebergSchemaProcessor schema_processor; LoggerPtr log; - mutable ManifestFilesStorage manifest_files_by_name; - mutable ManifestListsStorage manifest_lists_by_name; + IcebergMetadataFilesCachePtr manifest_cache; mutable ManifestEntryByDataFile manifest_file_by_data_file; std::tuple getVersion() const { return std::make_tuple(relevant_snapshot_id, relevant_snapshot_schema_id); } @@ -117,28 +116,26 @@ class IcebergMetadata : public IDataLakeMetadata, private WithContext mutable std::optional cached_unprunned_files_for_last_processed_snapshot; - Strings getDataFiles(const ActionsDAG * filter_dag) const; + void updateState(const ContextPtr & local_context, bool metadata_file_changed); - void updateState(const ContextPtr & local_context); + Strings getDataFiles(const ActionsDAG * filter_dag) const; void updateSnapshot(); - Iceberg::ManifestList initializeManifestList(const String & filename) const; + Iceberg::ManifestListPtr getManifestList(const String & filename) const; mutable std::vector positional_delete_files_for_current_query; void addTableSchemaById(Int32 schema_id); - Iceberg::ManifestListPtr getManifestList(const String & filename) const; - std::optional getSchemaVersionByFileIfOutdated(String data_path) const; - Iceberg::ManifestFilePtr initializeManifestFile(const String & filename, Int64 inherited_sequence_number) const; + void initializeDataFiles(Iceberg::ManifestListPtr manifest_list_ptr) const; + + Iceberg::ManifestFilePtr getManifestFile(const String & filename, Int64 inherited_sequence_number) const; std::optional getRelevantManifestList(const Poco::JSON::Object::Ptr & metadata); Poco::JSON::Object::Ptr readJSON(const String & metadata_file_path, const ContextPtr & local_context) const; - - Iceberg::ManifestFilePtr tryGetManifestFile(const String & filename) const; }; } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h new file mode 100644 index 000000000000..48390f9946c5 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h @@ -0,0 +1,164 @@ +#pragma once +#include "config.h" + +#if USE_AVRO + +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event IcebergMetadataFilesCacheMisses; + extern const Event IcebergMetadataFilesCacheHits; + extern const Event IcebergMetadataFilesCacheWeightLost; +} + +namespace CurrentMetrics +{ + extern const Metric IcebergMetadataFilesCacheSize; +} + +namespace DB +{ + +/// The structure that can identify a manifest file. We store it in cache. +/// And we can get `ManifestFileContent` from cache by ManifestFileEntry. +struct ManifestFileCacheKey +{ + String manifest_file_path; + Int64 added_sequence_number; +}; + +using ManifestFileCacheKeys = std::vector; + +/// We have three kinds of metadata files in iceberg: metadata object, manifest list and manifest files. +/// For simplicity, we keep them in the same cache. +struct IcebergMetadataFilesCacheCell : private boost::noncopyable +{ + /// The cached element could be + /// - metadata.json deserialized as Poco::JSON::Object::Ptr + /// - manifest list consists of cache keys which will retrieve the manifest file from cache + /// - manifest file + std::variant cached_element; + Int64 memory_bytes; + + explicit IcebergMetadataFilesCacheCell(Poco::JSON::Object::Ptr metadata_object_, size_t memory_bytes_) + : cached_element(metadata_object_) + , memory_bytes(memory_bytes_ + SIZE_IN_MEMORY_OVERHEAD) + , metric_increment{CurrentMetrics::IcebergMetadataFilesCacheSize, memory_bytes} + { + } + explicit IcebergMetadataFilesCacheCell(ManifestFileCacheKeys && manifest_file_cache_keys_) + : cached_element(std::move(manifest_file_cache_keys_)) + , memory_bytes(getMemorySizeOfManifestCacheKeys(std::get(cached_element)) + SIZE_IN_MEMORY_OVERHEAD) + , metric_increment{CurrentMetrics::IcebergMetadataFilesCacheSize, memory_bytes} + { + } + explicit IcebergMetadataFilesCacheCell(Iceberg::ManifestFilePtr manifest_file_) + : cached_element(manifest_file_) + , memory_bytes(std::get(cached_element)->getSizeInMemory() + SIZE_IN_MEMORY_OVERHEAD) + , metric_increment{CurrentMetrics::IcebergMetadataFilesCacheSize, memory_bytes} + { + } +private: + CurrentMetrics::Increment metric_increment; + static constexpr size_t SIZE_IN_MEMORY_OVERHEAD = 200; /// we always underestimate the size of an object; + + static size_t getMemorySizeOfManifestCacheKeys(const ManifestFileCacheKeys & manifest_file_cache_keys) + { + size_t total_size = 0; + for (const auto & entry: manifest_file_cache_keys) + { + total_size += sizeof(ManifestFileCacheKey) + entry.manifest_file_path.capacity(); + } + return total_size; + } + +}; + +struct IcebergMetadataFilesCacheWeightFunction +{ + size_t operator()(const IcebergMetadataFilesCacheCell & cell) const + { + return cell.memory_bytes; + } +}; + +class IcebergMetadataFilesCache : public CacheBase, IcebergMetadataFilesCacheWeightFunction> +{ +public: + using Base = CacheBase, IcebergMetadataFilesCacheWeightFunction>; + + IcebergMetadataFilesCache(const String & cache_policy, size_t max_size_in_bytes, size_t max_count, double size_ratio) + : Base(cache_policy, max_size_in_bytes, max_count, size_ratio) + {} + + static String getKey(StorageObjectStorage::ConfigurationPtr config, const String & data_path) + { + return std::filesystem::path(config->getDataSourceDescription()) / data_path; + } + + template + Poco::JSON::Object::Ptr getOrSetTableMetadata(const String & data_path, LoadFunc && load_fn) + { + auto load_fn_wrapper = [&]() + { + const auto & [json_ptr, json_size] = load_fn(); + return std::make_shared(json_ptr, json_size); + }; + auto result = Base::getOrSet(data_path, load_fn_wrapper); + if (result.second) + ProfileEvents::increment(ProfileEvents::IcebergMetadataFilesCacheMisses); + else + ProfileEvents::increment(ProfileEvents::IcebergMetadataFilesCacheHits); + return std::get(result.first->cached_element); + } + + template + ManifestFileCacheKeys getOrSetManifestFileCacheKeys(const String & data_path, LoadFunc && load_fn) + { + auto load_fn_wrapper = [&]() + { + auto && manifest_file_cache_keys = load_fn(); + return std::make_shared(std::move(manifest_file_cache_keys)); + }; + auto result = Base::getOrSet(data_path, load_fn_wrapper); + if (result.second) + ProfileEvents::increment(ProfileEvents::IcebergMetadataFilesCacheMisses); + else + ProfileEvents::increment(ProfileEvents::IcebergMetadataFilesCacheHits); + return std::get(result.first->cached_element); + } + + template + Iceberg::ManifestFilePtr getOrSetManifestFile(const String & data_path, LoadFunc && load_fn) + { + auto load_fn_wrapper = [&]() + { + Iceberg::ManifestFilePtr manifest_file = load_fn(); + return std::make_shared(manifest_file); + }; + auto result = Base::getOrSet(data_path, load_fn_wrapper); + if (result.second) + ProfileEvents::increment(ProfileEvents::IcebergMetadataFilesCacheMisses); + else + ProfileEvents::increment(ProfileEvents::IcebergMetadataFilesCacheHits); + return std::get(result.first->cached_element); + } + +private: + void onRemoveOverflowWeightLoss(size_t weight_loss) override + { + ProfileEvents::increment(ProfileEvents::IcebergMetadataFilesCacheWeightLost, weight_loss); + } +}; + +using IcebergMetadataFilesCachePtr = std::shared_ptr; + +} +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp index 650a5c4af033..4d9a4c817597 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp @@ -137,12 +137,14 @@ ManifestFileContent::ManifestFileContent( Int32 format_version_, const String & common_path, Int32 schema_id_, + Poco::JSON::Object::Ptr schema_object_, const IcebergSchemaProcessor & schema_processor, Int64 inherited_sequence_number, const String & table_location, DB::ContextPtr context) { this->schema_id = schema_id_; + this->schema_object = schema_object_; for (const auto & column_name : {COLUMN_STATUS_NAME, COLUMN_TUPLE_DATA_FILE_NAME}) { @@ -331,6 +333,16 @@ const std::set & ManifestFileContent::getColumnsIDsWithBounds() const return column_ids_which_have_bounds; } +size_t ManifestFileContent::getSizeInMemory() const +{ + size_t total_size = sizeof(ManifestFileContent); + if (partition_key_description) + total_size += sizeof(DB::KeyDescription); + total_size += column_ids_which_have_bounds.size() * sizeof(Int32); + total_size += files.capacity() * sizeof(ManifestFileEntry); + return total_size; +} + std::optional ManifestFileContent::getRowsCountInAllDataFilesExcludingDeleted() const { Int64 result = 0; @@ -376,7 +388,6 @@ std::optional ManifestFileContent::getBytesCountInAllDataFiles() const return std::nullopt; } return result; - } } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h index 83b531400326..df86b747d26f 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.h @@ -92,6 +92,7 @@ class ManifestFileContent Int32 format_version_, const String & common_path, Int32 schema_id_, + Poco::JSON::Object::Ptr schema_object_, const DB::IcebergSchemaProcessor & schema_processor, Int64 inherited_sequence_number, const std::string & table_location, @@ -102,6 +103,10 @@ class ManifestFileContent bool hasPartitionKey() const; const DB::KeyDescription & getPartitionKeyDescription() const; + Poco::JSON::Object::Ptr getSchemaObject() const { return schema_object; } + /// Get size in bytes of how much memory one instance of this ManifestFileContent class takes. + /// Used for in-memory caches size accounting. + size_t getSizeInMemory() const; /// Fields with rows count in manifest files are optional /// they can be absent. @@ -113,7 +118,7 @@ class ManifestFileContent private: Int32 schema_id; - + Poco::JSON::Object::Ptr schema_object; std::optional partition_key_description; // Size - number of files std::vector files; diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index e5107526be91..e0aed9c82567 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -116,6 +116,7 @@ SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYS SYSTEM DROP CONNECTIONS CACHE ['SYSTEM DROP CONNECTIONS CACHE','DROP CONNECTIONS CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM PREWARM MARK CACHE ['SYSTEM PREWARM MARK','PREWARM MARK CACHE','PREWARM MARKS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MARK CACHE ['SYSTEM DROP MARK','DROP MARK CACHE','DROP MARKS'] GLOBAL SYSTEM DROP CACHE +SYSTEM DROP ICEBERG METADATA CACHE ['SYSTEM DROP ICEBERG_METADATA_CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM PREWARM PRIMARY INDEX CACHE ['SYSTEM PREWARM PRIMARY INDEX','PREWARM PRIMARY INDEX CACHE','PREWARM PRIMARY INDEX'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP PRIMARY INDEX CACHE ['SYSTEM DROP PRIMARY INDEX','DROP PRIMARY INDEX CACHE','DROP PRIMARY INDEX'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP UNCOMPRESSED CACHE ['SYSTEM DROP UNCOMPRESSED','DROP UNCOMPRESSED CACHE','DROP UNCOMPRESSED'] GLOBAL SYSTEM DROP CACHE From 4c242f3e413c99ca66e7ffc32499a42f6e2c0749 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 8 Apr 2025 12:58:44 +0000 Subject: [PATCH 10/14] Merge pull request #78764 from ucasfl/fix-iceberg Fix reading iceberg failed when min-max value is null --- .../ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp index 4d9a4c817597..3f934cd2bf5b 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/ManifestFile.cpp @@ -273,8 +273,15 @@ ManifestFileContent::ManifestFileContent( for (const auto & [column_id, bounds] : value_for_bounds) { DB::NameAndTypePair name_and_type = schema_processor.getFieldCharacteristics(schema_id, column_id); - auto left = deserializeFieldFromBinaryRepr(bounds.first.safeGet(), name_and_type.type, true); - auto right = deserializeFieldFromBinaryRepr(bounds.second.safeGet(), name_and_type.type, false); + + String left_str; + String right_str; + /// lower_bound and upper_bound may be NULL. + if (!bounds.first.tryGet(left_str) || !bounds.second.tryGet(right_str)) + continue; + + auto left = deserializeFieldFromBinaryRepr(left_str, name_and_type.type, true); + auto right = deserializeFieldFromBinaryRepr(right_str, name_and_type.type, false); if (!left || !right) continue; From 0f427970de5defea174b56f84b3239e03768348a Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Fri, 18 Apr 2025 16:38:04 +0200 Subject: [PATCH 11/14] Merge pull request #742 from Altinity/feature/lazy_load_metadata Make DataLake metadata more lazy --- src/Disks/ObjectStorages/IObjectStorage.cpp | 10 ++++++++++ src/Disks/ObjectStorages/IObjectStorage.h | 2 ++ .../ObjectStorage/DataLakes/IDataLakeMetadata.cpp | 10 +++++++--- src/Storages/ObjectStorage/ReadBufferIterator.cpp | 8 +++----- .../ObjectStorage/StorageObjectStorageSource.cpp | 6 +----- 5 files changed, 23 insertions(+), 13 deletions(-) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index ce5f06e8f25f..f729b7ce6913 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -97,4 +97,14 @@ WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings return write_settings; } + +void RelativePathWithMetadata::loadMetadata(ObjectStoragePtr object_storage) +{ + if (!metadata) + { + const auto & path = isArchive() ? getPathToArchive() : getPath(); + metadata = object_storage->tryGetObjectMetadata(path); + } +} + } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 26d28906c17d..f99864ebb25c 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -83,6 +83,8 @@ struct RelativePathWithMetadata virtual bool isArchive() const { return false; } virtual std::string getPathToArchive() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not an archive"); } virtual size_t fileSizeInArchive() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not an archive"); } + + void loadMetadata(ObjectStoragePtr object_storage); }; struct ObjectKeyWithMetadata diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.cpp index 61f31766a455..6bbf81c74965 100644 --- a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.cpp @@ -34,12 +34,16 @@ class KeysIterator : public IObjectIterator return nullptr; auto key = data_files[current_index]; - auto object_metadata = object_storage->getObjectMetadata(key); if (callback) - callback(FileProgress(0, object_metadata.size_bytes)); + { + /// Too expencive to load size for metadata always + /// because it requires API call to external storage. + /// In many cases only keys are needed. + callback(FileProgress(0, 1)); + } - return std::make_shared(key, std::move(object_metadata)); + return std::make_shared(key, std::nullopt); } } diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index df9faa048c2b..2baed9ad4176 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -74,10 +74,7 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( const auto & object_info = (*it); auto get_last_mod_time = [&] -> std::optional { - const auto & path = object_info->isArchive() ? object_info->getPathToArchive() : object_info->getPath(); - if (!object_info->metadata) - object_info->metadata = object_storage->tryGetObjectMetadata(path); - + object_info->loadMetadata(object_storage); return object_info->metadata ? std::optional(object_info->metadata->last_modified.epochTime()) : std::nullopt; @@ -149,7 +146,6 @@ std::unique_ptr ReadBufferIterator::recreateLastReadBuffer() { auto context = getContext(); - const auto & path = current_object_info->isArchive() ? current_object_info->getPathToArchive() : current_object_info->getPath(); auto impl = StorageObjectStorageSource::createReadBuffer(*current_object_info, object_storage, context, getLogger("ReadBufferIterator")); const auto compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); @@ -248,6 +244,8 @@ ReadBufferIterator::Data ReadBufferIterator::next() prev_read_keys_size = read_keys.size(); } + current_object_info->loadMetadata(object_storage); + if (query_settings.skip_empty_files && current_object_info->metadata && current_object_info->metadata->size_bytes == 0) continue; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index cef27f90a925..f400376459e6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -409,11 +409,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade if (!object_info || object_info->getPath().empty()) return {}; - if (!object_info->metadata) - { - const auto & path = object_info->isArchive() ? object_info->getPathToArchive() : object_info->getPath(); - object_info->metadata = object_storage->getObjectMetadata(path); - } + object_info->loadMetadata(object_storage); } while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); From f6730ab59e9bc5fcd4d66801b2c749dd4ed1ec47 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 29 May 2025 12:10:59 +0200 Subject: [PATCH 12/14] Fix build after merge --- src/Databases/DataLake/DatabaseDataLake.cpp | 1 - src/Storages/ObjectStorage/StorageObjectStorage.cpp | 2 +- src/Storages/ObjectStorage/registerStorageObjectStorage.cpp | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Databases/DataLake/DatabaseDataLake.cpp b/src/Databases/DataLake/DatabaseDataLake.cpp index c3ef60da561d..1b556f25edd4 100644 --- a/src/Databases/DataLake/DatabaseDataLake.cpp +++ b/src/Databases/DataLake/DatabaseDataLake.cpp @@ -420,7 +420,6 @@ StoragePtr DatabaseDataLake::tryGetTableImpl(const String & name, ContextPtr con /* comment */"", getFormatSettings(context_copy), LoadingStrictnessLevel::CREATE, - /* is_table_function */false, /* partition_by */nullptr); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 33c2814fe268..218c9060aae2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -100,8 +100,8 @@ StorageObjectStorage::StorageObjectStorage( , distributed_processing(distributed_processing_) , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { - update_configuration_on_read = !is_table_function_ || do_lazy_init; bool do_lazy_init = lazy_init && !columns_.empty() && !configuration->getFormat().empty(); + update_configuration_on_read = !is_table_function_ || do_lazy_init; bool failed_init = false; auto do_init = [&]() { diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index d93d14618a73..16b8697b1d21 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -47,7 +47,7 @@ createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObject auto cluster_name = (*storage_settings)[StorageObjectStorageSetting::object_storage_cluster].value; -configuration->initialize(args.engine_args, context, false, storage_settings); + configuration->initialize(args.engine_args, context, false, storage_settings); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current From 9cd2b88ac6c23518bb2c0ac910de3eb8fc93179b Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 23 Apr 2025 15:23:37 +0200 Subject: [PATCH 13/14] Merge pull request #747 from Altinity/feature/fix_mysql_test_build Fix build --- src/Common/mysqlxx/tests/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/mysqlxx/tests/CMakeLists.txt b/src/Common/mysqlxx/tests/CMakeLists.txt index 287249798bf6..0771523fe1a2 100644 --- a/src/Common/mysqlxx/tests/CMakeLists.txt +++ b/src/Common/mysqlxx/tests/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (mysqlxx_pool_test mysqlxx_pool_test.cpp) -target_link_libraries (mysqlxx_pool_test PRIVATE mysqlxx dbms clickhouse_common_config loggers_no_text_log) +target_link_libraries (mysqlxx_pool_test PRIVATE mysqlxx dbms clickhouse_common_config loggers_no_text_log clickhouse_functions_extractkeyvaluepairs) From f8970296e92a2673d1d1e2eb59acb10b341a397e Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 2 Jun 2025 17:20:48 +0200 Subject: [PATCH 14/14] Fix tests --- .../integration/test_storage_iceberg/test.py | 325 +----------------- 1 file changed, 14 insertions(+), 311 deletions(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index f42ae28d8efe..8b4a2008d9c0 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -3216,337 +3216,40 @@ def check_validity_and_get_prunned_files(select_expression): ) @pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) -def test_schema_evolution_with_time_travel( - started_cluster, format_version, storage_type -): - instance = started_cluster.instances["node1"] - spark = started_cluster.spark_session - TABLE_NAME = ( - "test_schema_evolution_with_time_travel_" - + format_version - + "_" - + storage_type - + "_" - + get_uuid_str() - ) - - def execute_spark_query(query: str): - spark.sql(query) - default_upload_directory( - started_cluster, - storage_type, - f"/iceberg_data/default/{TABLE_NAME}/", - f"/iceberg_data/default/{TABLE_NAME}/", - ) - return - - execute_spark_query( - f""" - DROP TABLE IF EXISTS {TABLE_NAME}; - """ - ) - - execute_spark_query( - f""" - CREATE TABLE IF NOT EXISTS {TABLE_NAME} ( - a int NOT NULL - ) - USING iceberg - OPTIONS ('format-version'='{format_version}') - """ - ) - - table_creation_expression = get_creation_expression( - storage_type, - TABLE_NAME, - started_cluster, - table_function=True, - allow_dynamic_metadata_for_data_lakes=True, - ) - - table_select_expression = table_creation_expression - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"] - ], - [], - ) - - first_timestamp_ms = int(datetime.now().timestamp() * 1000) - - time.sleep(0.5) - - execute_spark_query( - f""" - INSERT INTO {TABLE_NAME} VALUES (4); - """ - ) - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"], - ], - [["4"]], - ) - - error_message = instance.query_and_get_error(f"SELECT * FROM {table_select_expression} ORDER BY ALL SETTINGS iceberg_timestamp_ms = {first_timestamp_ms}") - assert "No snapshot found in snapshot log before requested timestamp" in error_message - - - second_timestamp_ms = int(datetime.now().timestamp() * 1000) - - time.sleep(0.5) - - execute_spark_query( - f""" - ALTER TABLE {TABLE_NAME} ADD COLUMNS ( - b double - ); - """ - ) - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"], - ["b", "Nullable(Float64)"] - ], - [["4", "\\N"]], - ) - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"], - ], - [["4"]], - timestamp_ms=second_timestamp_ms, - ) - - third_timestamp_ms = int(datetime.now().timestamp() * 1000) - - time.sleep(0.5) - - - execute_spark_query( - f""" - INSERT INTO {TABLE_NAME} VALUES (7, 5.0); - """ - ) - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"], - ["b", "Nullable(Float64)"] - ], - [["4", "\\N"], ["7", "5"]], - ) - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"], - ], - [["4"]], - timestamp_ms=second_timestamp_ms, - ) - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"], ], - [["4"]], - timestamp_ms=third_timestamp_ms, - ) - - execute_spark_query( - f""" - ALTER TABLE {TABLE_NAME} ADD COLUMNS ( - c double - ); - """ - ) - - time.sleep(0.5) - fourth_timestamp_ms = int(datetime.now().timestamp() * 1000) - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"], - ["b", "Nullable(Float64)"] - ], - [["4", "\\N"], ["7", "5"]], - timestamp_ms=fourth_timestamp_ms, - ) - - check_schema_and_data( - instance, - table_select_expression, - [ - ["a", "Int32"], - ["b", "Nullable(Float64)"], - ["c", "Nullable(Float64)"] - ], - [["4", "\\N", "\\N"], ["7", "5", "\\N"]], - ) - -def get_last_snapshot(path_to_table): - import json - import os - - metadata_dir = f"{path_to_table}/metadata/" - last_timestamp = 0 - last_snapshot_id = -1 - for filename in os.listdir(metadata_dir): - if filename.endswith('.json'): - filepath = os.path.join(metadata_dir, filename) - with open(filepath, 'r') as f: - data = json.load(f) - print(data) - timestamp = data.get('last-updated-ms') - if (timestamp > last_timestamp): - last_timestamp = timestamp - last_snapshot_id = data.get('current-snapshot-id') - return last_snapshot_id - - -@pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) -def test_iceberg_snapshot_reads(started_cluster, format_version, storage_type): +def test_explicit_metadata_file(started_cluster, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session TABLE_NAME = ( - "test_iceberg_snapshot_reads" - + format_version - + "_" + "test_explicit_metadata_file_" + storage_type + "_" + get_uuid_str() ) - write_iceberg_from_df( - spark, - generate_data(spark, 0, 100), - TABLE_NAME, - mode="overwrite", - format_version=format_version, - ) - default_upload_directory( - started_cluster, - storage_type, - f"/iceberg_data/default/{TABLE_NAME}/", - "", + spark.sql( + f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" ) - create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - snapshot1_timestamp = datetime.now(timezone.utc) - snapshot1_id = get_last_snapshot(f"/iceberg_data/default/{TABLE_NAME}/") - time.sleep(0.1) + for i in range(50): + spark.sql( + f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)" + ) - write_iceberg_from_df( - spark, - generate_data(spark, 100, 200), - TABLE_NAME, - mode="append", - format_version=format_version, - ) default_upload_directory( started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", - ) - snapshot2_timestamp = datetime.now(timezone.utc) - snapshot2_id = get_last_snapshot(f"/iceberg_data/default/{TABLE_NAME}/") - time.sleep(0.1) - - write_iceberg_from_df( - spark, - generate_data(spark, 200, 300), - TABLE_NAME, - mode="append", - format_version=format_version, - ) - default_upload_directory( - started_cluster, - storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", - ) - snapshot3_timestamp = datetime.now(timezone.utc) - snapshot3_id = get_last_snapshot(f"/iceberg_data/default/{TABLE_NAME}/") - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 300 - assert instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY 1") == instance.query( - "SELECT number, toString(number + 1) FROM numbers(300)" - ) - - # Validate that each snapshot timestamp only sees the data inserted by that time. - assert ( - instance.query( - f""" - SELECT * FROM {TABLE_NAME} ORDER BY 1 - SETTINGS iceberg_timestamp_ms = {int(snapshot1_timestamp.timestamp() * 1000)}""" - ) - == instance.query("SELECT number, toString(number + 1) FROM numbers(100)") ) - assert ( - instance.query( - f""" - SELECT * FROM {TABLE_NAME} ORDER BY 1 - SETTINGS iceberg_snapshot_id = {snapshot1_id}""" - ) - == instance.query("SELECT number, toString(number + 1) FROM numbers(100)") - ) - - - assert ( - instance.query( - f""" - SELECT * FROM {TABLE_NAME} ORDER BY 1 - SETTINGS iceberg_timestamp_ms = {int(snapshot2_timestamp.timestamp() * 1000)}""" - ) - == instance.query("SELECT number, toString(number + 1) FROM numbers(200)") - ) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, explicit_metadata_path="") - assert ( - instance.query( - f""" - SELECT * FROM {TABLE_NAME} ORDER BY 1 - SETTINGS iceberg_snapshot_id = {snapshot2_id}""" - ) - == instance.query("SELECT number, toString(number + 1) FROM numbers(200)") - ) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, explicit_metadata_path="metadata/v31.metadata.json") - assert ( - instance.query( - f"""SELECT * FROM {TABLE_NAME} ORDER BY 1 - SETTINGS iceberg_timestamp_ms = {int(snapshot3_timestamp.timestamp() * 1000)}""" - ) - == instance.query("SELECT number, toString(number + 1) FROM numbers(300)") - ) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 300 - assert ( - instance.query( - f""" - SELECT * FROM {TABLE_NAME} ORDER BY 1 - SETTINGS iceberg_snapshot_id = {snapshot3_id}""" - ) - == instance.query("SELECT number, toString(number + 1) FROM numbers(300)") - ) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, explicit_metadata_path="metadata/v11.metadata.json") + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100