diff --git a/docs/en/engines/table-engines/mergetree-family/part_export.md b/docs/en/engines/table-engines/mergetree-family/part_export.md new file mode 100644 index 000000000000..287e0a17f3af --- /dev/null +++ b/docs/en/engines/table-engines/mergetree-family/part_export.md @@ -0,0 +1,307 @@ +# ALTER TABLE EXPORT PART + +## Overview + +The `ALTER TABLE EXPORT PART` command exports individual MergeTree data parts to object storage (S3, Azure Blob Storage, etc.), typically in Parquet format. A commit file is shipped to the same destination directory containing all data files exported within that transaction. + +**Key Characteristics:** +- **Experimental feature** - must be enabled via `allow_experimental_export_merge_tree_part` setting +- **Asynchronous** - executes in the background, returns immediately +- **Ephemeral** - no automatic retry mechanism; manual retry required on failure +- **Idempotent** - safe to re-export the same part (skips by default if file exists) +- **Preserves sort order** from the source table + +## Syntax + +```sql +ALTER TABLE [database.]table_name +EXPORT PART 'part_name' +TO TABLE [destination_database.]destination_table +SETTINGS allow_experimental_export_merge_tree_part = 1 + [, setting_name = value, ...] +``` + +## Syntax with table function + +```sql +ALTER TABLE [database.]table_name +EXPORT PART 'part_name' +TO TABLE FUNCTION s3(s3_conn, filename='table_function', partition_strategy...) +SETTINGS allow_experimental_export_merge_tree_part = 1 + [, setting_name = value, ...] +``` + +### Parameters + +- **`table_name`**: The source MergeTree table containing the part to export +- **`part_name`**: The exact name of the data part to export (e.g., `'2020_1_1_0'`, `'all_1_1_0'`) +- **`destination_table`**: The target table for the export (typically an S3, Azure, or other object storage table) + +## Requirements + +Source and destination tables must be 100% compatible: + +1. **Identical schemas** - same columns, types, and order +2. **Matching partition keys** - partition expressions must be identical + +In case a table function is used as the destination, the schema can be omitted and it will be inferred from the source table. + +## Settings + +### `allow_experimental_export_merge_tree_part` (Required) + +- **Type**: `Bool` +- **Default**: `false` +- **Description**: Must be set to `true` to enable the experimental feature. + +### `export_merge_tree_part_overwrite_file_if_exists` (Optional) + +- **Type**: `Bool` +- **Default**: `false` +- **Description**: If set to `true`, it will overwrite the file. Otherwise, fails with exception. + +### `export_merge_tree_part_max_bytes_per_file` (Optional) + +- **Type**: `UInt64` +- **Default**: `0` +- **Description**: Maximum number of bytes to write to a single file when exporting a merge tree part. 0 means no limit. This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. Using this might break idempotency, use it with care. + +### `export_merge_tree_part_max_rows_per_file` (Optional) + +- **Type**: `UInt64` +- **Default**: `0` +- **Description**: Maximum number of rows to write to a single file when exporting a merge tree part. 0 means no limit. This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. Using this might break idempotency, use it with care. + +### export_merge_tree_part_throw_on_pending_mutations + +- **Type**: `bool` +- **Default**: `true` +- **Description**: If set to true, throws if pending mutations exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. + +### export_merge_tree_part_throw_on_pending_patch_parts + +- **Type**: `bool` +- **Default**: `true` +- **Description**: If set to true, throws if pending patch parts exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. + +## Examples + +### Basic Export to S3 + +```sql +-- Create source and destination tables +CREATE TABLE mt_table (id UInt64, year UInt16) +ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +CREATE TABLE s3_table (id UInt64, year UInt16) +ENGINE = S3(s3_conn, filename='data', format=Parquet, partition_strategy='hive') +PARTITION BY year; + +-- Insert and export +INSERT INTO mt_table VALUES (1, 2020), (2, 2020), (3, 2021); + +ALTER TABLE mt_table EXPORT PART '2020_1_1_0' TO TABLE s3_table +SETTINGS allow_experimental_export_merge_tree_part = 1; + +ALTER TABLE mt_table EXPORT PART '2021_2_2_0' TO TABLE s3_table +SETTINGS allow_experimental_export_merge_tree_part = 1; +``` + +### Table function export + +```sql +-- Create source and destination tables +CREATE TABLE mt_table (id UInt64, year UInt16) +ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +-- Insert and export +INSERT INTO mt_table VALUES (1, 2020), (2, 2020), (3, 2021); + +ALTER TABLE mt_table EXPORT PART '2020_1_1_0' TO TABLE FUNCTION s3(s3_conn, filename='table_function', format=Parquet, partition_strategy='hive') PARTITION BY year +SETTINGS allow_experimental_export_merge_tree_part = 1; +``` + +## Monitoring + +### Active Exports + +Active exports can be found in the `system.exports` table. As of now, it only shows currently executing exports. It will not show pending or finished exports. + +```sql +arthur :) select * from system.exports; + +SELECT * +FROM system.exports + +Query id: 2026718c-d249-4208-891b-a271f1f93407 + +Row 1: +────── +source_database: default +source_table: source_mt_table +destination_database: default +destination_table: destination_table +create_time: 2025-11-19 09:09:11 +part_name: 20251016-365_1_1_0 +destination_file_paths: ['table_root/eventDate=2025-10-16/retention=365/20251016-365_1_1_0_17B2F6CD5D3C18E787C07AE3DAF16EB1.1.parquet'] +elapsed: 2.04845441 +rows_read: 1138688 -- 1.14 million +total_rows_to_read: 550961374 -- 550.96 million +total_size_bytes_compressed: 37619147120 -- 37.62 billion +total_size_bytes_uncompressed: 138166213721 -- 138.17 billion +bytes_read_uncompressed: 316892925 -- 316.89 million +memory_usage: 596006095 -- 596.01 million +peak_memory_usage: 601239033 -- 601.24 million +``` + +### Export History + +You can query succeeded or failed exports in `system.part_log`. For now, it only keeps track of completion events (either success or fails). + +```sql +arthur :) select * from system.part_log where event_type='ExportPart' and table = 'replicated_source' order by event_time desc limit 1; + +SELECT * +FROM system.part_log +WHERE (event_type = 'ExportPart') AND (`table` = 'replicated_source') +ORDER BY event_time DESC +LIMIT 1 + +Query id: ae1c1cd3-c20e-4f20-8b82-ed1f6af0237f + +Row 1: +────── +hostname: arthur +query_id: +event_type: ExportPart +merge_reason: NotAMerge +merge_algorithm: Undecided +event_date: 2025-11-19 +event_time: 2025-11-19 09:08:31 +event_time_microseconds: 2025-11-19 09:08:31.974701 +duration_ms: 4 +database: default +table: replicated_source +table_uuid: 78471c67-24f4-4398-9df5-ad0a6c3daf41 +part_name: 2021_0_0_0 +partition_id: 2021 +partition: 2021 +part_type: Compact +disk_name: default +path_on_disk: +remote_file_paths ['year=2021/2021_0_0_0_78C704B133D41CB0EF64DD2A9ED3B6BA.1.parquet'] +rows: 1 +size_in_bytes: 272 +merged_from: ['2021_0_0_0'] +bytes_uncompressed: 86 +read_rows: 1 +read_bytes: 6 +peak_memory_usage: 22 +error: 0 +exception: +ProfileEvents: {} +``` + +### Profile Events + +- `PartsExports` - Successful exports +- `PartsExportFailures` - Failed exports +- `PartsExportDuplicated` - Number of part exports that failed because target already exists. +- `PartsExportTotalMilliseconds` - Total time + +### Split large files + +```sql +alter table big_table export part '2025_0_32_3' to table replicated_big_destination SETTINGS export_merge_tree_part_max_bytes_per_file=10000000, output_format_parquet_row_group_size_bytes=5000000; + +arthur :) select * from system.exports; + +SELECT * +FROM system.exports + +Query id: d78d9ce5-cfbc-4957-b7dd-bc8129811634 + +Row 1: +────── +source_database: default +source_table: big_table +destination_database: default +destination_table: replicated_big_destination +create_time: 2025-12-15 13:12:48 +part_name: 2025_0_32_3 +destination_file_paths: ['replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.1.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.2.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.3.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.4.parquet'] +elapsed: 14.360427274 +rows_read: 10256384 -- 10.26 million +total_rows_to_read: 10485760 -- 10.49 million +total_size_bytes_compressed: 83779395 -- 83.78 million +total_size_bytes_uncompressed: 10611691600 -- 10.61 billion +bytes_read_uncompressed: 10440998912 -- 10.44 billion +memory_usage: 89795477 -- 89.80 million +peak_memory_usage: 107362133 -- 107.36 million + +1 row in set. Elapsed: 0.014 sec. + +arthur :) select * from system.part_log where event_type = 'ExportPart' order by event_time desc limit 1 format Vertical; + +SELECT * +FROM system.part_log +WHERE event_type = 'ExportPart' +ORDER BY event_time DESC +LIMIT 1 +FORMAT Vertical + +Query id: 95128b01-b751-4726-8e3e-320728ac6af7 + +Row 1: +────── +hostname: arthur +query_id: +event_type: ExportPart +merge_reason: NotAMerge +merge_algorithm: Undecided +event_date: 2025-12-15 +event_time: 2025-12-15 13:13:03 +event_time_microseconds: 2025-12-15 13:13:03.197492 +duration_ms: 14673 +database: default +table: big_table +table_uuid: a3eeeea0-295c-41a3-84ef-6b5463dbbe8c +part_name: 2025_0_32_3 +partition_id: 2025 +partition: 2025 +part_type: Wide +disk_name: default +path_on_disk: ./store/a3e/a3eeeea0-295c-41a3-84ef-6b5463dbbe8c/2025_0_32_3/ +remote_file_paths: ['replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.1.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.2.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.3.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.4.parquet'] +rows: 10485760 -- 10.49 million +size_in_bytes: 83779395 -- 83.78 million +merged_from: ['2025_0_32_3'] +bytes_uncompressed: 10611691600 -- 10.61 billion +read_rows: 10485760 -- 10.49 million +read_bytes: 10674503680 -- 10.67 billion +peak_memory_usage: 107362133 -- 107.36 million +error: 0 +exception: +ProfileEvents: {} + +1 row in set. Elapsed: 0.044 sec. + +arthur :) select _path, formatReadableSize(_size) as _size from s3(s3_conn, filename='**', format=One); + +SELECT + _path, + formatReadableSize(_size) AS _size +FROM s3(s3_conn, filename = '**', format = One) + +Query id: c48ae709-f590-4d1b-8158-191f8d628966 + + ┌─_path────────────────────────────────────────────────────────────────────────────────┬─_size─────┐ +1. │ test/replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.1.parquet │ 17.36 MiB │ +2. │ test/replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.2.parquet │ 17.32 MiB │ +3. │ test/replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.4.parquet │ 5.04 MiB │ +4. │ test/replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.3.parquet │ 17.40 MiB │ +5. │ test/replicated_big/year=2025/commit_2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7 │ 320.00 B │ + └──────────────────────────────────────────────────────────────────────────────────────┴───────────┘ + +5 rows in set. Elapsed: 0.072 sec. +``` diff --git a/docs/en/engines/table-engines/mergetree-family/partition_export.md b/docs/en/engines/table-engines/mergetree-family/partition_export.md new file mode 100644 index 000000000000..d91f226dbbf6 --- /dev/null +++ b/docs/en/engines/table-engines/mergetree-family/partition_export.md @@ -0,0 +1,182 @@ +# ALTER TABLE EXPORT PARTITION + +## Overview + +The `ALTER TABLE EXPORT PARTITION` command exports entire partitions from Replicated*MergeTree tables to object storage (S3, Azure Blob Storage, etc.), typically in Parquet format. This feature coordinates export part operations across all replicas using ZooKeeper. + +Each MergeTree part will become a separate file with the following name convention: `//_.`. To ensure atomicity, a commit file containing the relative paths of all exported parts is also shipped. A data file should only be considered part of the dataset if a commit file references it. The commit file will be named using the following convention: `/commit__`. + +The set of parts that are exported is based on the list of parts the replica that received the export command sees. The other replicas will assist in the export process if they have those parts locally. Otherwise they will ignore it. + +The partition export tasks can be observed through `system.replicated_partition_exports`. Querying this table results in a query to ZooKeeper, so it must be used with care. Individual part export progress can be observed as usual through `system.exports`. + +The same partition can not be exported to the same destination more than once. There are two ways to override this behavior: either by setting the `export_merge_tree_partition_force_export` setting or waiting for the task to expire. + +The export task can be killed by issuing the kill command: `KILL EXPORT PARTITION `. + +The task is persistent - it should be resumed after crashes, failures and etc. + +## Syntax + +```sql +ALTER TABLE [database.]table_name +EXPORT PARTITION ID 'partition_id' +TO TABLE [destination_database.]destination_table +[SETTINGS setting_name = value, ...] +``` + +### Parameters + +- **`table_name`**: The source Replicated*MergeTree table containing the partition to export +- **`partition_id`**: The partition identifier to export (e.g., `'2020'`, `'2021'`) +- **`destination_table`**: The target table for the export (typically an S3, Azure, or other object storage table) + +## Settings + +### Server Settings + +#### `enable_experimental_export_merge_tree_partition_feature` (Required) + +- **Type**: `Bool` +- **Default**: `false` +- **Description**: Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use. + +### Query Settings + +#### `export_merge_tree_partition_force_export` (Optional) + +- **Type**: `Bool` +- **Default**: `false` +- **Description**: Ignore existing partition export and overwrite the ZooKeeper entry. Allows re-exporting a partition to the same destination before the manifest expires. + +#### `export_merge_tree_partition_max_retries` (Optional) + +- **Type**: `UInt64` +- **Default**: `3` +- **Description**: Maximum number of retries for exporting a merge tree part in an export partition task. If it exceeds, the entire task fails. + +#### `export_merge_tree_partition_manifest_ttl` (Optional) + +- **Type**: `UInt64` +- **Default**: `180` (seconds) +- **Description**: Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. This setting does not affect or delete in-progress tasks; it only cleans up completed ones. + +#### `export_merge_tree_part_file_already_exists_policy` (Optional) + +- **Type**: `MergeTreePartExportFileAlreadyExistsPolicy` +- **Default**: `skip` +- **Description**: Policy for handling files that already exist during export. Possible values: + - `skip` - Skip the file if it already exists + - `error` - Throw an error if the file already exists + - `overwrite` - Overwrite the file + +### export_merge_tree_part_throw_on_pending_mutations + +- **Type**: `bool` +- **Default**: `true` +- **Description**: If set to true, throws if pending mutations exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. + +### export_merge_tree_part_throw_on_pending_patch_parts + +- **Type**: `bool` +- **Default**: `true` +- **Description**: If set to true, throws if pending patch parts exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. + +## Examples + +### Basic Export to S3 + +```sql +CREATE TABLE rmt_table (id UInt64, year UInt16) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/rmt_table', 'replica1') +PARTITION BY year ORDER BY tuple(); + +CREATE TABLE s3_table (id UInt64, year UInt16) +ENGINE = S3(s3_conn, filename='data', format=Parquet, partition_strategy='hive') +PARTITION BY year; + +INSERT INTO rmt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021); + +ALTER TABLE rmt_table EXPORT PARTITION ID '2020' TO TABLE s3_table; + +## Killing Exports + +You can cancel in-progress partition exports using the `KILL EXPORT PARTITION` command: + +```sql +KILL EXPORT PARTITION +WHERE partition_id = '2020' + AND source_table = 'rmt_table' + AND destination_table = 's3_table' +``` + +The `WHERE` clause filters exports from the `system.replicated_partition_exports` table. You can use any columns from that table in the filter. + +## Monitoring + +### Active and Completed Exports + +Monitor partition exports using the `system.replicated_partition_exports` table: + +```sql +arthur :) select * from system.replicated_partition_exports Format Vertical; + +SELECT * +FROM system.replicated_partition_exports +FORMAT Vertical + +Query id: 9efc271a-a501-44d1-834f-bc4d20156164 + +Row 1: +────── +source_database: default +source_table: replicated_source +destination_database: default +destination_table: replicated_destination +create_time: 2025-11-21 18:21:51 +partition_id: 2022 +transaction_id: 7397746091717128192 +source_replica: r1 +parts: ['2022_0_0_0','2022_1_1_0','2022_2_2_0'] +parts_count: 3 +parts_to_do: 0 +status: COMPLETED +exception_replica: +last_exception: +exception_part: +exception_count: 0 + +Row 2: +────── +source_database: default +source_table: replicated_source +destination_database: default +destination_table: replicated_destination +create_time: 2025-11-21 18:20:35 +partition_id: 2021 +transaction_id: 7397745772618674176 +source_replica: r1 +parts: ['2021_0_0_0'] +parts_count: 1 +parts_to_do: 0 +status: COMPLETED +exception_replica: +last_exception: +exception_part: +exception_count: 0 + +2 rows in set. Elapsed: 0.019 sec. + +arthur :) +``` + +Status values include: +- `PENDING` - Export is queued / in progress +- `COMPLETED` - Export finished successfully +- `FAILED` - Export failed +- `KILLED` - Export was cancelled + +## Related Features + +- [ALTER TABLE EXPORT PART](/docs/en/engines/table-engines/mergetree-family/part_export.md) - Export individual parts (non-replicated) + diff --git a/docs/en/operations/system-tables/exports.md b/docs/en/operations/system-tables/exports.md new file mode 100644 index 000000000000..e26514364008 --- /dev/null +++ b/docs/en/operations/system-tables/exports.md @@ -0,0 +1,56 @@ +--- +description: 'System table containing information about in progress merge tree part exports' +keywords: ['system table', 'exports', 'merge tree', 'part'] +slug: /operations/system-tables/exports +title: 'system.exports' +--- + +Contains information about in progress merge tree part exports + +Columns: + +- `source_database` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the source database. +- `source_table` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the source table. +- `destination_database` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the destination database. +- `destination_table` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the destination table. +- `create_time` ([DateTime](/docs/en/sql-reference/data-types/datetime.md)) — Date and time when the export command was received in the server. +- `part_name` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the part. +- `destination_file_path` ([String](/docs/en/sql-reference/data-types/string.md)) — File path relative to where the part is being exported to. +- `elapsed` ([Float64](/docs/en/sql-reference/data-types/float.md)) — The time elapsed (in seconds) since the export started. +- `rows_read` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The number of rows read from the exported part. +- `total_rows_to_read` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The total number of rows to read from the exported part. +- `total_size_bytes_compressed` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The total size of the compressed data in the exported part. +- `total_size_bytes_uncompressed` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The total size of the uncompressed data in the exported part. +- `bytes_read_uncompressed` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The number of uncompressed bytes read from the exported part. +- `memory_usage` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — Current memory usage in bytes for the export operation. +- `peak_memory_usage` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — Peak memory usage in bytes during the export operation. + +**Example** + +```sql +arthur :) select * from system.exports; + +SELECT * +FROM system.exports + +Query id: 2026718c-d249-4208-891b-a271f1f93407 + +Row 1: +────── +source_database: default +source_table: source_mt_table +destination_database: default +destination_table: destination_table +create_time: 2025-11-19 09:09:11 +part_name: 20251016-365_1_1_0 +destination_file_path: table_root/eventDate=2025-10-16/retention=365/20251016-365_1_1_0_17B2F6CD5D3C18E787C07AE3DAF16EB1.parquet +elapsed: 2.04845441 +rows_read: 1138688 -- 1.14 million +total_rows_to_read: 550961374 -- 550.96 million +total_size_bytes_compressed: 37619147120 -- 37.62 billion +total_size_bytes_uncompressed: 138166213721 -- 138.17 billion +bytes_read_uncompressed: 316892925 -- 316.89 million +memory_usage: 596006095 -- 596.01 million +peak_memory_usage: 601239033 -- 601.24 million +``` + diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 0eaa21082a5c..ceb86f9e02fd 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -213,6 +213,8 @@ enum class AccessType : uint8_t M(ALTER_REWRITE_PARTS, "REWRITE PARTS", TABLE, ALTER_TABLE) /* allows to execute ALTER REWRITE PARTS */\ M(ALTER_SETTINGS, "ALTER SETTING, ALTER MODIFY SETTING, MODIFY SETTING, RESET SETTING", TABLE, ALTER_TABLE) /* allows to execute ALTER MODIFY SETTING */\ M(ALTER_MOVE_PARTITION, "ALTER MOVE PART, MOVE PARTITION, MOVE PART", TABLE, ALTER_TABLE) \ + M(ALTER_EXPORT_PART, "ALTER EXPORT PART, EXPORT PART", TABLE, ALTER_TABLE) \ + M(ALTER_EXPORT_PARTITION, "ALTER EXPORT PARTITION, EXPORT PARTITION", TABLE, ALTER_TABLE) \ M(ALTER_FETCH_PARTITION, "ALTER FETCH PART, FETCH PARTITION", TABLE, ALTER_TABLE) \ M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \ M(ALTER_UNLOCK_SNAPSHOT, "UNLOCK SNAPSHOT", TABLE, ALTER_TABLE) \ diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 531fae5da11c..5192e4ec2136 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -136,6 +136,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage/Azure) add_headers_and_sources(dbms Storages/ObjectStorage/S3) add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Storages/ObjectStorage/Local) +add_headers_and_sources(dbms Storages/ObjectStorage/MergeTree) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes/Common) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes/Iceberg) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 2d21d053986d..0c4edd4a2f14 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -10,6 +10,7 @@ M(Merge, "Number of executing background merges") \ M(MergeParts, "Number of source parts participating in current background merges") \ M(Move, "Number of currently executing moves") \ + M(Export, "Number of currently executing exports") \ M(PartMutation, "Number of mutations (ALTER DELETE/UPDATE)") \ M(ReplicatedFetch, "Number of data parts being fetched from replica") \ M(ReplicatedSend, "Number of data parts being sent to replicas") \ diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 8b32f9be80b7..a66f6a2d5058 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -657,6 +657,7 @@ M(1002, UNKNOWN_EXCEPTION) \ M(1003, SSH_EXCEPTION) \ M(1004, STARTUP_SCRIPTS_ERROR) \ + M(1005, PENDING_MUTATIONS_NOT_ALLOWED) \ /* See END */ #ifdef APPLY_FOR_EXTERNAL_ERROR_CODES @@ -673,7 +674,7 @@ namespace ErrorCodes APPLY_FOR_ERROR_CODES(M) #undef M - constexpr ErrorCode END = 1004; + constexpr ErrorCode END = 1005; ErrorPairHolder values[END + 1]{}; struct ErrorCodesNames diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 3f9bd174afeb..7db179a1c005 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -39,6 +39,10 @@ M(FailedInitialQuery, "Number of failed initial queries.", ValueType::Number) \ M(FailedInitialSelectQuery, "Same as FailedInitialQuery, but only for SELECT queries.", ValueType::Number) \ M(FailedQuery, "Number of total failed queries, both internal and user queries.", ValueType::Number) \ + M(PartsExports, "Number of successful part exports.", ValueType::Number) \ + M(PartsExportFailures, "Number of failed part exports.", ValueType::Number) \ + M(PartsExportDuplicated, "Number of part exports that failed because target already exists.", ValueType::Number) \ + M(PartsExportTotalMilliseconds, "Total time spent on part export operations.", ValueType::Milliseconds) \ M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.", ValueType::Number) \ M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.", ValueType::Number) \ M(FailedAsyncInsertQuery, "Number of failed ASYNC INSERT queries.", ValueType::Number) \ @@ -218,6 +222,8 @@ M(MergesThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_merges_bandwidth_for_server' throttling.", ValueType::Microseconds) \ M(MutationsThrottlerBytes, "Bytes passed through 'max_mutations_bandwidth_for_server' throttler.", ValueType::Bytes) \ M(MutationsThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_mutations_bandwidth_for_server' throttling.", ValueType::Microseconds) \ + M(ExportsThrottlerBytes, "Bytes passed through 'max_exports_bandwidth_for_server' throttler.", ValueType::Bytes) \ + M(ExportsThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_exports_bandwidth_for_server' throttling.", ValueType::Microseconds) \ M(QueryRemoteReadThrottlerBytes, "Bytes passed through 'max_remote_read_network_bandwidth' throttler.", ValueType::Bytes) \ M(QueryRemoteReadThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_remote_read_network_bandwidth' throttling.", ValueType::Microseconds) \ M(QueryRemoteWriteThrottlerBytes, "Bytes passed through 'max_remote_write_network_bandwidth' throttler.", ValueType::Bytes) \ diff --git a/src/Common/setThreadName.h b/src/Common/setThreadName.h index 238fb3f4d119..7c0ad6a6ce61 100644 --- a/src/Common/setThreadName.h +++ b/src/Common/setThreadName.h @@ -157,6 +157,7 @@ namespace DB M(ZOOKEEPER_ACL_WATCHER, "ZooACLWatch") \ M(ZOOKEEPER_RECV, "ZooKeeperRecv") \ M(ZOOKEEPER_SEND, "ZooKeeperSend") \ + M(EXPORT_PART, "ExportPart") \ enum class ThreadName : uint8_t diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 9df634a0a8b1..d4e9eb982441 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -134,6 +134,7 @@ namespace DECLARE(UInt64, max_unexpected_parts_loading_thread_pool_size, 8, R"(The number of threads to load inactive set of data parts (Unexpected ones) at startup.)", 0) \ DECLARE(UInt64, max_parts_cleaning_thread_pool_size, 128, R"(The number of threads for concurrent removal of inactive data parts.)", 0) \ DECLARE(UInt64, max_mutations_bandwidth_for_server, 0, R"(The maximum read speed of all mutations on server in bytes per second. Zero means unlimited.)", 0) \ + DECLARE(UInt64, max_exports_bandwidth_for_server, 0, R"(The maximum read speed of all exports on server in bytes per second. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_merges_bandwidth_for_server, 0, R"(The maximum read speed of all merges on server in bytes per second. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, R"(The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, R"(The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.)", 0) \ @@ -1469,7 +1470,8 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ ```xml 1 ``` - )", 0) + )", 0) \ + DECLARE(Bool, enable_experimental_export_merge_tree_partition_feature, false, "Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use.", 0) /// Settings with a path are server settings with at least one layer of nesting that have a fixed structure (no lists, lists, enumerations, repetitions, ...). #define LIST_OF_SERVER_SETTINGS_WITH_PATH(DECLARE, ALIAS) \ diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index e72653747d54..539e289f40d6 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -7315,6 +7315,39 @@ Allows creation of [QBit](../../sql-reference/data-types/qbit.md) data type. )", BETA, allow_experimental_qbit_type) \ DECLARE(UInt64, archive_adaptive_buffer_max_size_bytes, 8 * DBMS_DEFAULT_BUFFER_SIZE, R"( Limits the maximum size of the adaptive buffer used when writing to archive files (for example, tar archives)", 0) \ + DECLARE(Bool, export_merge_tree_part_overwrite_file_if_exists, false, R"( +Overwrite file if it already exists when exporting a merge tree part +)", 0) \ + DECLARE(Bool, export_merge_tree_partition_force_export, false, R"( +Ignore existing partition export and overwrite the zookeeper entry +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_max_retries, 3, R"( +Maximum number of retries for exporting a merge tree part in an export partition task +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_manifest_ttl, 180, R"( +Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. +This setting does not affect / delete in progress tasks. It'll only cleanup the completed ones. +)", 0) \ + DECLARE(MergeTreePartExportFileAlreadyExistsPolicy, export_merge_tree_part_file_already_exists_policy, MergeTreePartExportFileAlreadyExistsPolicy::skip, R"( +Possible values: +- skip - Skip the file if it already exists. +- error - Throw an error if the file already exists. +- overwrite - Overwrite the file. +)", 0) \ + DECLARE(UInt64, export_merge_tree_part_max_bytes_per_file, 0, R"( +Maximum number of bytes to write to a single file when exporting a merge tree part. 0 means no limit. +This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. +)", 0) \ + DECLARE(UInt64, export_merge_tree_part_max_rows_per_file, 0, R"( +Maximum number of rows to write to a single file when exporting a merge tree part. 0 means no limit. +This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. +)", 0) \ + DECLARE(Bool, export_merge_tree_part_throw_on_pending_mutations, true, R"( +Throw an error if there are pending mutations when exporting a merge tree part. +)", 0) \ + DECLARE(Bool, export_merge_tree_part_throw_on_pending_patch_parts, true, R"( +Throw an error if there are pending patch parts when exporting a merge tree part. +)", 0) \ \ /* ####################################################### */ \ /* ########### START OF EXPERIMENTAL FEATURES ############ */ \ @@ -7525,6 +7558,9 @@ Rewrite expressions like 'x IN subquery' to JOIN. This might be useful for optim DECLARE_WITH_ALIAS(Bool, allow_experimental_time_series_aggregate_functions, false, R"( Experimental timeSeries* aggregate functions for Prometheus-like timeseries resampling, rate, delta calculation. )", EXPERIMENTAL, allow_experimental_ts_to_grid_aggregate_function) \ + DECLARE(Bool, allow_experimental_export_merge_tree_part, true, R"( +Experimental export merge tree part. +)", EXPERIMENTAL) \ \ DECLARE(String, promql_database, "", R"( Specifies the database name used by the 'promql' dialect. Empty string means the current database. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a80702f3a87..7b2f7eb8536d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -82,6 +82,7 @@ class WriteBuffer; M(CLASS_NAME, LogsLevel) \ M(CLASS_NAME, Map) \ M(CLASS_NAME, MaxThreads) \ + M(CLASS_NAME, MergeTreePartExportFileAlreadyExistsPolicy) \ M(CLASS_NAME, Milliseconds) \ M(CLASS_NAME, MsgPackUUIDRepresentation) \ M(CLASS_NAME, MySQLDataTypesSupport) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index e8686e2a43ea..e0481eb504fc 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -78,6 +78,16 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", true, false, "It becomes obsolete."}, {"database_datalake_require_metadata_access", true, true, "New setting."}, {"automatic_parallel_replicas_min_bytes_per_replica", 0, 1_MiB, "Better default value derived from testing results"}, + {"allow_experimental_export_merge_tree_part", false, false, "New setting."}, + {"export_merge_tree_part_overwrite_file_if_exists", false, false, "New setting."}, + {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, + {"export_merge_tree_partition_force_export", false, false, "New setting."}, + {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, + {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, + {"export_merge_tree_part_max_bytes_per_file", 0, 0, "New setting."}, + {"export_merge_tree_part_max_rows_per_file", 0, 0, "New setting."}, + {"export_merge_tree_part_throw_on_pending_mutations", true, true, "New setting."}, + {"export_merge_tree_part_throw_on_pending_patch_parts", true, true, "New setting."}, }); addSettingsChanges(settings_changes_history, "25.12", { @@ -220,6 +230,17 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"os_threads_nice_value_query", 0, 0, "New setting."}, {"os_threads_nice_value_materialized_view", 0, 0, "New setting."}, {"os_thread_priority", 0, 0, "Alias for os_threads_nice_value_query."}, + {"allow_experimental_export_merge_tree_part", false, false, "New setting."}, + {"export_merge_tree_part_overwrite_file_if_exists", false, false, "New setting."}, + {"allow_experimental_export_merge_tree_part", false, true, "Turned ON by default for Antalya."}, + {"export_merge_tree_partition_force_export", false, false, "New setting."}, + {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, + {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, + {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, + {"export_merge_tree_part_max_bytes_per_file", 0, 0, "New setting."}, + {"export_merge_tree_part_max_rows_per_file", 0, 0, "New setting."}, + {"export_merge_tree_part_throw_on_pending_mutations", true, true, "New setting."}, + {"export_merge_tree_part_throw_on_pending_patch_parts", true, true, "New setting."}, }); addSettingsChanges(settings_changes_history, "25.8", { @@ -312,6 +333,11 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_experimental_insert_into_iceberg", false, false, "New setting."}, /// RELEASE CLOSED }); + addSettingsChanges(settings_changes_history, "25.6.5.2000", + { + {"allow_experimental_export_merge_tree_part", false, false, "New setting."}, + {"export_merge_tree_part_overwrite_file_if_exists", false, false, "New setting."}, + }); addSettingsChanges(settings_changes_history, "25.6", { /// RELEASE CLOSED diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index c3fea7d48a48..d0b22c095dad 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -440,4 +440,7 @@ IMPLEMENT_SETTING_ENUM(DeduplicateInsertSelectMode, ErrorCodes::BAD_ARGUMENTS, {"force_enable", DeduplicateInsertSelectMode::FORCE_ENABLE}, {"disable", DeduplicateInsertSelectMode::DISABLE}, {"enable_even_for_bad_queries", DeduplicateInsertSelectMode::ENABLE_EVEN_FOR_BAD_QUERIES}}) + +IMPLEMENT_SETTING_AUTO_ENUM(MergeTreePartExportFileAlreadyExistsPolicy, ErrorCodes::BAD_ARGUMENTS); + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index aab55857d2a6..61b75eed5631 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -562,4 +562,13 @@ enum class DeduplicateInsertSelectMode : uint8_t DECLARE_SETTING_ENUM(DeduplicateInsertSelectMode) +enum class MergeTreePartExportFileAlreadyExistsPolicy : uint8_t +{ + skip, + error, + overwrite, +}; + +DECLARE_SETTING_ENUM(MergeTreePartExportFileAlreadyExistsPolicy) + } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f62fb51883f7..fed236e596b0 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -2430,7 +2430,7 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context, if (const auto * alter = query_ptr->as()) { if (alter->isAttachAlter() || alter->isFetchAlter() || alter->isDropPartitionAlter() || alter->isFreezeAlter() - || alter->isUnlockSnapshot()) + || alter->isUnlockSnapshot() || alter->isExportPartOrExportPartitionAlter()) return false; // Allowed ALTER operation on KeeperMap still should be replicated diff --git a/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h b/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h index e2605c10438e..a541ac722695 100644 --- a/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h +++ b/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h @@ -130,6 +130,7 @@ struct RelativePathWithMetadata std::string getFileName() const { return std::filesystem::path(relative_path).filename(); } std::string getPath() const { return relative_path; } + std::string getFileNameWithoutExtension() const { return std::filesystem::path(relative_path).stem(); } }; struct ObjectKeyWithMetadata diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 557277cceeeb..b39c74e0576e 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -154,6 +154,11 @@ uint64_t generateSnowflakeID() return fromSnowflakeId(snowflake_id); } +std::string generateSnowflakeIDString() +{ + return std::to_string(generateSnowflakeID()); +} + class FunctionGenerateSnowflakeID : public IFunction { public: diff --git a/src/Functions/generateSnowflakeID.h b/src/Functions/generateSnowflakeID.h index 38fa684a9b4b..4fc173dcf1be 100644 --- a/src/Functions/generateSnowflakeID.h +++ b/src/Functions/generateSnowflakeID.h @@ -7,4 +7,6 @@ namespace DB uint64_t generateSnowflakeID(); +std::string generateSnowflakeIDString(); + } diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 3a8a3a03685f..4385eaa64982 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -143,6 +143,7 @@ class ClientInfo NOT_A_BACKGROUND_OPERATION = 0, MERGE = 1, MUTATION = 2, + EXPORT_PART = 3, }; /// It's ClientInfo and context created for background operation (not real query) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 82756929cce9..6b76189723a0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -158,6 +159,8 @@ namespace ProfileEvents extern const Event BackupThrottlerSleepMicroseconds; extern const Event MergesThrottlerBytes; extern const Event MergesThrottlerSleepMicroseconds; + extern const Event ExportsThrottlerBytes; + extern const Event ExportsThrottlerSleepMicroseconds; extern const Event MutationsThrottlerBytes; extern const Event MutationsThrottlerSleepMicroseconds; extern const Event QueryLocalReadThrottlerBytes; @@ -351,6 +354,7 @@ namespace ServerSetting extern const ServerSettingsUInt64 max_local_write_bandwidth_for_server; extern const ServerSettingsUInt64 max_merges_bandwidth_for_server; extern const ServerSettingsUInt64 max_mutations_bandwidth_for_server; + extern const ServerSettingsUInt64 max_exports_bandwidth_for_server; extern const ServerSettingsUInt64 max_remote_read_network_bandwidth_for_server; extern const ServerSettingsUInt64 max_remote_write_network_bandwidth_for_server; extern const ServerSettingsUInt64 max_replicated_fetches_network_bandwidth_for_server; @@ -540,6 +544,7 @@ struct ContextSharedPart : boost::noncopyable GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) + ExportsList exports_list; /// The list of executing exports (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; RefreshSet refresh_set; /// The list of active refreshes (for MaterializedView) ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. @@ -584,6 +589,8 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr distributed_cache_read_throttler; /// A server-wide throttler for distributed cache read mutable ThrottlerPtr distributed_cache_write_throttler; /// A server-wide throttler for distributed cache write + mutable ThrottlerPtr exports_throttler; /// A server-wide throttler for exports + MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker TSA_GUARDED_BY(mutex); /// Process ddl commands from zk. LoadTaskPtr ddl_worker_startup_task; /// To postpone `ddl_worker->startup()` after all tables startup @@ -1104,6 +1111,9 @@ struct ContextSharedPart : boost::noncopyable if (auto bandwidth = server_settings[ServerSetting::max_merges_bandwidth_for_server]) merges_throttler = std::make_shared(bandwidth, ProfileEvents::MergesThrottlerBytes, ProfileEvents::MergesThrottlerSleepMicroseconds); + + if (auto bandwidth = server_settings[ServerSetting::max_exports_bandwidth_for_server]) + exports_throttler = std::make_shared(bandwidth, ProfileEvents::ExportsThrottlerBytes, ProfileEvents::ExportsThrottlerSleepMicroseconds); } }; @@ -1262,6 +1272,8 @@ MergeList & Context::getMergeList() { return shared->merge_list; } const MergeList & Context::getMergeList() const { return shared->merge_list; } MovesList & Context::getMovesList() { return shared->moves_list; } const MovesList & Context::getMovesList() const { return shared->moves_list; } +ExportsList & Context::getExportsList() { return shared->exports_list; } +const ExportsList & Context::getExportsList() const { return shared->exports_list; } ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; } RefreshSet & Context::getRefreshSet() { return shared->refresh_set; } @@ -3299,6 +3311,13 @@ void Context::makeQueryContextForMutate(const MergeTreeSettings & merge_tree_set = merge_tree_settings[MergeTreeSetting::mutation_workload].value.empty() ? getMutationWorkload() : merge_tree_settings[MergeTreeSetting::mutation_workload]; } +void Context::makeQueryContextForExportPart() +{ + makeQueryContext(); + classifier.reset(); // It is assumed that there are no active queries running using this classifier, otherwise this will lead to crashes + // Export part operations don't have a specific workload setting, so we leave the default workload +} + void Context::makeSessionContext() { session_context = shared_from_this(); @@ -4466,6 +4485,11 @@ ThrottlerPtr Context::getDistributedCacheWriteThrottler() const return shared->distributed_cache_write_throttler; } +ThrottlerPtr Context::getExportsThrottler() const +{ + return shared->exports_throttler; +} + void Context::reloadRemoteThrottlerConfig(size_t read_bandwidth, size_t write_bandwidth) const { if (read_bandwidth) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e5426b3dba56..49133e1c7758 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -91,6 +91,7 @@ class AsynchronousMetrics; class BackgroundSchedulePool; class MergeList; class MovesList; +class ExportsList; class ReplicatedFetchList; class RefreshSet; class Cluster; @@ -1182,6 +1183,7 @@ class Context: public ContextData, public std::enable_shared_from_this void makeQueryContext(); void makeQueryContextForMerge(const MergeTreeSettings & merge_tree_settings); void makeQueryContextForMutate(const MergeTreeSettings & merge_tree_settings); + void makeQueryContextForExportPart(); void makeSessionContext(); void makeGlobalContext(); @@ -1217,6 +1219,9 @@ class Context: public ContextData, public std::enable_shared_from_this MovesList & getMovesList(); const MovesList & getMovesList() const; + ExportsList & getExportsList(); + const ExportsList & getExportsList() const; + ReplicatedFetchList & getReplicatedFetchList(); const ReplicatedFetchList & getReplicatedFetchList() const; @@ -1749,6 +1754,7 @@ class Context: public ContextData, public std::enable_shared_from_this ThrottlerPtr getMutationsThrottler() const; ThrottlerPtr getMergesThrottler() const; + ThrottlerPtr getExportsThrottler() const; ThrottlerPtr getDistributedCacheReadThrottler() const; ThrottlerPtr getDistributedCacheWriteThrottler() const; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 91f899ec494e..a0a51ac7665a 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -776,7 +776,8 @@ bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr & ast_ddl, const Stora alter->isFreezeAlter() || alter->isUnlockSnapshot() || alter->isMovePartitionToDiskOrVolumeAlter() || - alter->isCommentAlter()) + alter->isCommentAlter() || + alter->isExportPartOrExportPartitionAlter()) return false; } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 7aaf9d2dde64..ffa5134a45d2 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -573,6 +573,20 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_DELETE | AccessType::INSERT, database, table); break; } + case ASTAlterCommand::EXPORT_PART: + { + required_access.emplace_back(AccessType::ALTER_EXPORT_PART, database, table); + /// For table functions, access control is handled by the table function itself + if (!command.to_table_function) + required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); + break; + } + case ASTAlterCommand::EXPORT_PARTITION: + { + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION, database, table); + required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); + break; + } case ASTAlterCommand::FETCH_PARTITION: { required_access.emplace_back(AccessType::ALTER_FETCH_PARTITION, database, table); diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 160e049f80e6..a8d182f73d4e 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -37,11 +38,17 @@ namespace Setting extern const SettingsUInt64 max_parser_depth; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ACCESS_DENIED; extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; } @@ -250,6 +257,82 @@ BlockIO InterpreterKillQueryQuery::execute() break; } + case ASTKillQueryQuery::Type::ExportPartition: + { + if (!getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + } + + Block exports_block = getSelectResult( + "source_database, source_table, transaction_id, destination_database, destination_table, partition_id", + "system.replicated_partition_exports"); + if (exports_block.empty()) + return res_io; + + const ColumnString & src_db_col = typeid_cast(*exports_block.getByName("source_database").column); + const ColumnString & src_table_col = typeid_cast(*exports_block.getByName("source_table").column); + const ColumnString & dst_db_col = typeid_cast(*exports_block.getByName("destination_database").column); + const ColumnString & dst_table_col = typeid_cast(*exports_block.getByName("destination_table").column); + const ColumnString & tx_col = typeid_cast(*exports_block.getByName("transaction_id").column); + + auto header = exports_block.cloneEmpty(); + header.insert(0, {ColumnString::create(), std::make_shared(), "kill_status"}); + + MutableColumns res_columns = header.cloneEmptyColumns(); + AccessRightsElements required_access_rights; + auto access = getContext()->getAccess(); + bool access_denied = false; + + for (size_t i = 0; i < exports_block.rows(); ++i) + { + const auto src_database = src_db_col.getDataAt(i); + const auto src_table = src_table_col.getDataAt(i); + const auto dst_database = dst_db_col.getDataAt(i); + const auto dst_table = dst_table_col.getDataAt(i); + + const auto table_id = StorageID{std::string{src_database}, std::string{src_table}}; + const auto transaction_id = tx_col.getDataAt(i); + + CancellationCode code = CancellationCode::Unknown; + if (!query.test) + { + auto storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); + if (!storage) + code = CancellationCode::NotFound; + else + { + ASTAlterCommand alter_command{}; + alter_command.type = ASTAlterCommand::EXPORT_PARTITION; + alter_command.move_destination_type = DataDestinationType::TABLE; + alter_command.from_database = src_database; + alter_command.from_table = src_table; + alter_command.to_database = dst_database; + alter_command.to_table = dst_table; + + required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand( + alter_command, table_id.database_name, table_id.table_name); + if (!access->isGranted(required_access_rights)) + { + access_denied = true; + continue; + } + code = storage->killExportPartition(std::string{transaction_id}); + } + } + + insertResultRow(i, code, exports_block, header, res_columns); + } + + if (res_columns[0]->empty() && access_denied) + throw Exception(ErrorCodes::ACCESS_DENIED, "Not allowed to kill export partition. " + "To execute this query, it's necessary to have the grant {}", required_access_rights.toString()); + + res_io.pipeline = QueryPipeline(Pipe(std::make_shared(std::make_shared(header.cloneWithColumns(std::move(res_columns)))))); + + break; + } case ASTKillQueryQuery::Type::Mutation: { Block mutations_block = getSelectResult("database, table, mutation_id, command", "system.mutations"); @@ -471,6 +554,9 @@ AccessRightsElements InterpreterKillQueryQuery::getRequiredAccessForDDLOnCluster | AccessType::ALTER_MATERIALIZE_TTL | AccessType::ALTER_REWRITE_PARTS ); + /// todo arthur think about this + else if (query.type == ASTKillQueryQuery::Type::ExportPartition) + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION); return required_access; } diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 8f5eee5703b0..294de307dfde 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -72,6 +72,7 @@ ColumnsDescription PartLogElement::getColumnsDescription() {"MovePart", static_cast(MOVE_PART)}, {"MergePartsStart", static_cast(MERGE_PARTS_START)}, {"MutatePartStart", static_cast(MUTATE_PART_START)}, + {"ExportPart", static_cast(EXPORT_PART)}, } ); @@ -113,7 +114,8 @@ ColumnsDescription PartLogElement::getColumnsDescription() "RemovePart — Removing or detaching a data part using [DETACH PARTITION](/sql-reference/statements/alter/partition#detach-partitionpart)." "MutatePartStart — Mutating of a data part has started, " "MutatePart — Mutating of a data part has finished, " - "MovePart — Moving the data part from the one disk to another one."}, + "MovePart — Moving the data part from the one disk to another one." + "ExportPart — Exporting the data part from a MergeTree table into a target table that represents external storage (e.g., object storage or a data lake).."}, {"merge_reason", std::move(merge_reason_datatype), "The reason for the event with type MERGE_PARTS. Can have one of the following values: " "NotAMerge — The current event has the type other than MERGE_PARTS, " @@ -136,6 +138,7 @@ ColumnsDescription PartLogElement::getColumnsDescription() {"part_type", std::make_shared(), "The type of the part. Possible values: Wide and Compact."}, {"disk_name", std::make_shared(), "The disk name data part lies on."}, {"path_on_disk", std::make_shared(), "Absolute path to the folder with data part files."}, + {"remote_file_paths", std::make_shared(std::make_shared()), "In case of an export operation to remote storages, the file paths a given export generated"}, {"rows", std::make_shared(), "The number of rows in the data part."}, {"size_in_bytes", std::make_shared(), "Size of the data part on disk in bytes."}, @@ -190,6 +193,12 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(disk_name); columns[i++]->insert(path_on_disk); + Array remote_file_paths_array; + remote_file_paths_array.reserve(remote_file_paths.size()); + for (const auto & remote_file_path : remote_file_paths) + remote_file_paths_array.push_back(remote_file_path); + columns[i++]->insert(remote_file_paths_array); + columns[i++]->insert(rows); columns[i++]->insert(bytes_compressed_on_disk); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 90e647123851..1bb20079f7ec 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -30,6 +30,7 @@ struct PartLogElement MOVE_PART = 6, MERGE_PARTS_START = 7, MUTATE_PART_START = 8, + EXPORT_PART = 9, }; /// Copy of MergeAlgorithm since values are written to disk. @@ -73,6 +74,7 @@ struct PartLogElement String disk_name; String path_on_disk; Strings deduplication_block_ids; + std::vector remote_file_paths; MergeTreeDataPartType part_type; diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 514f390e93ff..26f36d403fc1 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -54,6 +54,8 @@ bool isSupportedAlterTypeForOnClusterDDLQuery(int type) ASTAlterCommand::ATTACH_PARTITION, /// Usually followed by ATTACH PARTITION ASTAlterCommand::FETCH_PARTITION, + /// Data operation that should be executed locally on each replica + ASTAlterCommand::EXPORT_PART, /// Logical error ASTAlterCommand::NO_TYPE, }; diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index a94ea969d07b..cd3713ab2551 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -65,6 +65,10 @@ ASTPtr ASTAlterCommand::clone() const res->sql_security = res->children.emplace_back(sql_security->clone()).get(); if (rename_to) res->rename_to = res->children.emplace_back(rename_to->clone()).get(); + if (to_table_function) + res->to_table_function = res->children.emplace_back(to_table_function->clone()).get(); + if (partition_by_expr) + res->partition_by_expr = res->children.emplace_back(partition_by_expr->clone()).get(); return res; } @@ -368,6 +372,49 @@ void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & sett ostr << quoteString(move_destination_name); } } + else if (type == ASTAlterCommand::EXPORT_PART) + { + ostr << "EXPORT PART "; + partition->format(ostr, settings, state, frame); + ostr << " TO "; + switch (move_destination_type) + { + case DataDestinationType::TABLE: + ostr << "TABLE "; + if (to_table_function) + { + ostr << "FUNCTION "; + to_table_function->format(ostr, settings, state, frame); + if (partition_by_expr) + { + ostr << " PARTITION BY "; + partition_by_expr->format(ostr, settings, state, frame); + } + } + else + { + if (!to_database.empty()) + ostr << backQuoteIfNeed(to_database) << "."; + + ostr << backQuoteIfNeed(to_table); + } + return; + default: + break; + } + + } + else if (type == ASTAlterCommand::EXPORT_PARTITION) + { + ostr << "EXPORT PARTITION "; + partition->format(ostr, settings, state, frame); + ostr << " TO TABLE "; + if (!to_database.empty()) + { + ostr << backQuoteIfNeed(to_database) << "."; + } + ostr << backQuoteIfNeed(to_table); + } else if (type == ASTAlterCommand::REPLACE_PARTITION) { ostr << (replace ? "REPLACE" : "ATTACH") << " PARTITION " @@ -572,6 +619,8 @@ void ASTAlterCommand::forEachPointerToChild(std::functionmove_destination_name = ast_space_name->as().value.safeGet(); } + else if (s_export_part.ignore(pos, expected)) + { + if (!parser_string_and_substituion.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::EXPORT_PART; + command->part = true; + + if (!s_to_table.ignore(pos, expected)) + { + return false; + } + + if (s_function.ignore(pos, expected)) + { + ParserFunction table_function_parser(/*allow_function_parameters=*/true, /*is_table_function=*/true); + + if (!table_function_parser.parse(pos, export_table_function, expected)) + return false; + + if (s_partition_by.ignore(pos, expected)) + if (!parser_exp_elem.parse(pos, export_table_function_partition_by_expr, expected)) + return false; + + command->to_table_function = export_table_function.get(); + command->partition_by_expr = export_table_function_partition_by_expr.get(); + command->move_destination_type = DataDestinationType::TABLE; + } + else + { + if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) + return false; + command->move_destination_type = DataDestinationType::TABLE; + } + } + else if (s_export_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::EXPORT_PARTITION; + + if (!s_to_table.ignore(pos, expected)) + { + return false; + } + + if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) + return false; + command->move_destination_type = DataDestinationType::TABLE; + } else if (s_move_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command_partition, expected)) @@ -1086,6 +1143,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->rename_to = command->children.emplace_back(std::move(command_rename_to)).get(); if (command_snapshot_desc) command->snapshot_desc = command->children.emplace_back(std::move(command_snapshot_desc)).get(); + if (export_table_function) + command->to_table_function = command->children.emplace_back(std::move(export_table_function)).get(); + if (export_table_function_partition_by_expr) + command->partition_by_expr = command->children.emplace_back(std::move(export_table_function_partition_by_expr)).get(); return true; } diff --git a/src/Parsers/ParserKillQueryQuery.cpp b/src/Parsers/ParserKillQueryQuery.cpp index 97e58566af67..99f2d6fd2d64 100644 --- a/src/Parsers/ParserKillQueryQuery.cpp +++ b/src/Parsers/ParserKillQueryQuery.cpp @@ -17,6 +17,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ParserKeyword p_kill{Keyword::KILL}; ParserKeyword p_query{Keyword::QUERY}; ParserKeyword p_mutation{Keyword::MUTATION}; + ParserKeyword p_export_partition{Keyword::EXPORT_PARTITION}; ParserKeyword p_part_move_to_shard{Keyword::PART_MOVE_TO_SHARD}; ParserKeyword p_transaction{Keyword::TRANSACTION}; ParserKeyword p_on{Keyword::ON}; @@ -33,6 +34,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->type = ASTKillQueryQuery::Type::Query; else if (p_mutation.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::Mutation; + else if (p_export_partition.ignore(pos, expected)) + query->type = ASTKillQueryQuery::Type::ExportPartition; else if (p_part_move_to_shard.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::PartMoveToShard; else if (p_transaction.ignore(pos, expected)) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 003348901c5c..f314b5efcdda 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -494,6 +494,15 @@ NamesAndTypesList ColumnsDescription::getInsertable() const return ret; } +NamesAndTypesList ColumnsDescription::getReadable() const +{ + NamesAndTypesList ret; + for (const auto & col : columns) + if (col.default_desc.kind != ColumnDefaultKind::Ephemeral) + ret.emplace_back(col.name, col.type); + return ret; +} + NamesAndTypesList ColumnsDescription::getMaterialized() const { NamesAndTypesList ret; @@ -860,7 +869,6 @@ std::optional ColumnsDescription::getDefault(const String & colum return {}; } - bool ColumnsDescription::hasCompressionCodec(const String & column_name) const { const auto it = columns.get<1>().find(column_name); diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 2a6d5038766c..67308faeddc6 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -152,6 +152,7 @@ class ColumnsDescription : public IHints<> NamesAndTypesList getOrdinary() const; NamesAndTypesList getMaterialized() const; NamesAndTypesList getInsertable() const; /// ordinary + ephemeral + NamesAndTypesList getReadable() const; /// ordinary + materialized + aliases (no ephemeral) NamesAndTypesList getAliases() const; NamesAndTypesList getEphemeral() const; NamesAndTypesList getAllPhysical() const; /// ordinary + materialized. diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h new file mode 100644 index 000000000000..7c18b8a881c2 --- /dev/null +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -0,0 +1,191 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +struct ExportReplicatedMergeTreePartitionProcessingPartEntry +{ + + enum class Status + { + PENDING, + COMPLETED, + FAILED + }; + + String part_name; + Status status; + size_t retry_count; + String finished_by; + + std::string toJsonString() const + { + Poco::JSON::Object json; + + json.set("part_name", part_name); + json.set("status", String(magic_enum::enum_name(status))); + json.set("retry_count", retry_count); + json.set("finished_by", finished_by); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionProcessingPartEntry fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionProcessingPartEntry entry; + + entry.part_name = json->getValue("part_name"); + entry.status = magic_enum::enum_cast(json->getValue("status")).value(); + entry.retry_count = json->getValue("retry_count"); + if (json->has("finished_by")) + { + entry.finished_by = json->getValue("finished_by"); + } + return entry; + } +}; + +struct ExportReplicatedMergeTreePartitionProcessedPartEntry +{ + String part_name; + std::vector paths_in_destination; + String finished_by; + + std::string toJsonString() const + { + Poco::JSON::Object json; + json.set("part_name", part_name); + json.set("paths_in_destination", paths_in_destination); + json.set("finished_by", finished_by); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionProcessedPartEntry fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionProcessedPartEntry entry; + + entry.part_name = json->getValue("part_name"); + + const auto paths_in_destination_array = json->getArray("paths_in_destination"); + for (size_t i = 0; i < paths_in_destination_array->size(); ++i) + entry.paths_in_destination.emplace_back(paths_in_destination_array->getElement(static_cast(i))); + + entry.finished_by = json->getValue("finished_by"); + + return entry; + } +}; + +struct ExportReplicatedMergeTreePartitionManifest +{ + String transaction_id; + String query_id; + String partition_id; + String destination_database; + String destination_table; + String source_replica; + size_t number_of_parts; + std::vector parts; + time_t create_time; + size_t max_retries; + size_t ttl_seconds; + size_t max_threads; + bool parallel_formatting; + bool parquet_parallel_encoding; + size_t max_bytes_per_file; + size_t max_rows_per_file; + MergeTreePartExportManifest::FileAlreadyExistsPolicy file_already_exists_policy; + + std::string toJsonString() const + { + Poco::JSON::Object json; + json.set("transaction_id", transaction_id); + json.set("query_id", query_id); + json.set("partition_id", partition_id); + json.set("destination_database", destination_database); + json.set("destination_table", destination_table); + json.set("source_replica", source_replica); + json.set("number_of_parts", number_of_parts); + + Poco::JSON::Array::Ptr parts_array = new Poco::JSON::Array(); + for (const auto & part : parts) + parts_array->add(part); + json.set("parts", parts_array); + json.set("parallel_formatting", parallel_formatting); + json.set("max_threads", max_threads); + json.set("parquet_parallel_encoding", parquet_parallel_encoding); + json.set("max_bytes_per_file", max_bytes_per_file); + json.set("max_rows_per_file", max_rows_per_file); + json.set("file_already_exists_policy", String(magic_enum::enum_name(file_already_exists_policy))); + json.set("create_time", create_time); + json.set("max_retries", max_retries); + json.set("ttl_seconds", ttl_seconds); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionManifest fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionManifest manifest; + manifest.transaction_id = json->getValue("transaction_id"); + manifest.query_id = json->getValue("query_id"); + manifest.partition_id = json->getValue("partition_id"); + manifest.destination_database = json->getValue("destination_database"); + manifest.destination_table = json->getValue("destination_table"); + manifest.source_replica = json->getValue("source_replica"); + manifest.number_of_parts = json->getValue("number_of_parts"); + manifest.max_retries = json->getValue("max_retries"); + auto parts_array = json->getArray("parts"); + for (size_t i = 0; i < parts_array->size(); ++i) + manifest.parts.push_back(parts_array->getElement(static_cast(i))); + + manifest.create_time = json->getValue("create_time"); + manifest.ttl_seconds = json->getValue("ttl_seconds"); + manifest.max_threads = json->getValue("max_threads"); + manifest.parallel_formatting = json->getValue("parallel_formatting"); + manifest.parquet_parallel_encoding = json->getValue("parquet_parallel_encoding"); + manifest.max_bytes_per_file = json->getValue("max_bytes_per_file"); + manifest.max_rows_per_file = json->getValue("max_rows_per_file"); + if (json->has("file_already_exists_policy")) + { + const auto file_already_exists_policy = magic_enum::enum_cast(json->getValue("file_already_exists_policy")); + if (file_already_exists_policy) + { + manifest.file_already_exists_policy = file_already_exists_policy.value(); + } + + /// what to do if it's not a valid value? + } + + return manifest; + } +}; + +} diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h new file mode 100644 index 000000000000..76674bfc4a92 --- /dev/null +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -0,0 +1,78 @@ +#pragma once + +#include +#include +#include "Core/QualifiedTableName.h" +#include +#include +#include +#include + +namespace DB +{ +struct ExportReplicatedMergeTreePartitionTaskEntry +{ + using DataPartPtr = std::shared_ptr; + ExportReplicatedMergeTreePartitionManifest manifest; + + enum class Status + { + PENDING, + COMPLETED, + FAILED, + KILLED + }; + + /// Allows us to skip completed / failed entries during scheduling + mutable Status status; + + /// References to the parts that should be exported + /// This is used to prevent the parts from being deleted before finishing the export operation + /// It does not mean this replica will export all the parts + /// There is also a chance this replica does not contain a given part and it is totally ok. + std::vector part_references; + + std::string getCompositeKey() const + { + const auto qualified_table_name = QualifiedTableName {manifest.destination_database, manifest.destination_table}; + return manifest.partition_id + "_" + qualified_table_name.getFullName(); + } + + std::string getTransactionId() const + { + return manifest.transaction_id; + } + + /// Get create_time for sorted iteration + time_t getCreateTime() const + { + return manifest.create_time; + } +}; + +struct ExportPartitionTaskEntryTagByCompositeKey {}; +struct ExportPartitionTaskEntryTagByCreateTime {}; +struct ExportPartitionTaskEntryTagByTransactionId {}; + +// Multi-index container for export partition task entries +// - Index 0 (TagByCompositeKey): hashed_unique on composite key for O(1) lookup +// - Index 1 (TagByCreateTime): ordered_non_unique on create_time for sorted iteration +using ExportPartitionTaskEntriesContainer = boost::multi_index_container< + ExportReplicatedMergeTreePartitionTaskEntry, + boost::multi_index::indexed_by< + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + >, + boost::multi_index::ordered_non_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + >, + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + > + > +>; + +} diff --git a/src/Storages/IPartitionStrategy.cpp b/src/Storages/IPartitionStrategy.cpp index 4143c8b3c9cb..8a4ac6f12df2 100644 --- a/src/Storages/IPartitionStrategy.cpp +++ b/src/Storages/IPartitionStrategy.cpp @@ -274,17 +274,13 @@ ColumnPtr WildcardPartitionStrategy::computePartitionKey(const Chunk & chunk) return block_with_partition_by_expr.getByName(actions_with_column.column_name).column; } -std::string WildcardPartitionStrategy::getPathForRead( - const std::string & prefix) +ColumnPtr WildcardPartitionStrategy::computePartitionKey(Block & block) { - return prefix; -} - -std::string WildcardPartitionStrategy::getPathForWrite( - const std::string & prefix, - const std::string & partition_key) -{ - return PartitionedSink::replaceWildcards(prefix, partition_key); + ASTs arguments(1, partition_key_description.definition_ast); + ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); + auto actions_with_column = getPartitionExpressionActions(partition_by_string); + actions_with_column.actions->execute(block); + return block.getByName(actions_with_column.column_name).column; } HiveStylePartitionStrategy::HiveStylePartitionStrategy( @@ -306,41 +302,6 @@ HiveStylePartitionStrategy::HiveStylePartitionStrategy( block_without_partition_columns = buildBlockWithoutPartitionColumns(sample_block, partition_columns_name_set); } -std::string HiveStylePartitionStrategy::getPathForRead(const std::string & prefix) -{ - return prefix + "**." + Poco::toLower(file_format); -} - -std::string HiveStylePartitionStrategy::getPathForWrite( - const std::string & prefix, - const std::string & partition_key) -{ - std::string path; - - if (!prefix.empty()) - { - path += prefix; - if (path.back() != '/') - { - path += '/'; - } - } - - /// Not adding '/' because buildExpressionHive() always adds a trailing '/' - path += partition_key; - - /* - * File extension is toLower(format) - * This isn't ideal, but I guess multiple formats can be specified and introduced. - * So I think it is simpler to keep it this way. - * - * Or perhaps implement something like `IInputFormat::getFileExtension()` - */ - path += std::to_string(generateSnowflakeID()) + "." + Poco::toLower(file_format); - - return path; -} - ColumnPtr HiveStylePartitionStrategy::computePartitionKey(const Chunk & chunk) { auto hive_ast = buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); @@ -353,6 +314,14 @@ ColumnPtr HiveStylePartitionStrategy::computePartitionKey(const Chunk & chunk) return block_with_partition_by_expr.getByName(actions_with_column.column_name).column; } +ColumnPtr HiveStylePartitionStrategy::computePartitionKey(Block & block) +{ + auto hive_ast = buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); + auto actions_with_column = getPartitionExpressionActions(hive_ast); + actions_with_column.actions->execute(block); + return block.getByName(actions_with_column.column_name).column; +} + ColumnRawPtrs HiveStylePartitionStrategy::getFormatChunkColumns(const Chunk & chunk) { ColumnRawPtrs result; diff --git a/src/Storages/IPartitionStrategy.h b/src/Storages/IPartitionStrategy.h index a0f5f4cdf9d3..91397de2362d 100644 --- a/src/Storages/IPartitionStrategy.h +++ b/src/Storages/IPartitionStrategy.h @@ -29,8 +29,7 @@ struct IPartitionStrategy virtual ColumnPtr computePartitionKey(const Chunk & chunk) = 0; - virtual std::string getPathForRead(const std::string & prefix) = 0; - virtual std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) = 0; + virtual ColumnPtr computePartitionKey(Block & block) = 0; virtual ColumnRawPtrs getFormatChunkColumns(const Chunk & chunk) { @@ -93,8 +92,8 @@ struct WildcardPartitionStrategy : IPartitionStrategy WildcardPartitionStrategy(KeyDescription partition_key_description_, const Block & sample_block_, ContextPtr context_); ColumnPtr computePartitionKey(const Chunk & chunk) override; - std::string getPathForRead(const std::string & prefix) override; - std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) override; + + ColumnPtr computePartitionKey(Block & block) override; }; /* @@ -112,8 +111,8 @@ struct HiveStylePartitionStrategy : IPartitionStrategy bool partition_columns_in_data_file_); ColumnPtr computePartitionKey(const Chunk & chunk) override; - std::string getPathForRead(const std::string & prefix) override; - std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) override; + + ColumnPtr computePartitionKey(Block & block) override; ColumnRawPtrs getFormatChunkColumns(const Chunk & chunk) override; Block getFormatHeader() override; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 8f5d8f5b3e95..46d2fb9154b2 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -322,6 +322,11 @@ CancellationCode IStorage::killPartMoveToShard(const UUID & /*task_uuid*/) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part moves between shards are not supported by storage {}", getName()); } +CancellationCode IStorage::killExportPartition(const String & /*transaction_id*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Export partition is not supported by storage {}", getName()); +} + StorageID IStorage::getStorageID() const { std::lock_guard lock(id_mutex); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 96f5adbc2de5..583235bcf20c 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -473,6 +473,39 @@ class IStorage : public std::enable_shared_from_this, public TypePromo ContextPtr /*context*/, bool /*async_insert*/); + virtual bool supportsImport() const + { + return false; + } + + /* +It is currently only implemented in StorageObjectStorage. + It is meant to be used to import merge tree data parts into object storage. It is similar to the write API, + but it won't re-partition the data and should allow the filename to be set by the caller. + */ + virtual SinkToStoragePtr import( + const std::string & /* file_name */, + Block & /* block_with_partition_values */, + const std::function & /* new_file_path_callback */, + bool /* overwrite_if_exists */, + std::size_t /* max_bytes_per_file */, + std::size_t /* max_rows_per_file */, + const std::optional & /* format_settings */, + ContextPtr /* context */) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Import is not implemented for storage {}", getName()); + } + + virtual void commitExportPartitionTransaction( + const String & /* transaction_id */, + const String & /* partition_id */, + const Strings & /* exported_paths */, + ContextPtr /* local_context */) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "commitExportPartitionTransaction is not implemented for storage type {}", getName()); + } + + /** Writes the data to a table in distributed manner. * It is supposed that implementation looks into SELECT part of the query and executes distributed * INSERT SELECT if it is possible with current storage as a receiver and query SELECT part as a producer. @@ -579,6 +612,9 @@ class IStorage : public std::enable_shared_from_this, public TypePromo virtual void setMutationCSN(const String & /*mutation_id*/, UInt64 /*csn*/); + /// Cancel a replicated partition export by transaction id. + virtual CancellationCode killExportPartition(const String & /*transaction_id*/); + /// Cancel a part move to shard. virtual CancellationCode killPartMoveToShard(const UUID & /*task_uuid*/); diff --git a/src/Storages/MergeTree/ExportList.cpp b/src/Storages/MergeTree/ExportList.cpp new file mode 100644 index 000000000000..018c1f091ef9 --- /dev/null +++ b/src/Storages/MergeTree/ExportList.cpp @@ -0,0 +1,74 @@ +#include + +namespace DB +{ + +ExportsListElement::ExportsListElement( + const StorageID & source_table_id_, + const StorageID & destination_table_id_, + UInt64 part_size_, + const String & part_name_, + const std::vector & destination_file_paths_, + UInt64 total_rows_to_read_, + UInt64 total_size_bytes_compressed_, + UInt64 total_size_bytes_uncompressed_, + time_t create_time_, + const String & query_id_, + const ContextPtr & context) +: source_table_id(source_table_id_) +, destination_table_id(destination_table_id_) +, part_size(part_size_) +, part_name(part_name_) +, destination_file_paths(destination_file_paths_) +, total_rows_to_read(total_rows_to_read_) +, total_size_bytes_compressed(total_size_bytes_compressed_) +, total_size_bytes_uncompressed(total_size_bytes_uncompressed_) +, create_time(create_time_) +, query_id(query_id_) +{ + thread_group = ThreadGroup::createForMergeMutate(context); +} + +ExportsListElement::~ExportsListElement() +{ + background_memory_tracker.adjustOnBackgroundTaskEnd(&thread_group->memory_tracker); +} + +ExportInfo ExportsListElement::getInfo() const +{ + ExportInfo res; + res.source_database = source_table_id.database_name; + res.source_table = source_table_id.table_name; + res.destination_database = destination_table_id.database_name; + res.destination_table = destination_table_id.table_name; + res.part_name = part_name; + + { + std::shared_lock lock(destination_file_paths_mutex); + res.destination_file_paths = destination_file_paths; + } + + res.rows_read = rows_read.load(std::memory_order_relaxed); + res.total_rows_to_read = total_rows_to_read; + res.total_size_bytes_compressed = total_size_bytes_compressed; + res.total_size_bytes_uncompressed = total_size_bytes_uncompressed; + res.bytes_read_uncompressed = bytes_read_uncompressed.load(std::memory_order_relaxed); + res.memory_usage = getMemoryUsage(); + res.peak_memory_usage = getPeakMemoryUsage(); + res.create_time = create_time; + res.elapsed = watch.elapsedSeconds(); + res.query_id = query_id; + return res; +} + +UInt64 ExportsListElement::getMemoryUsage() const +{ + return thread_group->memory_tracker.get(); +} + +UInt64 ExportsListElement::getPeakMemoryUsage() const +{ + return thread_group->memory_tracker.getPeak(); +} + +} diff --git a/src/Storages/MergeTree/ExportList.h b/src/Storages/MergeTree/ExportList.h new file mode 100644 index 000000000000..4a02826dfe44 --- /dev/null +++ b/src/Storages/MergeTree/ExportList.h @@ -0,0 +1,96 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric Export; +} + +namespace DB +{ + +struct ExportInfo +{ + String source_database; + String source_table; + String destination_database; + String destination_table; + String part_name; + std::vector destination_file_paths; + UInt64 rows_read; + UInt64 total_rows_to_read; + UInt64 total_size_bytes_compressed; + UInt64 total_size_bytes_uncompressed; + UInt64 bytes_read_uncompressed; + UInt64 memory_usage; + UInt64 peak_memory_usage; + time_t create_time = 0; + Float64 elapsed; + String query_id; +}; + +struct ExportsListElement : private boost::noncopyable +{ + const StorageID source_table_id; + const StorageID destination_table_id; + const UInt64 part_size; + const String part_name; + + /// see destination_file_paths_mutex + std::vector destination_file_paths; + std::atomic rows_read {0}; + UInt64 total_rows_to_read {0}; + UInt64 total_size_bytes_compressed {0}; + UInt64 total_size_bytes_uncompressed {0}; + std::atomic bytes_read_uncompressed {0}; + time_t create_time {0}; + String query_id; + + Stopwatch watch; + ThreadGroupPtr thread_group; + mutable std::shared_mutex destination_file_paths_mutex; + + ExportsListElement( + const StorageID & source_table_id_, + const StorageID & destination_table_id_, + UInt64 part_size_, + const String & part_name_, + const std::vector & destination_file_paths_, + UInt64 total_rows_to_read_, + UInt64 total_size_bytes_compressed_, + UInt64 total_size_bytes_uncompressed_, + time_t create_time_, + const String & query_id_, + const ContextPtr & context); + + ~ExportsListElement(); + + ExportInfo getInfo() const; + + UInt64 getMemoryUsage() const; + UInt64 getPeakMemoryUsage() const; +}; + + +class ExportsList final : public BackgroundProcessList +{ +private: + using Parent = BackgroundProcessList; + +public: + ExportsList() + : Parent(CurrentMetrics::Export) + {} +}; + +using ExportsListEntry = BackgroundProcessListEntry; + +} diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp new file mode 100644 index 000000000000..07a2e49713b8 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -0,0 +1,349 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Common/setThreadName.h" +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event PartsExportDuplicated; + extern const Event PartsExportFailures; + extern const Event PartsExports; + extern const Event PartsExportTotalMilliseconds; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_TABLE; + extern const int FILE_ALREADY_EXISTS; + extern const int LOGICAL_ERROR; + extern const int QUERY_WAS_CANCELLED; +} + +namespace Setting +{ + extern const SettingsUInt64 min_bytes_to_use_direct_io; + extern const SettingsUInt64 export_merge_tree_part_max_bytes_per_file; + extern const SettingsUInt64 export_merge_tree_part_max_rows_per_file; + extern const SettingsBool allow_experimental_analyzer; +} + +namespace +{ + void materializeSpecialColumns( + const SharedHeader & header, + const StorageMetadataPtr & storage_metadata, + const ContextPtr & local_context, + QueryPlan & plan_for_part + ) + { + const auto readable_columns = storage_metadata->getColumns().getReadable(); + + // Enable all experimental settings for default expressions + // (same pattern as in IMergeTreeReader::evaluateMissingDefaults) + auto context_for_defaults = Context::createCopy(local_context); + enableAllExperimentalSettings(context_for_defaults); + + /// Copy the behavior of `IMergeTreeReader`, see https://github.com/ClickHouse/ClickHouse/blob/c45224e3f0a6dd9a9217e5d75723f378ffe0a86a/src/Storages/MergeTree/IMergeTreeReader.cpp#L215 + context_for_defaults->setSetting("enable_analyzer", local_context->getSettingsRef()[Setting::allow_experimental_analyzer].value); + + auto defaults_dag = evaluateMissingDefaults( + *header, + readable_columns, + storage_metadata->getColumns(), + context_for_defaults); + + if (defaults_dag) + { + ActionsDAG base_dag(header->getColumnsWithTypeAndName()); + + /// `evaluateMissingDefaults` has a new analyzer path since https://github.com/ClickHouse/ClickHouse/pull/87585 + /// which returns a DAG that does not contain all columns. We need to merge it with the base DAG to get all columns. + auto merged = ActionsDAG::merge(std::move(base_dag), std::move(*defaults_dag)); + + /// Ensure columns are in the correct order matching readable_columns + merged.removeUnusedActions(readable_columns.getNames(), false); + merged.addMaterializingOutputActions(/*materialize_sparse=*/ false); + + auto expression_step = std::make_unique( + header, + std::move(merged)); + expression_step->setStepDescription("Compute alias and default expressions for export"); + plan_for_part.addStep(std::move(expression_step)); + } + } +} + +ExportPartTask::ExportPartTask(MergeTreeData & storage_, const MergeTreePartExportManifest & manifest_) + : storage(storage_), + manifest(manifest_) +{ +} + +bool ExportPartTask::executeStep() +{ + auto local_context = Context::createCopy(storage.getContext()); + local_context->makeQueryContextForExportPart(); + local_context->setCurrentQueryId(manifest.query_id); + local_context->setBackgroundOperationTypeForContext(ClientInfo::BackgroundOperationType::EXPORT_PART); + local_context->setSettings(manifest.settings); + + const auto & metadata_snapshot = manifest.metadata_snapshot; + + /// Read only physical columns from the part + const auto columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); + + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Export; + + Block block_with_partition_values; + if (metadata_snapshot->hasPartitionKey()) + { + /// todo arthur do I need to init minmax_idx? + block_with_partition_values = manifest.data_part->minmax_idx->getBlock(storage); + } + + const auto & destination_storage = manifest.destination_storage_ptr; + const auto destination_storage_id = destination_storage->getStorageID(); + + auto exports_list_entry = storage.getContext()->getExportsList().insert( + getStorageID(), + destination_storage_id, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->name, + std::vector{}, + manifest.data_part->rows_count, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->getBytesUncompressedOnDisk(), + manifest.create_time, + manifest.query_id, + local_context); + + SinkToStoragePtr sink; + + const auto new_file_path_callback = [&exports_list_entry](const std::string & file_path) + { + std::unique_lock lock((*exports_list_entry)->destination_file_paths_mutex); + (*exports_list_entry)->destination_file_paths.push_back(file_path); + }; + + try + { + sink = destination_storage->import( + manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), + block_with_partition_values, + new_file_path_callback, + manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::overwrite, + manifest.settings[Setting::export_merge_tree_part_max_bytes_per_file], + manifest.settings[Setting::export_merge_tree_part_max_rows_per_file], + getFormatSettings(local_context), + local_context); + + bool apply_deleted_mask = true; + bool read_with_direct_io = local_context->getSettingsRef()[Setting::min_bytes_to_use_direct_io] > manifest.data_part->getBytesOnDisk(); + bool prefetch = false; + + MergeTreeData::IMutationsSnapshot::Params mutations_snapshot_params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = manifest.data_part->getMetadataVersion() + }; + + auto mutations_snapshot = storage.getMutationsSnapshot(mutations_snapshot_params); + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + manifest.data_part, + mutations_snapshot, + local_context); + + QueryPlan plan_for_part; + + createReadFromPartStep( + read_type, + plan_for_part, + storage, + storage.getStorageSnapshot(metadata_snapshot, local_context), + RangesInDataPart(manifest.data_part), + alter_conversions, + nullptr, + columns_to_read, + nullptr, + apply_deleted_mask, + std::nullopt, + read_with_direct_io, + prefetch, + local_context, + getLogger("ExportPartition")); + + ThreadGroupSwitcher switcher((*exports_list_entry)->thread_group, ThreadName::EXPORT_PART); + + /// We need to support exporting materialized and alias columns to object storage. For some reason, object storage engines don't support them. + /// This is a hack that materializes the columns before the export so they can be exported to tables that have matching columns + materializeSpecialColumns(plan_for_part.getCurrentHeader(), metadata_snapshot, local_context, plan_for_part); + + QueryPlanOptimizationSettings optimization_settings(local_context); + auto pipeline_settings = BuildQueryPipelineSettings(local_context); + auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); + + builder->setProgressCallback([&exports_list_entry](const Progress & progress) + { + (*exports_list_entry)->bytes_read_uncompressed += progress.read_bytes; + (*exports_list_entry)->rows_read += progress.read_rows; + }); + + pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + + pipeline.complete(sink); + + CompletedPipelineExecutor exec(pipeline); + + auto is_cancelled_callback = [this]() + { + return isCancelled(); + }; + + exec.setCancelCallback(is_cancelled_callback, 100); + + exec.execute(); + + if (isCancelled()) + { + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Export part was cancelled"); + } + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + {}, + (*exports_list_entry)->watch.elapsed(), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + storage.export_manifests.erase(manifest); + + ProfileEvents::increment(ProfileEvents::PartsExports); + ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, (*exports_list_entry)->watch.elapsedMilliseconds()); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess((*exports_list_entry)->destination_file_paths)); + } + catch (const Exception & e) + { + /// If an exception is thrown before the pipeline is started, the sink will not be canceled and might leave buffers open. + /// Cancel it manually to ensure the buffers are closed. + if (sink) + { + sink->cancel(); + } + + if (e.code() == ErrorCodes::FILE_ALREADY_EXISTS) + { + ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); + + /// File already exists and the policy is NO_OP, treat it as success. + if (manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::skip) + { + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + {}, + (*exports_list_entry)->watch.elapsed(), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + ProfileEvents::increment(ProfileEvents::PartsExports); + ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, (*exports_list_entry)->watch.elapsedMilliseconds()); + + if (manifest.completion_callback) + { + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess((*exports_list_entry)->destination_file_paths)); + } + + return false; + } + } + + ProfileEvents::increment(ProfileEvents::PartsExportFailures); + + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + ExecutionStatus::fromCurrentException("", true), + (*exports_list_entry)->watch.elapsed(), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e)); + return false; + } + + return false; +} + +void ExportPartTask::cancel() noexcept +{ + cancel_requested.store(true); + pipeline.cancel(); +} + +bool ExportPartTask::isCancelled() const +{ + return cancel_requested.load() || storage.parts_mover.moves_blocker.isCancelled(); +} + +void ExportPartTask::onCompleted() +{ +} + +StorageID ExportPartTask::getStorageID() const +{ + return storage.getStorageID(); +} + +Priority ExportPartTask::getPriority() const +{ + return Priority{}; +} + +String ExportPartTask::getQueryId() const +{ + return manifest.query_id; +} + +} diff --git a/src/Storages/MergeTree/ExportPartTask.h b/src/Storages/MergeTree/ExportPartTask.h new file mode 100644 index 000000000000..9c8fa6cc01dd --- /dev/null +++ b/src/Storages/MergeTree/ExportPartTask.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ExportPartTask : public IExecutableTask +{ +public: + explicit ExportPartTask( + MergeTreeData & storage_, + const MergeTreePartExportManifest & manifest_); + bool executeStep() override; + void onCompleted() override; + StorageID getStorageID() const override; + Priority getPriority() const override; + String getQueryId() const override; + + void cancel() noexcept override; + +private: + MergeTreeData & storage; + MergeTreePartExportManifest manifest; + QueryPipeline pipeline; + std::atomic cancel_requested = false; + + bool isCancelled() const; +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp new file mode 100644 index 000000000000..79b92663b7bf --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -0,0 +1,310 @@ +#include +#include +#include +#include "Storages/MergeTree/ExportPartitionUtils.h" +#include "Common/logger_useful.h" +#include +#include +#include + +namespace DB +{ +namespace +{ + /* + Remove expired entries and fix non-committed exports that have already exported all parts. + + Return values: + - true: the cleanup was successful, the entry is removed from the entries_by_key container and the function returns true. Proceed to the next entry. + - false: the cleanup was not successful, the entry is not removed from the entries_by_key container and the function returns false. + */ + bool tryCleanup( + const zkutil::ZooKeeperPtr & zk, + const std::string & entry_path, + const LoggerPtr & log, + const ContextPtr & context, + const std::string & key, + const ExportReplicatedMergeTreePartitionManifest & metadata, + const time_t now, + const bool is_pending, + auto & entries_by_key + ) + { + bool has_expired = metadata.create_time < now - static_cast(metadata.ttl_seconds); + + if (has_expired && !is_pending) + { + zk->tryRemoveRecursive(fs::path(entry_path)); + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.erase(it); + LOG_INFO(log, "ExportPartition Manifest Updating Task: Removed {}: expired", key); + + return true; + } + else if (is_pending) + { + std::vector parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(entry_path) / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to get parts in processing or pending, skipping"); + return false; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Cleanup found PENDING for {} with all parts exported, try to fix it by committing the export", entry_path); + + const auto destination_storage_id = StorageID(QualifiedTableName {metadata.destination_database, metadata.destination_table}); + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, context); + if (!destination_storage) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + return false; + } + + /// it sounds like a replica exported the last part, but was not able to commit the export. Try to fix it + ExportPartitionUtils::commit(metadata, destination_storage, zk, log, entry_path, context); + + return true; + } + } + + return false; + } +} + +ExportPartitionManifestUpdatingTask::ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage_) + : storage(storage_) +{ +} + +void ExportPartitionManifestUpdatingTask::poll() +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Polling for new entries for table {}. Current number of entries: {}", storage.getStorageID().getNameForLogs(), storage.export_merge_tree_partition_task_entries_by_key.size()); + + auto zk = storage.getZooKeeper(); + + const std::string exports_path = fs::path(storage.zookeeper_path) / "exports"; + const std::string cleanup_lock_path = fs::path(storage.zookeeper_path) / "exports_cleanup_lock"; + + auto cleanup_lock = zkutil::EphemeralNodeHolder::tryCreate(cleanup_lock_path, *zk, storage.replica_name); + if (cleanup_lock) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Cleanup lock acquired, will remove stale entries"); + } + + Coordination::Stat stat; + const auto children = zk->getChildrenWatch(exports_path, &stat, storage.export_merge_tree_partition_watch_callback); + const std::unordered_set zk_children(children.begin(), children.end()); + + const auto now = time(nullptr); + + auto & entries_by_key = storage.export_merge_tree_partition_task_entries_by_key; + + /// Load new entries + /// If we have the cleanup lock, also remove stale entries from zk and local + /// Upload dangling commit files if any + for (const auto & key : zk_children) + { + const std::string entry_path = fs::path(exports_path) / key; + + std::string metadata_json; + if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing metadata.json", key); + continue; + } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + const auto local_entry = entries_by_key.find(key); + + /// If the zk entry has been replaced with export_merge_tree_partition_force_export, checking only for the export key is not enough + /// we need to make sure it is the same transaction id. If it is not, it needs to be replaced. + bool has_local_entry_and_is_up_to_date = local_entry != entries_by_key.end() + && local_entry->manifest.transaction_id == metadata.transaction_id; + + /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. + if (!cleanup_lock && has_local_entry_and_is_up_to_date) + continue; + + std::weak_ptr weak_manifest_updater = storage.export_merge_tree_partition_manifest_updater; + + auto status_watch_callback = std::make_shared([weak_manifest_updater, key](const Coordination::WatchResponse &) + { + /// If the table is dropped but the watch is not removed, we need to prevent use after free + /// below code assumes that if manifest updater is still alive, the status handling task is also alive + if (auto manifest_updater = weak_manifest_updater.lock()) + { + manifest_updater->addStatusChange(key); + manifest_updater->storage.export_merge_tree_partition_status_handling_task->schedule(); + } + }); + + std::string status; + if (!zk->tryGetWatch(fs::path(entry_path) / "status", status, nullptr, status_watch_callback)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing status", key); + continue; + } + + bool is_pending = status == "PENDING"; + + /// if we have the cleanup lock, try to cleanup + /// if we successfully cleaned it up, early exit + if (cleanup_lock) + { + bool cleanup_successful = tryCleanup( + zk, + entry_path, + storage.log.load(), + storage.getContext(), + key, + metadata, + now, + is_pending, entries_by_key); + + if (cleanup_successful) + continue; + } + + if (!is_pending) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: status is not PENDING", key); + continue; + } + + if (has_local_entry_and_is_up_to_date) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: already exists", key); + continue; + } + + addTask(metadata, key, entries_by_key); + } + + /// Remove entries that were deleted by someone else + removeStaleEntries(zk_children, entries_by_key); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: finished polling for new entries. Number of entries: {}", entries_by_key.size()); + + storage.export_merge_tree_partition_select_task->schedule(); +} + +void ExportPartitionManifestUpdatingTask::addTask( + const ExportReplicatedMergeTreePartitionManifest & metadata, + const std::string & key, + auto & entries_by_key +) +{ + std::vector part_references; + + for (const auto & part_name : metadata.parts) + { + if (const auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + { + part_references.push_back(part); + } + } + + /// Insert or update entry. The multi_index container automatically maintains both indexes. + auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, std::move(part_references)}; + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.replace(it, entry); + else + entries_by_key.insert(entry); +} + +void ExportPartitionManifestUpdatingTask::removeStaleEntries( + const std::unordered_set & zk_children, + auto & entries_by_key +) +{ + for (auto it = entries_by_key.begin(); it != entries_by_key.end();) + { + const auto & key = it->getCompositeKey(); + if (zk_children.contains(key)) + { + ++it; + continue; + } + + const auto & transaction_id = it->manifest.transaction_id; + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Export task {} was deleted, calling killExportPartition for transaction {}", key, transaction_id); + + try + { + storage.killExportPart(transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + + it = entries_by_key.erase(it); + } +} + +void ExportPartitionManifestUpdatingTask::addStatusChange(const std::string & key) +{ + std::lock_guard lock(status_changes_mutex); + status_changes.emplace(key); +} + +void ExportPartitionManifestUpdatingTask::handleStatusChanges() +{ + std::lock_guard lock(status_changes_mutex); + std::lock_guard task_entries_lock(storage.export_merge_tree_partition_mutex); + auto zk = storage.getZooKeeper(); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status changes. Number of status changes: {}", status_changes.size()); + + while (!status_changes.empty()) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status change for task {}", status_changes.front()); + const auto key = status_changes.front(); + status_changes.pop(); + + auto it = storage.export_merge_tree_partition_task_entries_by_key.find(key); + if (it == storage.export_merge_tree_partition_task_entries_by_key.end()) + continue; + + /// get new status from zk + std::string new_status_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", new_status_string)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to get new status for task {}, skipping", key); + continue; + } + + const auto new_status = magic_enum::enum_cast(new_status_string); + if (!new_status) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Invalid status {} for task {}, skipping", new_status_string, key); + continue; + } + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: status changed for task {}. New status: {}", key, magic_enum::enum_name(*new_status).data()); + + /// If status changed to KILLED, cancel local export operations + if (*new_status == ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED) + { + try + { + storage.killExportPart(it->manifest.transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + } + + it->status = *new_status; + } +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h new file mode 100644 index 000000000000..ea52f679d654 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include +namespace DB +{ + +class StorageReplicatedMergeTree; +struct ExportReplicatedMergeTreePartitionManifest; + +class ExportPartitionManifestUpdatingTask +{ +public: + ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage); + + void poll(); + + void handleStatusChanges(); + + void addStatusChange(const std::string & key); + +private: + StorageReplicatedMergeTree & storage; + + void addTask( + const ExportReplicatedMergeTreePartitionManifest & metadata, + const std::string & key, + auto & entries_by_key + ); + + void removeStaleEntries( + const std::unordered_set & zk_children, + auto & entries_by_key + ); + + std::mutex status_changes_mutex; + std::queue status_changes; +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp new file mode 100644 index 000000000000..925d7eafe412 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -0,0 +1,402 @@ +#include +#include +#include +#include +#include +#include +#include "Storages/MergeTree/ExportPartitionUtils.h" +#include "Storages/MergeTree/MergeTreePartExportManifest.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int QUERY_WAS_CANCELLED; + extern const int LOGICAL_ERROR; +} + +namespace +{ + ContextPtr getContextCopyWithTaskSettings(const ContextPtr & context, const ExportReplicatedMergeTreePartitionManifest & manifest) + { + auto context_copy = Context::createCopy(context); + context_copy->makeQueryContextForExportPart(); + context_copy->setCurrentQueryId(manifest.query_id); + context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); + context_copy->setSetting("output_format_parquet_parallel_encoding", manifest.parquet_parallel_encoding); + context_copy->setSetting("max_threads", manifest.max_threads); + context_copy->setSetting("export_merge_tree_part_file_already_exists_policy", String(magic_enum::enum_name(manifest.file_already_exists_policy))); + context_copy->setSetting("export_merge_tree_part_max_bytes_per_file", manifest.max_bytes_per_file); + context_copy->setSetting("export_merge_tree_part_max_rows_per_file", manifest.max_rows_per_file); + + /// always skip pending mutations and patch parts because we already validated the parts during query processing + context_copy->setSetting("export_merge_tree_part_throw_on_pending_mutations", false); + context_copy->setSetting("export_merge_tree_part_throw_on_pending_patch_parts", false); + + return context_copy; + } +} + +ExportPartitionTaskScheduler::ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage_) + : storage(storage_) +{ +} + +void ExportPartitionTaskScheduler::run() +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + auto zk = storage.getZooKeeper(); + + // Iterate sorted by create_time + for (auto & entry : storage.export_merge_tree_partition_task_entries_by_create_time) + { + const auto & manifest = entry.manifest; + const auto key = entry.getCompositeKey(); + const auto database = storage.getContext()->resolveDatabase(manifest.destination_database); + const auto & table = manifest.destination_table; + + /// No need to query zk for status if the local one is not PENDING + if (entry.status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Local status is {}", magic_enum::enum_name(entry.status).data()); + continue; + } + + const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); + + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, storage.getContext()); + + if (!destination_storage) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + continue; + } + + std::string status_in_zk_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", status_in_zk_string)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status, skipping"); + continue; + } + + const auto status_in_zk = magic_enum::enum_cast(status_in_zk_string); + + if (!status_in_zk) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status from zk, skipping"); + continue; + } + + if (status_in_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + entry.status = status_in_zk.value(); + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Status from zk is {}", entry.status); + continue; + } + + std::vector parts_in_processing_or_pending; + + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, skipping"); + continue; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: No parts in processing or pending, skipping"); + continue; + } + + std::vector locked_parts; + + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "locks", locked_parts)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get locked parts, skipping"); + continue; + } + + std::unordered_set locked_parts_set(locked_parts.begin(), locked_parts.end()); + + for (const auto & zk_part_name : parts_in_processing_or_pending) + { + if (locked_parts_set.contains(zk_part_name)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked, skipping", zk_part_name); + continue; + } + + const auto part = storage.getPartIfExists(zk_part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!part) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} not found locally, skipping", zk_part_name); + continue; + } + + if (Coordination::Error::ZOK != zk->tryCreate(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name, storage.replica_name, zkutil::CreateMode::Ephemeral)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to lock part {}, skipping", zk_part_name); + continue; + } + + try + { + storage.exportPartToTable( + part->name, + destination_storage_id, + manifest.transaction_id, + getContextCopyWithTaskSettings(storage.getContext(), manifest), + /*allow_outdated_parts*/ true, + [this, key, zk_part_name, manifest, destination_storage] + (MergeTreePartExportManifest::CompletionCallbackResult result) + { + handlePartExportCompletion(key, zk_part_name, manifest, destination_storage, result); + }); + } + catch (const Exception &) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + zk->tryRemove(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name); + /// we should not increment retry_count because the node might just be full + } + } + } + + /// maybe we failed to schedule or failed to export, need to retry eventually + storage.export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); +} + +void ExportPartitionTaskScheduler::handlePartExportCompletion( + const std::string & export_key, + const std::string & part_name, + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const MergeTreePartExportManifest::CompletionCallbackResult & result) +{ + const auto export_path = fs::path(storage.zookeeper_path) / "exports" / export_key; + const auto processing_parts_path = export_path / "processing"; + const auto processed_part_path = export_path / "processed" / part_name; + const auto zk = storage.getZooKeeper(); + + if (result.success) + { + handlePartExportSuccess(manifest, destination_storage, processing_parts_path, processed_part_path, part_name, export_path, zk, result.relative_paths_in_destination_storage); + } + else + { + handlePartExportFailure(processing_parts_path, part_name, export_path, zk, result.exception, manifest.max_retries); + } +} + +void ExportPartitionTaskScheduler::handlePartExportSuccess( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::vector & relative_paths_in_destination_storage +) +{ + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} exported successfully, paths size: {}", part_name, relative_paths_in_destination_storage.size()); + + for (const auto & relative_path_in_destination_storage : relative_paths_in_destination_storage) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: {}", relative_path_in_destination_storage); + } + + if (!tryToMovePartToProcessed(export_path, processing_parts_path, processed_part_path, part_name, relative_paths_in_destination_storage, zk)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to move part to processed, will not commit export partition"); + return; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: Marked part export {} as completed", part_name); + + if (!areAllPartsProcessed(export_path, zk)) + { + return; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: All parts are processed, will try to commit export partition"); + + ExportPartitionUtils::commit(manifest, destination_storage, zk, storage.log.load(), export_path, storage.getContext()); +} + +void ExportPartitionTaskScheduler::handlePartExportFailure( + const std::filesystem::path & processing_parts_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::optional & exception, + size_t max_retries +) +{ + if (!exception) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "ExportPartition scheduler task: No exception provided for error handling. Sounds like a bug"); + } + + /// Early exit if the query was cancelled - no need to increment error counts + if (exception->code() == ErrorCodes::QUERY_WAS_CANCELLED) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export was cancelled, skipping error handling", part_name); + return; + } + + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not increment error counts", part_name); + return; + } + + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not increment error counts", part_name); + return; + } + + Coordination::Requests ops; + + const auto processing_part_path = processing_parts_path / part_name; + + std::string processing_part_string; + + if (!zk->tryGet(processing_part_path, processing_part_string)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get processing part, will not increment error counts"); + return; + } + + /// todo arthur could this have been cached? + auto processing_part_entry = ExportReplicatedMergeTreePartitionProcessingPartEntry::fromJsonString(processing_part_string); + + processing_part_entry.retry_count++; + + if (processing_part_entry.retry_count) + { + ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(processing_part_path, processing_part_entry.toJsonString(), -1)); + + if (processing_part_entry.retry_count >= max_retries) + { + /// just set status in processing_part_path and finished_by + processing_part_entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::FAILED; + processing_part_entry.finished_by = storage.replica_name; + + ops.emplace_back(zkutil::makeSetRequest(export_path / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::FAILED)).data(), -1)); + LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); + } + + std::size_t num_exceptions = 0; + + const auto exceptions_per_replica_path = export_path / "exceptions_per_replica" / storage.replica_name; + const auto count_path = exceptions_per_replica_path / "count"; + const auto last_exception_path = exceptions_per_replica_path / "last_exception"; + + if (zk->exists(exceptions_per_replica_path)) + { + std::string num_exceptions_string; + zk->tryGet(count_path, num_exceptions_string); + num_exceptions = std::stoull(num_exceptions_string.c_str()); + + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception->message(), -1)); + } + else + { + ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(count_path, "0", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception->message(), zkutil::CreateMode::Persistent)); + } + + num_exceptions++; + ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(ops, responses)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: All failure mechanism failed, will not try to update it"); + return; + } + } +} + +bool ExportPartitionTaskScheduler::tryToMovePartToProcessed( + const std::filesystem::path & export_path, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::vector & relative_paths_in_destination_storage, + const zkutil::ZooKeeperPtr & zk +) +{ + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not commit or set it as completed", part_name); + return false; + } + + /// Is this a good idea? what if the file we just pushed to s3 ends up triggering an exception in the replica that actually locks the part and it does not commit? + /// I guess we should not throw if file already exists for export partition, hard coded. + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not commit or set it as completed", part_name); + return false; + } + + Coordination::Requests requests; + + ExportReplicatedMergeTreePartitionProcessedPartEntry processed_part_entry; + processed_part_entry.part_name = part_name; + processed_part_entry.paths_in_destination = relative_paths_in_destination_storage; + processed_part_entry.finished_by = storage.replica_name; + + requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); + requests.emplace_back(zkutil::makeCreateRequest(processed_part_path, processed_part_entry.toJsonString(), zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(requests, responses)) + { + /// todo arthur remember what to do here + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to update export path, skipping"); + return false; + } + + return true; +} + +bool ExportPartitionTaskScheduler::areAllPartsProcessed( + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk) +{ + Strings parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_path / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, will not try to commit export partition"); + return false; + } + + if (!parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: There are still parts in processing or pending, will not try to commit export partition"); + return false; + } + + return true; +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h new file mode 100644 index 000000000000..29a41fde1cb9 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Exception; +class StorageReplicatedMergeTree; + +struct ExportReplicatedMergeTreePartitionManifest; + +/// todo arthur remember to add check(lock, version) when updating stuff because maybe if we believe we have the lock, we might not actually have it +class ExportPartitionTaskScheduler +{ +public: + ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage); + + void run(); +private: + StorageReplicatedMergeTree & storage; + + /// todo arthur maybe it is invalid to grab the manifst here + void handlePartExportCompletion( + const std::string & export_key, + const std::string & part_name, + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const MergeTreePartExportManifest::CompletionCallbackResult & result); + + void handlePartExportSuccess( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::vector & relative_paths_in_destination_storage + ); + + void handlePartExportFailure( + const std::filesystem::path & processing_parts_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::optional & exception, + size_t max_retries); + + bool tryToMovePartToProcessed( + const std::filesystem::path & export_path, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::vector & relative_paths_in_destination_storage, + const zkutil::ZooKeeperPtr & zk + ); + + bool areAllPartsProcessed( + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk + ); +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionUtils.cpp b/src/Storages/MergeTree/ExportPartitionUtils.cpp new file mode 100644 index 000000000000..51ed72b7f6f6 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionUtils.cpp @@ -0,0 +1,104 @@ +#include +#include +#include +#include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" +#include + +namespace DB +{ + +namespace fs = std::filesystem; + +namespace ExportPartitionUtils +{ + /// Collect all the exported paths from the processed parts + /// If multiRead is supported by the keeper implementation, it is done in a single request + /// Otherwise, multiple async requests are sent + std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path) + { + std::vector exported_paths; + + LOG_INFO(log, "ExportPartition: Getting exported paths for {}", export_path); + + const auto processed_parts_path = fs::path(export_path) / "processed"; + + std::vector processed_parts; + if (Coordination::Error::ZOK != zk->tryGetChildren(processed_parts_path, processed_parts)) + { + /// todo arthur do something here + LOG_INFO(log, "ExportPartition: Failed to get parts children, exiting"); + return {}; + } + + std::vector get_paths; + + for (const auto & processed_part : processed_parts) + { + get_paths.emplace_back(processed_parts_path / processed_part); + } + + auto responses = zk->tryGet(get_paths); + + responses.waitForResponses(); + + for (size_t i = 0; i < responses.size(); ++i) + { + if (responses[i].error != Coordination::Error::ZOK) + { + /// todo arthur what to do in this case? + /// It could be that zk is corrupt, in that case we should fail the task + /// but it can also be some temporary network issue? not sure + LOG_INFO(log, "ExportPartition: Failed to get exported path, exiting"); + return {}; + } + + const auto processed_part_entry = ExportReplicatedMergeTreePartitionProcessedPartEntry::fromJsonString(responses[i].data); + + for (const auto & path_in_destination : processed_part_entry.paths_in_destination) + { + exported_paths.emplace_back(path_in_destination); + } + } + + return exported_paths; + } + + void commit( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const zkutil::ZooKeeperPtr & zk, + const LoggerPtr & log, + const std::string & entry_path, + const ContextPtr & context) + { + const auto exported_paths = ExportPartitionUtils::getExportedPaths(log, zk, entry_path); + + if (exported_paths.empty()) + { + LOG_WARNING(log, "ExportPartition: No exported paths found, will not commit export. This might be a bug"); + return; + } + + //// not checking for an exact match because a single part might generate multiple files + if (exported_paths.size() < manifest.parts.size()) + { + LOG_WARNING(log, "ExportPartition: Reached the commit phase, but exported paths size is less than the number of parts, will not commit export. This might be a bug"); + return; + } + + destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); + + LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); + if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::COMPLETED)).data(), -1)) + { + LOG_INFO(log, "ExportPartition: Marked export as completed"); + } + else + { + LOG_INFO(log, "ExportPartition: Failed to mark export as completed, will not try to fix it"); + } + } +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionUtils.h b/src/Storages/MergeTree/ExportPartitionUtils.h new file mode 100644 index 000000000000..40fe04a5bfd3 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionUtils.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include +#include +#include "Storages/IStorage.h" + +namespace DB +{ + +struct ExportReplicatedMergeTreePartitionManifest; + +namespace ExportPartitionUtils +{ + std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path); + + void commit( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const zkutil::ZooKeeperPtr & zk, + const LoggerPtr & log, + const std::string & entry_path, + const ContextPtr & context + ); +} + +} diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0fdb1ddcde69..95a9a3d88c96 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -287,6 +287,42 @@ String IMergeTreeDataPart::MinMaxIndex::getFileColumnName(const String & column_ return stream_name; } +Block IMergeTreeDataPart::MinMaxIndex::getBlock(const MergeTreeData & data) const +{ + if (!initialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to get block from uninitialized MinMax index."); + + Block block; + + const auto metadata_snapshot = data.getInMemoryMetadataPtr(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + + const auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); + const auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); + const auto minmax_idx_size = minmax_column_types.size(); + + for (size_t i = 0; i < minmax_idx_size; ++i) + { + const auto & data_type = minmax_column_types[i]; + const auto & column_name = minmax_column_names[i]; + + const auto column = data_type->createColumn(); + + auto range = hyperrectangle.at(i); + range.shrinkToIncludedIfPossible(); + + const auto & min_val = range.left; + const auto & max_val = range.right; + + column->insert(min_val); + column->insert(max_val); + + block.insert(ColumnWithTypeAndName(column->getPtr(), data_type, column_name)); + } + + return block; +} + void IMergeTreeDataPart::incrementStateMetric(MergeTreeDataPartState state_) const { switch (state_) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2f47d9c27e8b..1aabd0999ee9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -369,6 +369,8 @@ class IMergeTreeDataPart : public std::enable_shared_from_this; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cdd229c91bfa..9337759caf1e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2,7 +2,9 @@ #include #include +#include #include +#include #include #include #include @@ -23,6 +25,11 @@ #include #include #include +#include "Storages/MergeTree/ExportPartTask.h" +#include +#include +#include +#include #include #include #include @@ -39,6 +46,7 @@ #include #include #include +#include #include #include #include @@ -86,6 +94,7 @@ #include #include #include +#include #include @@ -104,6 +113,7 @@ #include #include #include +#include #include #include @@ -150,6 +160,10 @@ namespace ProfileEvents extern const Event RestorePartsSkippedFiles; extern const Event RestorePartsSkippedBytes; extern const Event LoadedStatisticsMicroseconds; + extern const Event PartsExports; + extern const Event PartsExportTotalMilliseconds; + extern const Event PartsExportFailures; + extern const Event PartsExportDuplicated; } namespace CurrentMetrics @@ -199,6 +213,13 @@ namespace Setting extern const SettingsUInt64 max_table_size_to_drop; extern const SettingsBool use_statistics; extern const SettingsBool use_statistics_cache; + extern const SettingsBool allow_experimental_export_merge_tree_part; + extern const SettingsUInt64 min_bytes_to_use_direct_io; + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; + extern const SettingsBool output_format_parallel_formatting; + extern const SettingsBool output_format_parquet_parallel_encoding; + extern const SettingsBool export_merge_tree_part_throw_on_pending_mutations; + extern const SettingsBool export_merge_tree_part_throw_on_pending_patch_parts; } namespace MergeTreeSetting @@ -325,6 +346,9 @@ namespace ErrorCodes extern const int CANNOT_FORGET_PARTITION; extern const int DATA_TYPE_CANNOT_BE_USED_IN_KEY; extern const int TOO_LARGE_LIGHTWEIGHT_UPDATES; + extern const int UNKNOWN_TABLE; + extern const int FILE_ALREADY_EXISTS; + extern const int PENDING_MUTATIONS_NOT_ALLOWED; } static void checkSuspiciousIndices(const ASTFunction * index_function) @@ -4675,8 +4699,6 @@ void MergeTreeData::changeSettings( { if (new_settings) { - bool has_storage_policy_changed = false; - const auto & new_changes = new_settings->as().changes; StoragePolicyPtr new_storage_policy = nullptr; @@ -4715,8 +4737,6 @@ void MergeTreeData::changeSettings( disk->createDirectories(fs::path(relative_data_path) / DETACHED_DIR_NAME); } /// FIXME how would that be done while reloading configuration??? - - has_storage_policy_changed = true; } } } @@ -4746,9 +4766,6 @@ void MergeTreeData::changeSettings( } setInMemoryMetadata(new_metadata); - - if (has_storage_policy_changed) - startBackgroundMovesIfNeeded(); } } @@ -6418,6 +6435,189 @@ void MergeTreeData::movePartitionToTable(const PartitionCommand & command, Conte movePartitionToTable(dest_storage, command.partition, query_context); } +void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextPtr query_context) +{ + if (!query_context->getSettingsRef()[Setting::allow_experimental_export_merge_tree_part]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree part is experimental. Set `allow_experimental_export_merge_tree_part` to enable it"); + } + + const auto part_name = command.partition->as().value.safeGet(); + + if (!command.to_table_function) + { + const auto database_name = query_context->resolveDatabase(command.to_database); + exportPartToTable(part_name, StorageID{database_name, command.to_table}, generateSnowflakeIDString(), query_context); + + return; + } + + auto table_function_ast = command.to_table_function; + auto table_function_ptr = TableFunctionFactory::instance().get(command.to_table_function, query_context); + + if (table_function_ptr->needStructureHint()) + { + const auto source_metadata_ptr = getInMemoryMetadataPtr(); + + /// Grab only the readable columns from the source metadata to skip ephemeral columns + const auto readable_columns = ColumnsDescription(source_metadata_ptr->getColumns().getReadable()); + table_function_ptr->setStructureHint(readable_columns); + } + + if (command.partition_by_expr) + { + table_function_ptr->setPartitionBy(command.partition_by_expr); + } + + auto dest_storage = table_function_ptr->execute( + table_function_ast, + query_context, + table_function_ptr->getName(), + /* cached_columns */ {}, + /* use_global_context */ false, + /* is_insert_query */ true); + + if (!dest_storage) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to reconstruct destination storage"); + } + + exportPartToTable(part_name, dest_storage, generateSnowflakeIDString(), query_context); +} + +void MergeTreeData::exportPartToTable( + const std::string & part_name, + const StorageID & destination_storage_id, + const String & transaction_id, + ContextPtr query_context, + bool allow_outdated_parts, + std::function completion_callback) +{ + auto dest_storage = DatabaseCatalog::instance().getTable(destination_storage_id, query_context); + + if (destination_storage_id == this->getStorageID()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); + } + + exportPartToTable(part_name, dest_storage, transaction_id, query_context, allow_outdated_parts, completion_callback); +} + +void MergeTreeData::exportPartToTable( + const std::string & part_name, + const StoragePtr & dest_storage, + const String & transaction_id, + ContextPtr query_context, + bool allow_outdated_parts, + std::function completion_callback) +{ + if (!dest_storage->supportsImport()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support MergeTree parts or uses unsupported partitioning", dest_storage->getName()); + + auto query_to_string = [] (const ASTPtr & ast) + { + return ast ? ast->formatWithSecretsOneLine() : ""; + }; + + auto source_metadata_ptr = getInMemoryMetadataPtr(); + auto destination_metadata_ptr = dest_storage->getInMemoryMetadataPtr(); + + const auto & source_columns = source_metadata_ptr->getColumns(); + + const auto & destination_columns = destination_metadata_ptr->getColumns(); + + /// compare all source readable columns with all destination insertable columns + /// this allows us to skip ephemeral columns + if (source_columns.getReadable().sizeOfDifference(destination_columns.getInsertable())) + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); + + if (query_to_string(source_metadata_ptr->getPartitionKeyAST()) != query_to_string(destination_metadata_ptr->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + + if (!part) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No such data part '{}' to export in table '{}'", + part_name, getStorageID().getFullTableName()); + + if (part->getState() == MergeTreeDataPartState::Outdated && !allow_outdated_parts) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Part {} is in the outdated state and cannot be exported", + part_name); + + const bool throw_on_pending_mutations = query_context->getSettingsRef()[Setting::export_merge_tree_part_throw_on_pending_mutations]; + const bool throw_on_pending_patch_parts = query_context->getSettingsRef()[Setting::export_merge_tree_part_throw_on_pending_patch_parts]; + + MergeTreeData::IMutationsSnapshot::Params mutations_snapshot_params + { + .metadata_version = source_metadata_ptr->getMetadataVersion(), + .min_part_metadata_version = part->getMetadataVersion(), + .need_data_mutations = throw_on_pending_mutations, + .need_alter_mutations = throw_on_pending_mutations || throw_on_pending_patch_parts, + .need_patch_parts = throw_on_pending_patch_parts, + }; + + const auto mutations_snapshot = getMutationsSnapshot(mutations_snapshot_params); + + const auto alter_conversions = getAlterConversionsForPart(part, mutations_snapshot, query_context); + + /// re-check `throw_on_pending_mutations` because `pending_mutations` might have been filled due to `throw_on_pending_patch_parts` + if (throw_on_pending_mutations && alter_conversions->hasMutations()) + { + throw Exception(ErrorCodes::PENDING_MUTATIONS_NOT_ALLOWED, + "Part {} can not be exported because there are pending mutations. Either wait for the mutations to be applied or set `export_merge_tree_part_throw_on_pending_mutations` to false", + part_name); + } + + if (alter_conversions->hasPatches()) + { + throw Exception(ErrorCodes::PENDING_MUTATIONS_NOT_ALLOWED, + "Part {} can not be exported because there are pending patch parts. Either wait for the patch parts to be applied or set `export_merge_tree_part_throw_on_pending_patch_parts` to false", + part_name); + } + + { + const auto format_settings = getFormatSettings(query_context); + MergeTreePartExportManifest manifest( + dest_storage, + part, + transaction_id, + query_context->getCurrentQueryId(), + query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value, + query_context->getSettingsCopy(), + source_metadata_ptr, + completion_callback); + + std::lock_guard lock(export_manifests_mutex); + + if (!export_manifests.emplace(std::move(manifest)).second) + { + throw Exception(ErrorCodes::ABORTED, "Data part '{}' is already being exported", part_name); + } + } + + background_moves_assignee.trigger(); +} + +void MergeTreeData::killExportPart(const String & transaction_id) +{ + std::lock_guard lock(export_manifests_mutex); + + std::erase_if(export_manifests, [&](const auto & manifest) + { + if (manifest.transaction_id == transaction_id) + { + if (manifest.task) + manifest.task->cancel(); + + return true; + } + return false; + }); +} + void MergeTreeData::movePartitionToShard(const ASTPtr & /*partition*/, bool /*move_part*/, const String & /*to*/, ContextPtr /*query_context*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MOVE PARTITION TO SHARD is not supported by storage {}", getName()); @@ -6469,6 +6669,17 @@ Pipe MergeTreeData::alterPartition( } } break; + case PartitionCommand::EXPORT_PART: + { + exportPartToTable(command, query_context); + break; + } + + case PartitionCommand::EXPORT_PARTITION: + { + exportPartitionToTable(command, query_context); + break; + } case PartitionCommand::DROP_DETACHED_PARTITION: dropDetached(command.partition, command.part, query_context); @@ -8860,6 +9071,33 @@ std::pair MergeTreeData::cloneAn return std::make_pair(dst_data_part, std::move(temporary_directory_lock)); } +std::vector MergeTreeData::getExportsStatus() const +{ + std::lock_guard lock(export_manifests_mutex); + std::vector result; + + auto source_database = getStorageID().database_name; + auto source_table = getStorageID().table_name; + + for (const auto & manifest : export_manifests) + { + MergeTreeExportStatus status; + + status.source_database = source_database; + status.source_table = source_table; + const auto destination_storage_id = manifest.destination_storage_ptr->getStorageID(); + status.destination_database = destination_storage_id.database_name; + status.destination_table = destination_storage_id.table_name; + status.create_time = manifest.create_time; + status.part_name = manifest.data_part->name; + + result.emplace_back(std::move(status)); + } + + return result; +} + + bool MergeTreeData::canUseAdaptiveGranularity() const { const auto settings = getSettings(); @@ -9171,7 +9409,8 @@ void MergeTreeData::writePartLog( const DataPartPtr & result_part, const DataPartsVector & source_parts, const MergeListEntry * merge_entry, - std::shared_ptr profile_counters) + std::shared_ptr profile_counters, + const ExportsListEntry * exports_entry) try { auto table_id = getStorageID(); @@ -9239,6 +9478,16 @@ try part_log_elem.rows = (*merge_entry)->rows_written; part_log_elem.peak_memory_usage = (*merge_entry)->getMemoryTracker().getPeak(); } + else if (exports_entry) + { + part_log_elem.rows_read = (*exports_entry)->rows_read; + part_log_elem.bytes_read_uncompressed = (*exports_entry)->bytes_read_uncompressed; + part_log_elem.peak_memory_usage = (*exports_entry)->getPeakMemoryUsage(); + part_log_elem.query_id = (*exports_entry)->query_id; + + /// no need to lock because at this point no one is writing to the destination file paths + part_log_elem.remote_file_paths = (*exports_entry)->destination_file_paths; + } if (profile_counters) { @@ -9280,21 +9529,46 @@ MergeTreeData::CurrentlyMovingPartsTagger::~CurrentlyMovingPartsTagger() bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) { - if (parts_mover.moves_blocker.isCancelled()) - return false; + if (!parts_mover.moves_blocker.isCancelled()) + { + auto moving_tagger = selectPartsForMove(); + if (!moving_tagger->parts_to_move.empty()) + { + assignee.scheduleMoveTask(std::make_shared( + [this, moving_tagger] () mutable + { + ReadSettings read_settings = Context::getGlobalContextInstance()->getReadSettings(); + WriteSettings write_settings = Context::getGlobalContextInstance()->getWriteSettings(); + return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved; + }, moves_assignee_trigger, getStorageID())); + return true; + } + } - auto moving_tagger = selectPartsForMove(); - if (moving_tagger->parts_to_move.empty()) - return false; + std::lock_guard lock(export_manifests_mutex); - assignee.scheduleMoveTask(std::make_shared( - [this, moving_tagger] () mutable + for (auto & manifest : export_manifests) + { + if (manifest.in_progress) { - ReadSettings read_settings = Context::getGlobalContextInstance()->getReadSettings(); - WriteSettings write_settings = Context::getGlobalContextInstance()->getWriteSettings(); - return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved; - }, moves_assignee_trigger, getStorageID())); - return true; + continue; + } + + auto task = std::make_shared(*this, manifest); + + manifest.in_progress = assignee.scheduleMoveTask(task); + + if (!manifest.in_progress) + { + continue; + } + + manifest.task = task; + + return true; + } + + return false; } bool MergeTreeData::areBackgroundMovesNeeded() const @@ -9512,6 +9786,10 @@ bool MergeTreeData::canUsePolymorphicParts() const return canUsePolymorphicParts(*getSettings(), unused); } +void MergeTreeData::startBackgroundMoves() +{ + background_moves_assignee.start(); +} void MergeTreeData::checkDropOrRenameCommandDoesntAffectInProgressMutations( const AlterCommand & command, const std::map & unfinished_mutations, ContextPtr local_context) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 05ac33b04d9f..152c15701505 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +39,8 @@ #include #include #include +#include +#include #include #include @@ -1020,6 +1023,31 @@ class MergeTreeData : public IStorage, public WithMutableContext /// Moves partition to specified Table void movePartitionToTable(const PartitionCommand & command, ContextPtr query_context); + void exportPartToTable(const PartitionCommand & command, ContextPtr query_context); + + void exportPartToTable( + const std::string & part_name, + const StoragePtr & destination_storage, + const String & transaction_id, + ContextPtr query_context, + bool allow_outdated_parts = false, + std::function completion_callback = {}); + + void exportPartToTable( + const std::string & part_name, + const StorageID & destination_storage_id, + const String & transaction_id, + ContextPtr query_context, + bool allow_outdated_parts = false, + std::function completion_callback = {}); + + void killExportPart(const String & transaction_id); + + virtual void exportPartitionToTable(const PartitionCommand &, ContextPtr) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "EXPORT PARTITION is not implemented for engine {}", getName()); + } + /// Checks that Partition could be dropped right now /// Otherwise - throws an exception with detailed information. /// We do not use mutex because it is not very important that the size could change during the operation. @@ -1094,6 +1122,7 @@ class MergeTreeData : public IStorage, public WithMutableContext const WriteSettings & write_settings); virtual std::vector getMutationsStatus() const = 0; + std::vector getExportsStatus() const; /// Returns true if table can create new parts with adaptive granularity /// Has additional constraint in replicated version @@ -1276,6 +1305,10 @@ class MergeTreeData : public IStorage, public WithMutableContext /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; + mutable std::mutex export_manifests_mutex; + + std::set export_manifests; + PinnedPartUUIDsPtr getPinnedPartUUIDs() const; /// Schedules background job to like merge/mutate/fetch an executor @@ -1370,6 +1403,7 @@ class MergeTreeData : public IStorage, public WithMutableContext friend class IPartMetadataManager; friend class IMergedBlockOutputStream; // for access to log friend struct DataPartsLock; // for access to shared_parts_list/shared_ranges_in_parts + friend class ExportPartTask; bool require_part_metadata; @@ -1415,6 +1449,8 @@ class MergeTreeData : public IStorage, public WithMutableContext size_t getColumnsDescriptionsCacheSize() const; protected: + void startBackgroundMoves(); + /// Engine-specific methods BrokenPartCallback broken_part_callback; @@ -1673,7 +1709,8 @@ class MergeTreeData : public IStorage, public WithMutableContext const DataPartPtr & result_part, const DataPartsVector & source_parts, const MergeListEntry * merge_entry, - std::shared_ptr profile_counters); + std::shared_ptr profile_counters, + const ExportsListEntry * exports_entry = nullptr); /// If part is assigned to merge or mutation (possibly replicated) /// Should be overridden by children, because they can have different @@ -1888,8 +1925,6 @@ class MergeTreeData : public IStorage, public WithMutableContext bool canUsePolymorphicParts(const MergeTreeSettings & settings, String & out_reason) const; - virtual void startBackgroundMovesIfNeeded() = 0; - bool allow_nullable_key = false; bool allow_reverse_key = false; diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h new file mode 100644 index 000000000000..05506ecb004a --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -0,0 +1,50 @@ +#include +#include + +namespace DB +{ + +struct MergeTreeExportManifest +{ + using DataPartPtr = std::shared_ptr; + + + MergeTreeExportManifest( + const StorageID & destination_storage_id_, + const DataPartPtr & data_part_, + bool overwrite_file_if_exists_, + const FormatSettings & format_settings_) + : destination_storage_id(destination_storage_id_), + data_part(data_part_), + overwrite_file_if_exists(overwrite_file_if_exists_), + format_settings(format_settings_), + create_time(time(nullptr)) {} + + StorageID destination_storage_id; + DataPartPtr data_part; + bool overwrite_file_if_exists; + FormatSettings format_settings; + + time_t create_time; + mutable bool in_progress = false; + + bool operator<(const MergeTreeExportManifest & rhs) const + { + // Lexicographic comparison: first compare destination storage, then part name + auto lhs_storage = destination_storage_id.getQualifiedName(); + auto rhs_storage = rhs.destination_storage_id.getQualifiedName(); + + if (lhs_storage != rhs_storage) + return lhs_storage < rhs_storage; + + return data_part->name < rhs.data_part->name; + } + + bool operator==(const MergeTreeExportManifest & rhs) const + { + return destination_storage_id.getQualifiedName() == rhs.destination_storage_id.getQualifiedName() + && data_part->name == rhs.data_part->name; + } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h new file mode 100644 index 000000000000..b9527d5140f6 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -0,0 +1,94 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class Exception; + +class ExportPartTask; + +struct MergeTreePartExportManifest +{ + using FileAlreadyExistsPolicy = MergeTreePartExportFileAlreadyExistsPolicy; + + using DataPartPtr = std::shared_ptr; + + struct CompletionCallbackResult + { + private: + CompletionCallbackResult(bool success_, const std::vector & relative_paths_in_destination_storage_, std::optional exception_) + : success(success_), relative_paths_in_destination_storage(relative_paths_in_destination_storage_), exception(std::move(exception_)) {} + public: + + static CompletionCallbackResult createSuccess(const std::vector & relative_paths_in_destination_storage_) + { + return CompletionCallbackResult(true, relative_paths_in_destination_storage_, std::nullopt); + } + + static CompletionCallbackResult createFailure(Exception exception_) + { + return CompletionCallbackResult(false, {}, std::move(exception_)); + } + + bool success = false; + std::vector relative_paths_in_destination_storage; + std::optional exception; + }; + + MergeTreePartExportManifest( + const StoragePtr destination_storage_ptr_, + const DataPartPtr & data_part_, + const String & transaction_id_, + const String & query_id_, + FileAlreadyExistsPolicy file_already_exists_policy_, + const Settings & settings_, + const StorageMetadataPtr & metadata_snapshot_, + std::function completion_callback_ = {}) + : destination_storage_ptr(destination_storage_ptr_), + data_part(data_part_), + transaction_id(transaction_id_), + query_id(query_id_), + file_already_exists_policy(file_already_exists_policy_), + settings(settings_), + metadata_snapshot(metadata_snapshot_), + completion_callback(completion_callback_), + create_time(time(nullptr)) {} + + StoragePtr destination_storage_ptr; + DataPartPtr data_part; + /// Used for killing the export. + String transaction_id; + String query_id; + FileAlreadyExistsPolicy file_already_exists_policy; + Settings settings; + + /// Metadata snapshot captured at the time of query validation to prevent race conditions with mutations + /// Otherwise the export could fail if the schema changes between validation and execution + StorageMetadataPtr metadata_snapshot; + + std::function completion_callback; + + time_t create_time; + mutable bool in_progress = false; + mutable std::shared_ptr task = nullptr; + + bool operator<(const MergeTreePartExportManifest & rhs) const + { + return data_part->name < rhs.data_part->name; + } + + bool operator==(const MergeTreePartExportManifest & rhs) const + { + return data_part->name == rhs.data_part->name; + } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartExportStatus.h b/src/Storages/MergeTree/MergeTreePartExportStatus.h new file mode 100644 index 000000000000..e71a2f15e6ed --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartExportStatus.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +struct MergeTreeExportStatus +{ + String source_database; + String source_table; + String destination_database; + String destination_table; + time_t create_time = 0; + std::string part_name; +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index ea16a5ff6602..6eed4c712e11 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -467,6 +467,22 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } +Block MergeTreePartition::getBlockWithPartitionValues(const NamesAndTypesList & partition_columns) const +{ + chassert(partition_columns.size() == value.size()); + + Block result; + + std::size_t i = 0; + for (const auto & partition_column : partition_columns) + { + auto column = partition_column.type->createColumnConst(1, value[i++]); + result.insert({column, partition_column.type, partition_column.name}); + } + + return result; +} + NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 4338b216cdb8..811cfdc2a90c 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -60,6 +60,8 @@ struct MergeTreePartition void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); + Block getBlockWithPartitionValues(const NamesAndTypesList & partition_columns) const; + /// Adjust partition key and execute its expression on block. Return sample block according to used expression. static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index b2b0862ce87e..bfa7ace6f52e 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -169,6 +169,10 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( addThrottler(read_settings.remote_throttler, context->getMergesThrottler()); addThrottler(read_settings.local_throttler, context->getMergesThrottler()); break; + case Export: + addThrottler(read_settings.local_throttler, context->getExportsThrottler()); + addThrottler(read_settings.remote_throttler, context->getExportsThrottler()); + break; } MergeTreeReadTask::Extras extras = diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index abba230d9e79..a858adf33bb5 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -15,6 +15,7 @@ enum MergeTreeSequentialSourceType { Mutation, Merge, + Export, }; /// Create stream for reading single part from MergeTree. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 7832bfed96d9..7f1554ad67eb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -27,6 +28,11 @@ namespace MergeTreeSetting extern const MergeTreeSettingsSeconds zookeeper_session_expiration_check_period; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ErrorCodes { extern const int REPLICA_IS_ALREADY_ACTIVE; @@ -171,10 +177,19 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.mutations_updating_task->activateAndSchedule(); storage.mutations_finalizing_task->activateAndSchedule(); storage.merge_selecting_task->activateAndSchedule(); + + if (storage.getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + storage.export_merge_tree_partition_updating_task->activateAndSchedule(); + storage.export_merge_tree_partition_select_task->activateAndSchedule(); + storage.export_merge_tree_partition_status_handling_task->activateAndSchedule(); + } + storage.cleanup_thread.start(); storage.async_block_ids_cache.start(); storage.part_check_thread.start(); + LOG_DEBUG(log, "Table started successfully"); return true; } diff --git a/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp new file mode 100644 index 000000000000..c9e3ffd9eef9 --- /dev/null +++ b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp @@ -0,0 +1,75 @@ +#include +#include + +namespace DB +{ + +class ExportPartitionOrderingTest : public ::testing::Test +{ +protected: + ExportPartitionTaskEntriesContainer container; + ExportPartitionTaskEntriesContainer::index::type & by_key; + ExportPartitionTaskEntriesContainer::index::type & by_create_time; + + ExportPartitionOrderingTest() + : by_key(container.get()) + , by_create_time(container.get()) + { + } +}; + +TEST_F(ExportPartitionOrderingTest, IterationOrderMatchesCreateTime) +{ + time_t base_time = 1000; + + ExportReplicatedMergeTreePartitionManifest manifest1; + manifest1.partition_id = "2020"; + manifest1.destination_database = "db1"; + manifest1.destination_table = "table1"; + manifest1.transaction_id = "tx1"; + manifest1.create_time = base_time + 300; // Latest + + ExportReplicatedMergeTreePartitionManifest manifest2; + manifest2.partition_id = "2021"; + manifest2.destination_database = "db1"; + manifest2.destination_table = "table1"; + manifest2.transaction_id = "tx2"; + manifest2.create_time = base_time + 100; // Middle + + ExportReplicatedMergeTreePartitionManifest manifest3; + manifest3.partition_id = "2022"; + manifest3.destination_database = "db1"; + manifest3.destination_table = "table1"; + manifest3.transaction_id = "tx3"; + manifest3.create_time = base_time; // Oldest + + ExportReplicatedMergeTreePartitionTaskEntry entry1{manifest1, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry2{manifest2, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry3{manifest3, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + + // Insert in reverse order + by_key.insert(entry1); + by_key.insert(entry2); + by_key.insert(entry3); + + // Verify iteration order matches create_time (ascending) + auto it = by_create_time.begin(); + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2022"); // Oldest first + EXPECT_EQ(it->manifest.create_time, base_time); + + ++it; + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2021"); + EXPECT_EQ(it->manifest.create_time, base_time + 100); + + ++it; + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2020"); + EXPECT_EQ(it->manifest.create_time, base_time + 300); + + ++it; + EXPECT_EQ(it, by_create_time.end()); +} + +} diff --git a/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.cpp new file mode 100644 index 000000000000..f46985b9a52f --- /dev/null +++ b/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.cpp @@ -0,0 +1,153 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FILE_ALREADY_EXISTS; +} + +MultiFileStorageObjectStorageSink::MultiFileStorageObjectStorageSink( + const std::string & base_path_, + const String & transaction_id_, + ObjectStoragePtr object_storage_, + StorageObjectStorageConfigurationPtr configuration_, + std::size_t max_bytes_per_file_, + std::size_t max_rows_per_file_, + bool overwrite_if_exists_, + const std::function & new_file_path_callback_, + const std::optional & format_settings_, + SharedHeader sample_block_, + ContextPtr context_) + : SinkToStorage(sample_block_), + base_path(base_path_), + transaction_id(transaction_id_), + object_storage(object_storage_), + configuration(configuration_), + max_bytes_per_file(max_bytes_per_file_), + max_rows_per_file(max_rows_per_file_), + overwrite_if_exists(overwrite_if_exists_), + new_file_path_callback(new_file_path_callback_), + format_settings(format_settings_), + sample_block(sample_block_), + context(context_) +{ + current_sink = createNewSink(); +} + +MultiFileStorageObjectStorageSink::~MultiFileStorageObjectStorageSink() +{ + if (isCancelled()) + current_sink->cancel(); +} + +/// Adds a counter that represents file index to the file path. +/// Example: +/// Input is `table_root/year=2025/month=12/day=12/file.parquet` +/// Output is `table_root/year=2025/month=12/day=12/file.1.parquet` +std::string MultiFileStorageObjectStorageSink::generateNewFilePath() +{ + const auto file_format = Poco::toLower(configuration->format); + const auto index_string = std::to_string(file_paths.size() + 1); + std::size_t pos = base_path.rfind(file_format); + + /// normal case - path ends with the file format + if (pos != std::string::npos) + { + const auto path_without_extension = base_path.substr(0, pos); + const auto file_format_extension = "." + file_format; + + return path_without_extension + index_string + file_format_extension; + } + + /// if no extension is found, just append the index - I am not even sure this is possible + return base_path + "." + index_string; +} + +std::shared_ptr MultiFileStorageObjectStorageSink::createNewSink() +{ + auto new_path = generateNewFilePath(); + + /// todo + /// sounds like bad design, but callers might decide to ignore the exception, and if we throw it before the callback + /// they will not be able to grab the file path. + /// maybe I should consider moving the file already exists policy in here? + new_file_path_callback(new_path); + + file_paths.emplace_back(std::move(new_path)); + + if (!overwrite_if_exists && object_storage->exists(StoredObject(file_paths.back()))) + { + throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exists", file_paths.back()); + } + + return std::make_shared( + file_paths.back(), + object_storage, + format_settings, + sample_block, + context, + configuration->format, + configuration->compression_method); +} + +void MultiFileStorageObjectStorageSink::consume(Chunk & chunk) +{ + if (isCancelled()) + { + current_sink->cancel(); + return; + } + + const auto written_bytes = current_sink->getWrittenBytes(); + + const bool exceeded_bytes_limit = max_bytes_per_file && written_bytes >= max_bytes_per_file; + const bool exceeded_rows_limit = max_rows_per_file && current_sink_written_rows >= max_rows_per_file; + + if (exceeded_bytes_limit || exceeded_rows_limit) + { + current_sink->onFinish(); + current_sink = createNewSink(); + current_sink_written_rows = 0; + } + + current_sink->consume(chunk); + current_sink_written_rows += chunk.getNumRows(); +} + +void MultiFileStorageObjectStorageSink::onFinish() +{ + current_sink->onFinish(); + commit(); +} + +void MultiFileStorageObjectStorageSink::commit() +{ + /// the commit file path should be in the same directory as the data files + const auto commit_file_path = fs::path(base_path).parent_path() / ("commit_" + transaction_id); + + if (!overwrite_if_exists && object_storage->exists(StoredObject(commit_file_path))) + { + throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Commit file {} already exists, aborting {} export", commit_file_path, transaction_id); + } + + auto out = object_storage->writeObject( + StoredObject(commit_file_path), + WriteMode::Rewrite, /* attributes= */ + {}, DBMS_DEFAULT_BUFFER_SIZE, + context->getWriteSettings()); + + for (const auto & p : file_paths) + { + out->write(p.data(), p.size()); + out->write("\n", 1); + } + + out->finalize(); +} + +} diff --git a/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.h b/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.h new file mode 100644 index 000000000000..51f6b8094232 --- /dev/null +++ b/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.h @@ -0,0 +1,57 @@ +#pragma once + +#include + +namespace DB +{ + +/// This is useful when the data is too large to fit into a single file. +/// It will create a new file when the current file exceeds the max bytes or max rows limit. +/// Ships a commit file including the list of data files to make it transactional +class MultiFileStorageObjectStorageSink : public SinkToStorage +{ +public: + MultiFileStorageObjectStorageSink( + const std::string & base_path_, + const String & transaction_id_, + ObjectStoragePtr object_storage_, + StorageObjectStorageConfigurationPtr configuration_, + std::size_t max_bytes_per_file_, + std::size_t max_rows_per_file_, + bool overwrite_if_exists_, + const std::function & new_file_path_callback_, + const std::optional & format_settings_, + SharedHeader sample_block_, + ContextPtr context_); + + ~MultiFileStorageObjectStorageSink() override; + + void consume(Chunk & chunk) override; + + void onFinish() override; + + String getName() const override { return "MultiFileStorageObjectStorageSink"; } + +private: + const std::string base_path; + const String transaction_id; + ObjectStoragePtr object_storage; + StorageObjectStorageConfigurationPtr configuration; + std::size_t max_bytes_per_file; + std::size_t max_rows_per_file; + bool overwrite_if_exists; + std::function new_file_path_callback; + const std::optional format_settings; + SharedHeader sample_block; + ContextPtr context; + + std::vector file_paths; + std::shared_ptr current_sink; + std::size_t current_sink_written_rows = 0; + + std::string generateNewFilePath(); + std::shared_ptr createNewSink(); + void commit(); +}; + +} diff --git a/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h b/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h new file mode 100644 index 000000000000..a1f21dc502d5 --- /dev/null +++ b/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h @@ -0,0 +1,83 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + struct ObjectStorageFilePathGenerator + { + virtual ~ObjectStorageFilePathGenerator() = default; + std::string getPathForWrite(const std::string & partition_id) const { + return getPathForWrite(partition_id, ""); + } + virtual std::string getPathForWrite(const std::string & partition_id, const std::string & /* file_name_override */) const = 0; + virtual std::string getPathForRead() const = 0; + }; + + struct ObjectStorageWildcardFilePathGenerator : ObjectStorageFilePathGenerator + { + static constexpr const char * FILE_WILDCARD = "{_file}"; + explicit ObjectStorageWildcardFilePathGenerator(const std::string & raw_path_) : raw_path(raw_path_) {} + + using ObjectStorageFilePathGenerator::getPathForWrite; // Bring base class overloads into scope + std::string getPathForWrite(const std::string & partition_id, const std::string & file_name_override) const override + { + const auto partition_replaced_path = PartitionedSink::replaceWildcards(raw_path, partition_id); + const auto final_path = boost::replace_all_copy(partition_replaced_path, FILE_WILDCARD, file_name_override); + return final_path; + } + + std::string getPathForRead() const override + { + return raw_path; + } + + private: + std::string raw_path; + + }; + + struct ObjectStorageAppendFilePathGenerator : ObjectStorageFilePathGenerator + { + explicit ObjectStorageAppendFilePathGenerator( + const std::string & raw_path_, + const std::string & file_format_) + : raw_path(raw_path_), file_format(Poco::toLower(file_format_)){} + + using ObjectStorageFilePathGenerator::getPathForWrite; // Bring base class overloads into scope + std::string getPathForWrite(const std::string & partition_id, const std::string & file_name_override) const override + { + std::string result; + + result += raw_path; + + if (!result.empty() && result.back() != '/') + { + result += "/"; + } + + /// Not adding '/' because buildExpressionHive() always adds a trailing '/' + result += partition_id; + + const auto file_name = file_name_override.empty() ? std::to_string(generateSnowflakeID()) : file_name_override; + + result += file_name + "." + file_format; + + return result; + } + + std::string getPathForRead() const override + { + return raw_path + "**." + file_format; + } + + private: + std::string raw_path; + std::string file_format; + }; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 39aab0f30e06..d68c4b4d62c9 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -28,11 +29,13 @@ #include #include #include +#include #include #include #include #include #include +#include namespace DB @@ -52,6 +55,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int INCORRECT_DATA; extern const int BAD_ARGUMENTS; + extern const int FILE_ALREADY_EXISTS; } String StorageObjectStorage::getPathSample(ContextPtr context) @@ -514,7 +518,8 @@ SinkToStoragePtr StorageObjectStorage::write( if (configuration->partition_strategy) { - return std::make_shared(object_storage, configuration, format_settings, sample_block, local_context); + auto sink_creator = std::make_shared(object_storage, configuration, format_settings, sample_block, local_context); + return std::make_shared(configuration->partition_strategy, sink_creator, local_context, sample_block); } auto paths = configuration->getPaths(); @@ -547,6 +552,76 @@ bool StorageObjectStorage::optimize( return configuration->optimize(metadata_snapshot, context, format_settings); } +bool StorageObjectStorage::supportsImport() const +{ + if (!configuration->partition_strategy) + return false; + + if (configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::WILDCARD) + return configuration->getRawPath().hasExportFilenameWildcard(); + + return configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE; +} + + +SinkToStoragePtr StorageObjectStorage::import( + const std::string & file_name, + Block & block_with_partition_values, + const std::function & new_file_path_callback, + bool overwrite_if_exists, + std::size_t max_bytes_per_file, + std::size_t max_rows_per_file, + const std::optional & format_settings_, + ContextPtr local_context) +{ + std::string partition_key; + + if (configuration->partition_strategy) + { + const auto column_with_partition_key = configuration->partition_strategy->computePartitionKey(block_with_partition_values); + + if (!column_with_partition_key->empty()) + { + partition_key = column_with_partition_key->getDataAt(0); + } + } + + const auto base_path = configuration->getPathForWrite(partition_key, file_name).path; + + return std::make_shared( + base_path, + /* transaction_id= */ file_name, /// not pretty, but the sink needs some sort of id to generate the commit file name. Using the source part name should be enough + object_storage, + configuration, + max_bytes_per_file, + max_rows_per_file, + overwrite_if_exists, + new_file_path_callback, + format_settings_ ? format_settings_ : format_settings, + std::make_shared(getInMemoryMetadataPtr()->getSampleBlock()), + local_context); +} + +void StorageObjectStorage::commitExportPartitionTransaction(const String & transaction_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_context) +{ + const String commit_object = configuration->getRawPath().path + "/commit_" + partition_id + "_" + transaction_id; + + /// if file already exists, nothing to be done + if (object_storage->exists(StoredObject(commit_object))) + { + LOG_DEBUG(getLogger("StorageObjectStorage"), "Commit file already exists, nothing to be done: {}", commit_object); + return; + } + + auto out = object_storage->writeObject(StoredObject(commit_object), WriteMode::Rewrite, /* attributes= */ {}, DBMS_DEFAULT_BUFFER_SIZE, local_context->getWriteSettings()); + for (const auto & p : exported_paths) + { + out->write(p.data(), p.size()); + out->write("\n", 1); + } + out->finalize(); +} + void StorageObjectStorage::truncate( const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, @@ -729,5 +804,4 @@ void StorageObjectStorage::checkAlterIsPossible(const AlterCommands & commands, configuration->checkAlterIsPossible(commands); } - } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index e65f6e3495d0..cdf71f68c22d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -7,6 +7,7 @@ #include #include #include +#include "Storages/ObjectStorage/ObjectStorageFilePathGenerator.h" #include #include #include @@ -73,6 +74,25 @@ class StorageObjectStorage : public IStorage ContextPtr context, bool async_insert) override; + + bool supportsImport() const override; + + SinkToStoragePtr import( + const std::string & /* file_name */, + Block & /* block_with_partition_values */, + const std::function & new_file_path_callback, + bool /* overwrite_if_exists */, + std::size_t /* max_bytes_per_file */, + std::size_t /* max_rows_per_file */, + const std::optional & /* format_settings_ */, + ContextPtr /* context */) override; + + void commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + ContextPtr local_context) override; + void truncate( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 4611aa7cb271..cd2afbbb5094 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -110,6 +111,11 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( metadata.setColumns(columns); metadata.setConstraints(constraints_); + if (configuration->partition_strategy) + { + metadata.partition_key = configuration->partition_strategy->getPartitionKeyDescription(); + } + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage( metadata.columns, context_, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index a5e3b0e992d9..b74560e3e1e6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -126,10 +126,20 @@ void StorageObjectStorageConfiguration::initialize( else FormatFactory::instance().checkFormatName(configuration_to_initialize.format); - /// It might be changed on `StorageObjectStorageConfiguration::initPartitionStrategy` + if (configuration_to_initialize.partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE) + { + configuration_to_initialize.file_path_generator = std::make_shared( + configuration_to_initialize.getRawPath().path, + configuration_to_initialize.format); + } + else + { + configuration_to_initialize.file_path_generator = std::make_shared(configuration_to_initialize.getRawPath().path); + } + /// We shouldn't set path for disk setup because path prefix is already set in used object_storage. if (disk_name.empty()) - configuration_to_initialize.read_path = configuration_to_initialize.getRawPath(); + configuration_to_initialize.read_path = configuration_to_initialize.file_path_generator->getPathForRead(); configuration_to_initialize.initialized = true; } @@ -148,7 +158,6 @@ void StorageObjectStorageConfiguration::initPartitionStrategy(ASTPtr partition_b if (partition_strategy) { - read_path = partition_strategy->getPathForRead(getRawPath().path); LOG_DEBUG(getLogger("StorageObjectStorageConfiguration"), "Initialized partition strategy {}", magic_enum::enum_name(partition_strategy_type)); } } @@ -160,14 +169,12 @@ const StorageObjectStorageConfiguration::Path & StorageObjectStorageConfiguratio StorageObjectStorageConfiguration::Path StorageObjectStorageConfiguration::getPathForWrite(const std::string & partition_id) const { - auto raw_path = getRawPath(); - - if (!partition_strategy) - { - return raw_path; - } + return getPathForWrite(partition_id, /* filename_override */ ""); +} - return Path {partition_strategy->getPathForWrite(raw_path.path, partition_id)}; +StorageObjectStorageConfiguration::Path StorageObjectStorageConfiguration::getPathForWrite(const std::string & partition_id, const std::string & filename_override) const +{ + return Path {file_path_generator->getPathForWrite(partition_id, filename_override)}; } bool StorageObjectStorageConfiguration::Path::hasPartitionWildcard() const @@ -176,6 +183,12 @@ bool StorageObjectStorageConfiguration::Path::hasPartitionWildcard() const return path.find(PARTITION_ID_WILDCARD) != String::npos; } +bool StorageObjectStorageConfiguration::Path::hasExportFilenameWildcard() const +{ + return path.find(ObjectStorageWildcardFilePathGenerator::FILE_WILDCARD) != String::npos; +} + + bool StorageObjectStorageConfiguration::Path::hasGlobsIgnorePartitionWildcard() const { if (!hasPartitionWildcard()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index c1dd04340c06..a05a43ce6f74 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { @@ -69,6 +70,7 @@ class StorageObjectStorageConfiguration std::string path; bool hasPartitionWildcard() const; + bool hasExportFilenameWildcard() const; bool hasGlobsIgnorePartitionWildcard() const; bool hasGlobs() const; std::string cutGlobs(bool supports_partial_prefix) const; @@ -100,8 +102,10 @@ class StorageObjectStorageConfiguration virtual const String & getRawURI() const = 0; const Path & getPathForRead() const; + // Path used for writing, it should not be globbed and might contain a partition key Path getPathForWrite(const std::string & partition_id = "") const; + Path getPathForWrite(const std::string & partition_id, const std::string & filename_override) const; void setPathForRead(const Path & path) { @@ -263,11 +267,12 @@ class StorageObjectStorageConfiguration String format = "auto"; String compression_method = "auto"; String structure = "auto"; + PartitionStrategyFactory::StrategyType partition_strategy_type = PartitionStrategyFactory::StrategyType::NONE; + std::shared_ptr partition_strategy; /// Whether partition column values are contained in the actual data. /// And alternative is with hive partitioning, when they are contained in file path. bool partition_columns_in_data_file = true; - std::shared_ptr partition_strategy; protected: void initializeFromParsedArguments(const StorageParsedArguments & parsed_arguments); @@ -286,6 +291,8 @@ class StorageObjectStorageConfiguration // Path used for reading, by default it is the same as `getRawPath` // When using `partition_strategy=hive`, a recursive reading pattern will be appended `'table_root/**.parquet' Path read_path; + + std::shared_ptr file_path_generator; }; using StorageObjectStorageConfigurationPtr = std::shared_ptr; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 2e4fee714f6c..4baa44e55d32 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -136,14 +136,20 @@ size_t StorageObjectStorageSink::getFileSize() const return *result_file_size; } +size_t StorageObjectStorageSink::getWrittenBytes() const +{ + if (!write_buf) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer must be initialized before requesting written bytes"); + return write_buf->count(); +} + PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationPtr configuration_, std::optional format_settings_, SharedHeader sample_block_, ContextPtr context_) - : PartitionedSink(configuration_->partition_strategy, context_, sample_block_) - , object_storage(object_storage_) + : object_storage(object_storage_) , configuration(configuration_) , query_settings(configuration_->getQuerySettings(context_)) , format_settings(format_settings_) @@ -175,10 +181,11 @@ SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String file_path, object_storage, format_settings, - std::make_shared(partition_strategy->getFormatHeader()), + std::make_shared(configuration->partition_strategy->getFormatHeader()), context, configuration->format, - configuration->compression_method); + configuration->compression_method + ); } } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index 823a6170a7e5..f2fc1fdd6d5a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -8,6 +8,8 @@ namespace DB { class StorageObjectStorageSink : public SinkToStorage { +friend class StorageObjectStorageImporterSink; + public: StorageObjectStorageSink( const std::string & path_, @@ -28,6 +30,8 @@ class StorageObjectStorageSink : public SinkToStorage const String & getPath() const { return path; } + size_t getWrittenBytes() const; + size_t getFileSize() const; private: @@ -42,7 +46,7 @@ class StorageObjectStorageSink : public SinkToStorage void cancelBuffers(); }; -class PartitionedStorageObjectStorageSink : public PartitionedSink +class PartitionedStorageObjectStorageSink : public PartitionedSink::SinkCreator { public: PartitionedStorageObjectStorageSink( diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 593f4d895aad..c4d038f925c1 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -131,6 +131,37 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.with_name = command_ast->with_name; return res; } + if (command_ast->type == ASTAlterCommand::EXPORT_PART) + { + PartitionCommand res; + res.type = EXPORT_PART; + res.partition = command_ast->partition->clone(); + res.part = command_ast->part; + res.to_database = command_ast->to_database; + res.to_table = command_ast->to_table; + if (command_ast->to_table_function) + { + res.to_table_function = command_ast->to_table_function->ptr(); + if (command_ast->partition_by_expr) + res.partition_by_expr = command_ast->partition_by_expr->clone(); + } + return res; + } + if (command_ast->type == ASTAlterCommand::EXPORT_PARTITION) + { + PartitionCommand res; + res.type = EXPORT_PARTITION; + res.partition = command_ast->partition->clone(); + res.to_database = command_ast->to_database; + res.to_table = command_ast->to_table; + if (command_ast->to_table_function) + { + res.to_table_function = command_ast->to_table_function->ptr(); + if (command_ast->partition_by_expr) + res.partition_by_expr = command_ast->partition_by_expr->clone(); + } + return res; + } return {}; } @@ -172,6 +203,10 @@ std::string PartitionCommand::typeToString() const return "UNFREEZE ALL"; case PartitionCommand::Type::REPLACE_PARTITION: return "REPLACE PARTITION"; + case PartitionCommand::Type::EXPORT_PART: + return "EXPORT PART"; + case PartitionCommand::Type::EXPORT_PARTITION: + return "EXPORT PARTITION"; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Uninitialized partition command"); } diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 5c2b10097a83..ea5956aad3a0 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -33,6 +33,8 @@ struct PartitionCommand UNFREEZE_ALL_PARTITIONS, UNFREEZE_PARTITION, REPLACE_PARTITION, + EXPORT_PART, + EXPORT_PARTITION, }; Type type = UNKNOWN; @@ -50,10 +52,14 @@ struct PartitionCommand String from_table; bool replace = true; - /// For MOVE PARTITION + /// For MOVE PARTITION and EXPORT PART and EXPORT PARTITION String to_database; String to_table; + /// For EXPORT PART and EXPORT PARTITION with table functions + ASTPtr to_table_function; + ASTPtr partition_by_expr; + /// For FETCH PARTITION - path in ZK to the shard, from which to download the partition. String from_path; diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 61789c3db19a..2be1ba5b93fc 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -26,10 +26,12 @@ namespace ErrorCodes PartitionedSink::PartitionedSink( std::shared_ptr partition_strategy_, + std::shared_ptr sink_creator_, ContextPtr context_, SharedHeader source_header_) : SinkToStorage(source_header_) , partition_strategy(partition_strategy_) + , sink_creator(sink_creator_) , context(context_) , source_header(source_header_) { @@ -41,7 +43,7 @@ SinkPtr PartitionedSink::getSinkForPartitionKey(std::string_view partition_key) auto it = partition_id_to_sink.find(partition_key); if (it == partition_id_to_sink.end()) { - auto sink = createSinkForPartition(std::string{partition_key}); + auto sink = sink_creator->createSinkForPartition(std::string{partition_key}); std::tie(it, std::ignore) = partition_id_to_sink.emplace(partition_key, sink); } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index 3aa9cf6db16a..ee953296d068 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -17,10 +17,17 @@ namespace DB class PartitionedSink : public SinkToStorage { public: + struct SinkCreator + { + virtual ~SinkCreator() = default; + virtual SinkPtr createSinkForPartition(const String & partition_id) = 0; + }; + static constexpr auto PARTITION_ID_WILDCARD = "{_partition_id}"; PartitionedSink( std::shared_ptr partition_strategy_, + std::shared_ptr sink_creator_, ContextPtr context_, SharedHeader source_header_); @@ -34,16 +41,15 @@ class PartitionedSink : public SinkToStorage void onFinish() override; - virtual SinkPtr createSinkForPartition(const String & partition_id) = 0; - static void validatePartitionKey(const String & str, bool allow_slash); static String replaceWildcards(const String & haystack, const String & partition_id); + protected: std::shared_ptr partition_strategy; - private: + std::shared_ptr sink_creator; ContextPtr context; SharedHeader source_header; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 6b028a2371a3..5087df4d25b7 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2074,7 +2074,7 @@ class StorageFileSink final : public SinkToStorage, WithContext std::unique_lock lock; }; -class PartitionedStorageFileSink : public PartitionedSink +class PartitionedStorageFileSink : public PartitionedSink::SinkCreator { public: PartitionedStorageFileSink( @@ -2089,7 +2089,7 @@ class PartitionedStorageFileSink : public PartitionedSink const String format_name_, ContextPtr context_, int flags_) - : PartitionedSink(partition_strategy_, context_, std::make_shared(metadata_snapshot_->getSampleBlock())) + : partition_strategy(partition_strategy_) , path(path_) , metadata_snapshot(metadata_snapshot_) , table_name_for_log(table_name_for_log_) @@ -2105,11 +2105,12 @@ class PartitionedStorageFileSink : public PartitionedSink SinkPtr createSinkForPartition(const String & partition_id) override { - std::string filepath = partition_strategy->getPathForWrite(path, partition_id); + const auto file_path_generator = std::make_shared(path); + std::string filepath = file_path_generator->getPathForWrite(partition_id); fs::create_directories(fs::path(filepath).parent_path()); - validatePartitionKey(filepath, true); + PartitionedSink::validatePartitionKey(filepath, true); checkCreationIsAllowed(context, context->getUserFilesPath(), filepath, /*can_be_directory=*/ true); return std::make_shared( metadata_snapshot, @@ -2126,6 +2127,7 @@ class PartitionedStorageFileSink : public PartitionedSink } private: + std::shared_ptr partition_strategy; const String path; StorageMetadataPtr metadata_snapshot; String table_name_for_log; @@ -2177,7 +2179,7 @@ SinkToStoragePtr StorageFile::write( has_wildcards, /* partition_columns_in_data_file */true); - return std::make_shared( + auto sink_creator = std::make_shared( partition_strategy, metadata_snapshot, getStorageID().getNameForLogs(), @@ -2189,6 +2191,13 @@ SinkToStoragePtr StorageFile::write( format_name, context, flags); + + return std::make_shared( + partition_strategy, + sink_creator, + context, + std::make_shared(metadata_snapshot->getSampleBlock()) + ); } String path; @@ -2214,6 +2223,7 @@ SinkToStoragePtr StorageFile::write( String new_path; do { + new_path = path.substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : path.substr(pos)); ++index; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c0f25cd528e9..70eaa175a15b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -123,6 +123,7 @@ namespace ErrorCodes extern const int TABLE_IS_READ_ONLY; extern const int TOO_MANY_PARTS; extern const int PART_IS_LOCKED; + extern const int INCOMPATIBLE_COLUMNS; } namespace ActionLocks @@ -222,7 +223,7 @@ void StorageMergeTree::startup() try { background_operations_assignee.start(); - startBackgroundMovesIfNeeded(); + startBackgroundMoves(); startOutdatedAndUnexpectedDataPartsLoadingTask(); } catch (...) @@ -271,6 +272,11 @@ void StorageMergeTree::shutdown(bool) if (deduplication_log) deduplication_log->shutdown(); + + { + std::lock_guard lock(export_manifests_mutex); + export_manifests.clear(); + } } @@ -2957,12 +2963,6 @@ MutationCounters StorageMergeTree::getMutationCounters() const return mutation_counters; } -void StorageMergeTree::startBackgroundMovesIfNeeded() -{ - if (areBackgroundMovesNeeded()) - background_moves_assignee.start(); -} - std::unique_ptr StorageMergeTree::getDefaultSettings() const { return std::make_unique(getContext()->getMergeTreeSettings()); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 9805996f47fc..8136cbe8c94c 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -288,8 +288,6 @@ class StorageMergeTree final : public MergeTreeData std::unique_ptr fillNewPartName(MutableDataPartPtr & part, DataPartsLock & lock); std::unique_ptr fillNewPartNameAndResetLevel(MutableDataPartPtr & part, DataPartsLock & lock); - void startBackgroundMovesIfNeeded() override; - BackupEntries backupMutations(UInt64 version, const String & data_path_in_backup) const; /// Attaches restored parts to the storage. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d88f93e6fd95..9cceae8595ad 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7,6 +7,7 @@ #include #include +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -68,6 +69,7 @@ #include #include #include +#include #include #include #include @@ -117,6 +119,13 @@ #include #include +#include "Functions/generateSnowflakeID.h" +#include "Interpreters/StorageID.h" +#include "QueryPipeline/QueryPlanResourceHolder.h" +#include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" +#include +#include #include #include @@ -186,6 +195,18 @@ namespace Setting extern const SettingsInt64 replication_wait_for_inactive_replica_timeout; extern const SettingsUInt64 select_sequential_consistency; extern const SettingsBool update_sequential_consistency; + extern const SettingsBool allow_experimental_export_merge_tree_part; + extern const SettingsBool export_merge_tree_partition_force_export; + extern const SettingsUInt64 export_merge_tree_partition_max_retries; + extern const SettingsUInt64 export_merge_tree_partition_manifest_ttl; + extern const SettingsBool output_format_parallel_formatting; + extern const SettingsBool output_format_parquet_parallel_encoding; + extern const SettingsMaxThreads max_threads; + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; + extern const SettingsUInt64 export_merge_tree_part_max_bytes_per_file; + extern const SettingsUInt64 export_merge_tree_part_max_rows_per_file; + extern const SettingsBool export_merge_tree_part_throw_on_pending_mutations; + extern const SettingsBool export_merge_tree_part_throw_on_pending_patch_parts; } namespace MergeTreeSetting @@ -291,6 +312,13 @@ namespace ErrorCodes extern const int FAULT_INJECTED; extern const int CANNOT_FORGET_PARTITION; extern const int TIMEOUT_EXCEEDED; + extern const int INVALID_SETTING_VALUE; + extern const int PENDING_MUTATIONS_NOT_ALLOWED; +} + +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; } namespace ActionLocks @@ -419,6 +447,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , merge_strategy_picker(*this) , queue(*this, merge_strategy_picker) , fetcher(*this) + , export_merge_tree_partition_task_entries_by_key(export_merge_tree_partition_task_entries.get()) + , export_merge_tree_partition_task_entries_by_transaction_id(export_merge_tree_partition_task_entries.get()) + , export_merge_tree_partition_task_entries_by_create_time(export_merge_tree_partition_task_entries.get()) , cleanup_thread(*this) , async_block_ids_cache(*this) , part_check_thread(*this) @@ -463,6 +494,31 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// Will be activated by restarting thread. mutations_finalizing_task->deactivate(); + if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + export_merge_tree_partition_manifest_updater = std::make_shared(*this); + + export_merge_tree_partition_task_scheduler = std::make_shared(*this); + + export_merge_tree_partition_updating_task = getContext()->getSchedulePool().createTask( + getStorageID(), getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_updating_task)", [this] { exportMergeTreePartitionUpdatingTask(); }); + + export_merge_tree_partition_updating_task->deactivate(); + + export_merge_tree_partition_status_handling_task = getContext()->getSchedulePool().createTask( + getStorageID(), getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_status_handling_task)", [this] { exportMergeTreePartitionStatusHandlingTask(); }); + + export_merge_tree_partition_status_handling_task->deactivate(); + + export_merge_tree_partition_watch_callback = export_merge_tree_partition_updating_task->getWatchCallback(); + + export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( + getStorageID(), getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_select_task)", [this] { selectPartsToExport(); }); + + export_merge_tree_partition_select_task->deactivate(); + } + + bool has_zookeeper = getContext()->hasZooKeeper() || getContext()->hasAuxiliaryZooKeeper(zookeeper_info.zookeeper_name); if (has_zookeeper) { @@ -890,6 +946,7 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodesAttempt() const futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/last_part", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/failed_parts", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/mutations", String(), zkutil::CreateMode::Persistent)); + futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/exports", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/parallel", String(), zkutil::CreateMode::Persistent)); @@ -1052,6 +1109,8 @@ bool StorageReplicatedMergeTree::createTableIfNotExistsAttempt(const StorageMeta zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/mutations", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/exports", "", + zkutil::CreateMode::Persistent)); /// And create first replica atomically. See also "createReplica" method that is used to create not the first replicas. @@ -4358,6 +4417,160 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() } } +void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() +{ + try + { + export_merge_tree_partition_manifest_updater->poll(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + + export_merge_tree_partition_updating_task->scheduleAfter(30 * 1000); +} + +void StorageReplicatedMergeTree::selectPartsToExport() +{ + try + { + export_merge_tree_partition_task_scheduler->run(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); +} + +void StorageReplicatedMergeTree::exportMergeTreePartitionStatusHandlingTask() +{ + try + { + export_merge_tree_partition_manifest_updater->handleStatusChanges(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } +} + +std::vector StorageReplicatedMergeTree::getPartitionExportsInfo() const +{ + std::vector infos; + + const auto zk = getZooKeeper(); + const auto exports_path = fs::path(zookeeper_path) / "exports"; + std::vector children; + if (Coordination::Error::ZOK != zk->tryGetChildren(exports_path, children)) + { + LOG_INFO(log, "Failed to get children from exports path, returning empty export info list"); + return infos; + } + + for (const auto & child : children) + { + ReplicatedPartitionExportInfo info; + + const auto export_partition_path = fs::path(exports_path) / child; + std::string metadata_json; + if (!zk->tryGet(export_partition_path / "metadata.json", metadata_json)) + { + LOG_INFO(log, "Skipping {}: missing metadata.json", child); + continue; + } + + std::string status; + if (!zk->tryGet(export_partition_path / "status", status)) + { + LOG_INFO(log, "Skipping {}: missing status", child); + continue; + } + + std::vector processing_parts; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_partition_path / "processing", processing_parts)) + { + LOG_INFO(log, "Skipping {}: missing processing parts", child); + continue; + } + + const auto parts_to_do = processing_parts.size(); + + std::string exception_replica; + std::string last_exception; + std::string exception_part; + std::size_t exception_count = 0; + + const auto exceptions_per_replica_path = export_partition_path / "exceptions_per_replica"; + + Strings exception_replicas; + if (Coordination::Error::ZOK != zk->tryGetChildren(exceptions_per_replica_path, exception_replicas)) + { + LOG_INFO(log, "Skipping {}: missing exceptions_per_replica", export_partition_path); + continue; + } + + for (const auto & replica : exception_replicas) + { + std::string exception_count_string; + if (!zk->tryGet(exceptions_per_replica_path / replica / "count", exception_count_string)) + { + LOG_INFO(log, "Skipping {}: missing count", replica); + continue; + } + + exception_count += std::stoull(exception_count_string.c_str()); + + if (last_exception.empty()) + { + const auto last_exception_path = exceptions_per_replica_path / replica / "last_exception"; + std::string last_exception_string; + if (!zk->tryGet(last_exception_path / "exception", last_exception_string)) + { + LOG_INFO(log, "Skipping {}: missing last_exception/exception", last_exception_path); + continue; + } + + std::string exception_part_zk; + if (!zk->tryGet(last_exception_path / "part", exception_part_zk)) + { + LOG_INFO(log, "Skipping {}: missing exception part", last_exception_path); + continue; + } + + exception_replica = replica; + last_exception = last_exception_string; + exception_part = exception_part_zk; + } + } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + info.destination_database = metadata.destination_database; + info.destination_table = metadata.destination_table; + info.partition_id = metadata.partition_id; + info.transaction_id = metadata.transaction_id; + info.query_id = metadata.query_id; + info.create_time = metadata.create_time; + info.source_replica = metadata.source_replica; + info.parts_count = metadata.number_of_parts; + info.parts_to_do = parts_to_do; + info.parts = metadata.parts; + info.status = status; + info.exception_replica = exception_replica; + info.last_exception = last_exception; + info.exception_part = exception_part; + info.exception_count = exception_count; + + infos.emplace_back(std::move(info)); + } + + return infos; +} + StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::createLogEntryToMergeParts( zkutil::ZooKeeperPtr & zookeeper, @@ -5644,7 +5857,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread, const ZooK restarting_thread.start(true); }); - startBackgroundMovesIfNeeded(); + startBackgroundMoves(); part_moves_between_shards_orchestrator.start(); @@ -5743,6 +5956,13 @@ void StorageReplicatedMergeTree::partialShutdown() mutations_updating_task->deactivate(); mutations_finalizing_task->deactivate(); + if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + export_merge_tree_partition_updating_task->deactivate(); + export_merge_tree_partition_select_task->deactivate(); + export_merge_tree_partition_status_handling_task->deactivate(); + } + cleanup_thread.stop(); async_block_ids_cache.stop(); part_check_thread.stop(); @@ -5812,6 +6032,17 @@ void StorageReplicatedMergeTree::shutdown(bool) /// Wait for all of them std::lock_guard lock(data_parts_exchange_ptr->rwlock); } + + { + std::lock_guard lock(export_merge_tree_partition_mutex); + export_merge_tree_partition_task_entries.clear(); + } + + { + std::lock_guard lock(export_manifests_mutex); + export_manifests.clear(); + } + LOG_TRACE(log, "Shutdown finished"); } @@ -7899,6 +8130,221 @@ void StorageReplicatedMergeTree::fetchPartition( LOG_TRACE(log, "Fetch took {} sec. ({} tries)", watch.elapsedSeconds(), try_no); } +void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) +{ + if (!query_context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + } + + const auto dest_database = query_context->resolveDatabase(command.to_database); + const auto dest_table = command.to_table; + const auto dest_storage_id = StorageID(dest_database, dest_table); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, dest_table}, query_context); + + if (dest_storage->getStorageID() == this->getStorageID()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); + } + + if (!dest_storage->supportsImport()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support MergeTree parts or uses unsupported partitioning", dest_storage->getName()); + + auto query_to_string = [] (const ASTPtr & ast) + { + return ast ? ast->formatWithSecretsOneLine() : ""; + }; + + auto src_snapshot = getInMemoryMetadataPtr(); + auto destination_snapshot = dest_storage->getInMemoryMetadataPtr(); + + /// compare all source readable columns with all destination insertable columns + /// this allows us to skip ephemeral columns + if (src_snapshot->getColumns().getReadable().sizeOfDifference(destination_snapshot->getColumns().getInsertable())) + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); + + if (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + + zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly(); + + const String partition_id = getPartitionIDFromQuery(command.partition, query_context); + + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + const auto export_key = partition_id + "_" + dest_storage_id.getQualifiedName().getFullName(); + + const auto partition_exports_path = fs::path(exports_path) / export_key; + + /// check if entry already exists + if (zookeeper->exists(partition_exports_path)) + { + LOG_INFO(log, "Export with key {} is already exported or it is being exported. Checking if it has expired so that we can overwrite it", export_key); + + bool has_expired = false; + + if (zookeeper->exists(fs::path(partition_exports_path) / "metadata.json")) + { + std::string metadata_json; + if (zookeeper->tryGet(fs::path(partition_exports_path) / "metadata.json", metadata_json)) + { + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + const auto now = time(nullptr); + const auto expiration_time = manifest.create_time + manifest.ttl_seconds; + + LOG_INFO(log, "Export with key {} has expiration time {}, now is {}", export_key, expiration_time, now); + + if (static_cast(expiration_time) < now) + { + has_expired = true; + } + } + } + + if (!has_expired && !query_context->getSettingsRef()[Setting::export_merge_tree_partition_force_export]) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Export with key {} already exported or it is being exported, and it has not expired. Set `export_merge_tree_partition_force_export` to overwrite it.", export_key); + } + + LOG_INFO(log, "Overwriting export with key {}", export_key); + + /// Not putting in ops (same transaction) because we can't construct a "tryRemoveRecursive" request. + /// It is possible that the zk being used does not support RemoveRecursive requests. + /// It is ok for this to be non transactional. Worst case scenario an on-going export is going to be killed and a new task won't be scheduled. + zookeeper->tryRemoveRecursive(partition_exports_path); + } + + Coordination::Requests ops; + + ops.emplace_back(zkutil::makeCreateRequest(partition_exports_path, "", zkutil::CreateMode::Persistent)); + + DataPartsVector parts; + + { + auto data_parts_lock = lockParts(); + parts = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, partition_id, data_parts_lock); + } + + if (parts.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} doesn't exist", partition_id); + } + + const bool throw_on_pending_mutations = query_context->getSettingsRef()[Setting::export_merge_tree_part_throw_on_pending_mutations]; + const bool throw_on_pending_patch_parts = query_context->getSettingsRef()[Setting::export_merge_tree_part_throw_on_pending_patch_parts]; + + MergeTreeData::IMutationsSnapshot::Params mutations_snapshot_params + { + .metadata_version = getInMemoryMetadataPtr()->getMetadataVersion(), + .min_part_metadata_version = MergeTreeData::getMinMetadataVersion(parts), + .need_data_mutations = throw_on_pending_mutations, + .need_alter_mutations = throw_on_pending_mutations || throw_on_pending_patch_parts, + .need_patch_parts = throw_on_pending_patch_parts, + }; + + const auto mutations_snapshot = getMutationsSnapshot(mutations_snapshot_params); + + std::vector part_names; + for (const auto & part : parts) + { + const auto alter_conversions = getAlterConversionsForPart(part, mutations_snapshot, query_context); + + /// re-check `throw_on_pending_mutations` because `pending_mutations` might have been filled due to `throw_on_pending_patch_parts` + if (alter_conversions->hasMutations() && throw_on_pending_mutations) + { + throw Exception(ErrorCodes::PENDING_MUTATIONS_NOT_ALLOWED, + "Partition {} can not be exported because the part {} has pending mutations. Either wait for the mutations to be applied or set `export_merge_tree_part_throw_on_pending_mutations` to false", + partition_id, + part->name); + } + + if (alter_conversions->hasPatches()) + { + throw Exception(ErrorCodes::PENDING_MUTATIONS_NOT_ALLOWED, + "Partition {} can not be exported because the part {} has pending patch parts. Either wait for the patch parts to be applied or set `export_merge_tree_part_throw_on_pending_patch_parts` to false", + partition_id, + part->name); + } + + part_names.push_back(part->name); + } + + /// TODO arthur somehow check if the list of parts is updated "enough" + + ExportReplicatedMergeTreePartitionManifest manifest; + + manifest.transaction_id = generateSnowflakeIDString(); + manifest.query_id = query_context->getCurrentQueryId(); + manifest.partition_id = partition_id; + manifest.destination_database = dest_database; + manifest.destination_table = dest_table; + manifest.source_replica = replica_name; + manifest.number_of_parts = part_names.size(); + manifest.parts = part_names; + manifest.create_time = time(nullptr); + manifest.max_retries = query_context->getSettingsRef()[Setting::export_merge_tree_partition_max_retries]; + manifest.ttl_seconds = query_context->getSettingsRef()[Setting::export_merge_tree_partition_manifest_ttl]; + manifest.max_threads = query_context->getSettingsRef()[Setting::max_threads]; + manifest.parallel_formatting = query_context->getSettingsRef()[Setting::output_format_parallel_formatting]; + manifest.parquet_parallel_encoding = query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding]; + manifest.max_bytes_per_file = query_context->getSettingsRef()[Setting::export_merge_tree_part_max_bytes_per_file]; + manifest.max_rows_per_file = query_context->getSettingsRef()[Setting::export_merge_tree_part_max_rows_per_file]; + + manifest.file_already_exists_policy = query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value; + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "metadata.json", + manifest.toJsonString(), + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "exceptions_per_replica", + "", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing", + "", + zkutil::CreateMode::Persistent)); + + for (const auto & part : part_names) + { + ExportReplicatedMergeTreePartitionProcessingPartEntry entry; + entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::PENDING; + entry.part_name = part; + entry.retry_count = 0; + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing" / part, + entry.toJsonString(), + zkutil::CreateMode::Persistent)); + } + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processed", + "", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "locks", + "", + zkutil::CreateMode::Persistent)); + + /// status: IN_PROGRESS, COMPLETED, FAILED + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "status", + "PENDING", + zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + Coordination::Error code = zookeeper->tryMulti(ops, responses); + + if (code != Coordination::Error::ZOK) + throw zkutil::KeeperException::fromPath(code, partition_exports_path); +} + void StorageReplicatedMergeTree::forgetPartition(const ASTPtr & partition, ContextPtr query_context) { @@ -9330,6 +9776,89 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta return part_moves_between_shards_orchestrator.killPartMoveToShard(task_uuid); } +CancellationCode StorageReplicatedMergeTree::killExportPartition(const String & transaction_id) +{ + auto try_set_status_to_killed = [this](const zkutil::ZooKeeperPtr & zk, const std::string & status_path) + { + Coordination::Stat stat; + std::string status_from_zk_string; + + if (!zk->tryGet(status_path, status_from_zk_string, &stat)) + { + /// found entry locally, but not in zk. It might have been deleted by another replica and we did not have time to update the local entry. + LOG_INFO(log, "Export partition task not found in zk, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + const auto status_from_zk = magic_enum::enum_cast(status_from_zk_string); + + if (!status_from_zk) + { + LOG_INFO(log, "Export partition task status is invalid, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + if (status_from_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "Export partition task is {}, can not cancel it", String(magic_enum::enum_name(status_from_zk.value()))); + return CancellationCode::CancelCannotBeSent; + } + + if (zk->trySet(status_path, String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED)), stat.version) != Coordination::Error::ZOK) + { + LOG_INFO(log, "Status has been updated while trying to kill the export partition task, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + return CancellationCode::CancelSent; + }; + + std::lock_guard lock(export_merge_tree_partition_mutex); + + const auto zk = getZooKeeper(); + + /// if we have the entry locally, no need to list from zk. we can save some requests. + const auto & entry = export_merge_tree_partition_task_entries_by_transaction_id.find(transaction_id); + if (entry != export_merge_tree_partition_task_entries_by_transaction_id.end()) + { + LOG_INFO(log, "Export partition task found locally, trying to cancel it"); + /// found locally, no need to get children on zk + if (entry->status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "Export partition task is not pending, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + return try_set_status_to_killed(zk, fs::path(zookeeper_path) / "exports" / entry->getCompositeKey() / "status"); + } + else + { + LOG_INFO(log, "Export partition task not found locally, trying to find it on zk"); + /// for some reason, we don't have the entry locally. ls on zk to find the entry + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + const auto export_keys = zk->getChildren(exports_path); + String export_key_to_be_cancelled; + + for (const auto & export_key : export_keys) + { + std::string metadata_json; + if (!zk->tryGet(fs::path(exports_path) / export_key / "metadata.json", metadata_json)) + continue; + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + if (manifest.transaction_id == transaction_id) + { + LOG_INFO(log, "Export partition task found on zk, trying to cancel it"); + return try_set_status_to_killed(zk, fs::path(exports_path) / export_key / "status"); + } + } + } + + LOG_INFO(log, "Export partition task not found, can not cancel it"); + + return CancellationCode::NotFound; +} + void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, const DataPartPtr & part, @@ -9888,13 +10417,6 @@ MutationCounters StorageReplicatedMergeTree::getMutationCounters() const return queue.getMutationCounters(); } -void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() -{ - if (areBackgroundMovesNeeded()) - background_moves_assignee.start(); -} - - std::unique_ptr StorageReplicatedMergeTree::getDefaultSettings() const { return std::make_unique(getContext()->getReplicatedMergeTreeSettings()); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index b600e15f4807..493167ffcbad 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -13,6 +13,10 @@ #include #include #include +#include "Interpreters/CancellationCode.h" +#include "Storages/MergeTree/ExportPartitionManifestUpdatingTask.h" +#include "Storages/MergeTree/ExportPartitionTaskScheduler.h" +#include #include #include #include @@ -97,6 +101,8 @@ namespace DB class ZooKeeperWithFaultInjection; using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; +struct ReplicatedPartitionExportInfo; + class StorageReplicatedMergeTree final : public MergeTreeData { public: @@ -372,6 +378,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData using ShutdownDeadline = std::chrono::time_point; void waitForUniquePartsToBeFetchedByOtherReplicas(ShutdownDeadline shutdown_deadline); + std::vector getPartitionExportsInfo() const; + private: std::atomic_bool are_restoring_replica {false}; @@ -396,6 +404,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData friend class MergeFromLogEntryTask; friend class MutateFromLogEntryTask; friend class ReplicatedMergeMutateTaskBase; + friend class ExportPartitionManifestUpdatingTask; + friend class ExportPartitionTaskScheduler; using MergeStrategyPicker = ReplicatedMergeTreeMergeStrategyPicker; using LogEntry = ReplicatedMergeTreeLogEntry; @@ -508,6 +518,26 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// A task that marks finished mutations as done. BackgroundSchedulePoolTaskHolder mutations_finalizing_task; + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_updating_task; + + /// mostly handle kill operations + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_status_handling_task; + std::shared_ptr export_merge_tree_partition_manifest_updater; + + std::shared_ptr export_merge_tree_partition_task_scheduler; + + Coordination::WatchCallbackPtr export_merge_tree_partition_watch_callback; + + std::mutex export_merge_tree_partition_mutex; + + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_select_task; + + ExportPartitionTaskEntriesContainer export_merge_tree_partition_task_entries; + + // Convenience references to indexes + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_key; + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_transaction_id; + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_create_time; /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; @@ -735,6 +765,14 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// Checks if some mutations are done and marks them as done. void mutationsFinalizingTask(); + void selectPartsToExport(); + + /// update in-memory list of partition exports + void exportMergeTreePartitionUpdatingTask(); + + /// handle status changes for export partition tasks + void exportMergeTreePartitionStatusHandlingTask(); + /** Write the selected parts to merge into the log, * Call when merge_selecting_mutex is locked. * Returns false if any part is not in ZK. @@ -917,6 +955,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context) override; CancellationCode killPartMoveToShard(const UUID & task_uuid) override; + CancellationCode killExportPartition(const String & transaction_id) override; void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, @@ -924,7 +963,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData bool fetch_part, ContextPtr query_context) override; void forgetPartition(const ASTPtr & partition, ContextPtr query_context) override; - + + void exportPartitionToTable(const PartitionCommand &, ContextPtr) override; /// NOTE: there are no guarantees for concurrent merges. Dropping part can /// be concurrently merged into some covering part and dropPart will do @@ -956,8 +996,6 @@ class StorageReplicatedMergeTree final : public MergeTreeData MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; - void startBackgroundMovesIfNeeded() override; - /// Attaches restored parts to the storage. void attachRestoredParts(MutableDataPartsVector && parts) override; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9a177b4894f1..7e052ec9cabc 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -732,7 +732,7 @@ void StorageURLSink::cancelBuffers() write_buf->cancel(); } -class PartitionedStorageURLSink : public PartitionedSink +class PartitionedStorageURLSink : public PartitionedSink::SinkCreator { public: PartitionedStorageURLSink( @@ -746,7 +746,7 @@ class PartitionedStorageURLSink : public PartitionedSink const CompressionMethod compression_method_, const HTTPHeaderEntries & headers_, const String & http_method_) - : PartitionedSink(partition_strategy_, context_, std::make_shared(sample_block_)) + : partition_strategy(partition_strategy_) , uri(uri_) , format(format_) , format_settings(format_settings_) @@ -761,7 +761,8 @@ class PartitionedStorageURLSink : public PartitionedSink SinkPtr createSinkForPartition(const String & partition_id) override { - std::string partition_path = partition_strategy->getPathForWrite(uri, partition_id); + const auto file_path_generator = std::make_shared(uri); + std::string partition_path = file_path_generator->getPathForWrite(partition_id); context->getRemoteHostFilter().checkURL(Poco::URI(partition_path)); return std::make_shared( @@ -769,6 +770,7 @@ class PartitionedStorageURLSink : public PartitionedSink } private: + std::shared_ptr partition_strategy; const String uri; const String format; const std::optional format_settings; @@ -1437,7 +1439,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad has_wildcards, /* partition_columns_in_data_file */true); - return std::make_shared( + auto sink_creator = std::make_shared( partition_strategy, uri, format_name, @@ -1448,6 +1450,8 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad compression_method, headers, http_method); + + return std::make_shared(partition_strategy, sink_creator, context, std::make_shared(metadata_snapshot->getSampleBlock())); } return std::make_shared( diff --git a/src/Storages/System/StorageSystemExports.cpp b/src/Storages/System/StorageSystemExports.cpp new file mode 100644 index 000000000000..fad8b43d0d2f --- /dev/null +++ b/src/Storages/System/StorageSystemExports.cpp @@ -0,0 +1,72 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemExports::getColumnsDescription() +{ + return ColumnsDescription + { + {"source_database", std::make_shared(), "Name of the source database."}, + {"source_table", std::make_shared(), "Name of the source table."}, + {"destination_database", std::make_shared(), "Name of the destination database."}, + {"destination_table", std::make_shared(), "Name of the destination table."}, + {"create_time", std::make_shared(), "Date and time when the export command was received in the server."}, + {"part_name", std::make_shared(), "Name of the part"}, + {"query_id", std::make_shared(), "Query ID of the export operation."}, + {"destination_file_paths", std::make_shared(std::make_shared()), "File paths where the part is being exported."}, + {"elapsed", std::make_shared(), "The time elapsed (in seconds) since the export started."}, + {"rows_read", std::make_shared(), "The number of rows read from the exported part."}, + {"total_rows_to_read", std::make_shared(), "The total number of rows to read from the exported part."}, + {"total_size_bytes_compressed", std::make_shared(), "The total size of the compressed data in the exported part."}, + {"total_size_bytes_uncompressed", std::make_shared(), "The total size of the uncompressed data in the exported part."}, + {"bytes_read_uncompressed", std::make_shared(), "The number of uncompressed bytes read from the exported part."}, + {"memory_usage", std::make_shared(), "Current memory usage in bytes for the export operation."}, + {"peak_memory_usage", std::make_shared(), "Peak memory usage in bytes during the export operation."}, + }; +} + +void StorageSystemExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto access = context->getAccess(); + const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); + + for (const auto & export_info : context->getExportsList().get()) + { + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, export_info.source_database, export_info.source_table)) + continue; + + size_t i = 0; + res_columns[i++]->insert(export_info.source_database); + res_columns[i++]->insert(export_info.source_table); + res_columns[i++]->insert(export_info.destination_database); + res_columns[i++]->insert(export_info.destination_table); + res_columns[i++]->insert(export_info.create_time); + res_columns[i++]->insert(export_info.part_name); + res_columns[i++]->insert(export_info.query_id); + Array destination_file_paths_array; + destination_file_paths_array.reserve(export_info.destination_file_paths.size()); + for (const auto & file_path : export_info.destination_file_paths) + destination_file_paths_array.push_back(file_path); + res_columns[i++]->insert(destination_file_paths_array); + res_columns[i++]->insert(export_info.elapsed); + res_columns[i++]->insert(export_info.rows_read); + res_columns[i++]->insert(export_info.total_rows_to_read); + res_columns[i++]->insert(export_info.total_size_bytes_compressed); + res_columns[i++]->insert(export_info.total_size_bytes_uncompressed); + res_columns[i++]->insert(export_info.bytes_read_uncompressed); + res_columns[i++]->insert(export_info.memory_usage); + res_columns[i++]->insert(export_info.peak_memory_usage); + } +} + +} diff --git a/src/Storages/System/StorageSystemExports.h b/src/Storages/System/StorageSystemExports.h new file mode 100644 index 000000000000..e13fbfa26aaa --- /dev/null +++ b/src/Storages/System/StorageSystemExports.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +class StorageSystemExports final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemExports"; } + + static ColumnsDescription getColumnsDescription(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 0fca5dc84a2b..c8c569ff4696 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -1,5 +1,5 @@ -#include #include +#include #include #include diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp new file mode 100644 index 000000000000..ed4b2bc9f4c2 --- /dev/null +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp @@ -0,0 +1,145 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Columns/ColumnString.h" +#include "Storages/VirtualColumnUtils.h" + + +namespace DB +{ + +ColumnsDescription StorageSystemReplicatedPartitionExports::getColumnsDescription() +{ + return ColumnsDescription + { + {"source_database", std::make_shared(), "Name of the source database."}, + {"source_table", std::make_shared(), "Name of the source table."}, + {"destination_database", std::make_shared(), "Name of the destination database."}, + {"destination_table", std::make_shared(), "Name of the destination table."}, + {"create_time", std::make_shared(), "Date and time when the export command was submitted"}, + {"partition_id", std::make_shared(), "ID of the partition"}, + {"transaction_id", std::make_shared(), "ID of the transaction."}, + {"query_id", std::make_shared(), "Query ID of the export operation."}, + {"source_replica", std::make_shared(), "Name of the source replica."}, + {"parts", std::make_shared(std::make_shared()), "List of part names to be exported."}, + {"parts_count", std::make_shared(), "Number of parts in the export."}, + {"parts_to_do", std::make_shared(), "Number of parts pending to be exported."}, + {"status", std::make_shared(), "Status of the export."}, + {"exception_replica", std::make_shared(), "Replica that caused the last exception"}, + {"last_exception", std::make_shared(), "Last exception message of any part (not necessarily the last global exception)"}, + {"exception_part", std::make_shared(), "Part that caused the last exception"}, + {"exception_count", std::make_shared(), "Number of global exceptions"}, + }; +} + +void StorageSystemReplicatedPartitionExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const +{ + const auto access = context->getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + std::map> replicated_merge_tree_tables; + for (const auto & db : DatabaseCatalog::instance().getDatabases(GetDatabasesOptions{.with_datalake_catalogs = false})) + { + /// skip data lakes + if (db.second->isExternal()) + continue; + + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); + + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + const auto & table = iterator->table(); + if (!table) + continue; + + StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); + if (!table_replicated) + continue; + + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + continue; + + replicated_merge_tree_tables[db.first][iterator->name()] = table; + } + } + + MutableColumnPtr col_database_mut = ColumnString::create(); + MutableColumnPtr col_table_mut = ColumnString::create(); + + for (auto & db : replicated_merge_tree_tables) + { + for (auto & table : db.second) + { + col_database_mut->insert(db.first); + col_table_mut->insert(table.first); + } + } + + ColumnPtr col_database = std::move(col_database_mut); + ColumnPtr col_table = std::move(col_table_mut); + + /// Determine what tables are needed by the conditions in the query. + { + Block filtered_block + { + { col_database, std::make_shared(), "database" }, + { col_table, std::make_shared(), "table" }, + }; + + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); + + if (!filtered_block.rows()) + return; + + col_database = filtered_block.getByName("database").column; + col_table = filtered_block.getByName("table").column; + } + + for (size_t i_storage = 0; i_storage < col_database->size(); ++i_storage) + { + const auto database = (*col_database)[i_storage].safeGet(); + const auto table = (*col_table)[i_storage].safeGet(); + + std::vector partition_exports_info; + { + const IStorage * storage = replicated_merge_tree_tables[database][table].get(); + if (const auto * replicated_merge_tree = dynamic_cast(storage)) + partition_exports_info = replicated_merge_tree->getPartitionExportsInfo(); + } + + for (const ReplicatedPartitionExportInfo & info : partition_exports_info) + { + std::size_t i = 0; + res_columns[i++]->insert(database); + res_columns[i++]->insert(table); + res_columns[i++]->insert(info.destination_database); + res_columns[i++]->insert(info.destination_table); + res_columns[i++]->insert(info.create_time); + res_columns[i++]->insert(info.partition_id); + res_columns[i++]->insert(info.transaction_id); + res_columns[i++]->insert(info.query_id); + res_columns[i++]->insert(info.source_replica); + Array parts_array; + parts_array.reserve(info.parts.size()); + for (const auto & part : info.parts) + parts_array.push_back(part); + res_columns[i++]->insert(parts_array); + res_columns[i++]->insert(info.parts_count); + res_columns[i++]->insert(info.parts_to_do); + res_columns[i++]->insert(info.status); + res_columns[i++]->insert(info.exception_replica); + res_columns[i++]->insert(info.last_exception); + res_columns[i++]->insert(info.exception_part); + res_columns[i++]->insert(info.exception_count); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.h b/src/Storages/System/StorageSystemReplicatedPartitionExports.h new file mode 100644 index 000000000000..d57844cbb34e --- /dev/null +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.h @@ -0,0 +1,43 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +struct ReplicatedPartitionExportInfo +{ + String destination_database; + String destination_table; + String partition_id; + String transaction_id; + String query_id; + time_t create_time; + String source_replica; + size_t parts_count; + size_t parts_to_do; + std::vector parts; + String status; + std::string exception_replica; + std::string last_exception; + std::string exception_part; + size_t exception_count; +}; + +class StorageSystemReplicatedPartitionExports final : public IStorageSystemOneBlock +{ +public: + + std::string getName() const override { return "SystemReplicatedPartitionExports"; } + + static ColumnsDescription getColumnsDescription(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 309b4846300e..9eadf7ee5f1b 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -1,10 +1,11 @@ #include +#include #include "config.h" #include #include #include - +#include #include #include #include @@ -32,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -110,6 +112,7 @@ #if USE_ICU # include #endif +#include #include #include @@ -134,6 +137,11 @@ namespace DB { +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) { attachNoDescription(context, system_database, "one", "This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`."); @@ -218,6 +226,11 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "dimensional_metrics", "Contains dimensional metrics, which have multiple dimensions (labels) to provide more granular information. For example, counting failed merges by their error code. This table is always up to date."); attach(context, system_database, "merges", "Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row."); attach(context, system_database, "moves", "Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row."); + attach(context, system_database, "exports", "Contains a list of exports currently executing exports of MergeTree tables and their progress. Each export operation is represented by a single row."); + if (context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + attach(context, system_database, "replicated_partition_exports", "Contains a list of partition exports of ReplicatedMergeTree tables and their progress. Each export operation is represented by a single row."); + } attach(context, system_database, "mutations", "Contains a list of mutations and their progress. Each mutation command is represented by a single row."); attachNoDescription(context, system_database, "replicas", "Contains information and status of all table replicas on current server. Each replica is represented by a single row."); attachNoDescription(context, system_database, "database_replicas", "Contains information and status of all database replicas on current server. Each database replica is represented by a single row."); diff --git a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml new file mode 100644 index 000000000000..72014c9de4db --- /dev/null +++ b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/config/install.sh b/tests/config/install.sh index 04344af26669..ae3b5ec636a7 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -88,6 +88,7 @@ ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_catalog_drop_table_concurrency.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_access_control_improvements.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/enable_experimental_export_merge_tree_partition.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/__init__.py b/tests/integration/test_export_merge_tree_part_to_object_storage/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/configs/named_collections.xml b/tests/integration/test_export_merge_tree_part_to_object_storage/configs/named_collections.xml new file mode 100644 index 000000000000..d46920b7ba88 --- /dev/null +++ b/tests/integration/test_export_merge_tree_part_to_object_storage/configs/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/data + minio + ClickHouse_Minio_P@ssw0rd + + + \ No newline at end of file diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/test.py b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py new file mode 100644 index 000000000000..36a2035d0f12 --- /dev/null +++ b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py @@ -0,0 +1,284 @@ +import logging +import pytest +import random +import string +import time +from typing import Optional +import uuid + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node1", + main_configs=["configs/named_collections.xml"], + with_minio=True, + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_s3_table(node, s3_table): + node.query(f"CREATE TABLE {s3_table} (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') PARTITION BY year") + + +def create_tables_and_insert_data(node, mt_table, s3_table): + node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple() SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1") + node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + + create_s3_table(node, s3_table) + + +def test_drop_column_during_export_snapshot(cluster): + node = cluster.instances["node1"] + + mt_table = "mutations_snapshot_mt_table" + s3_table = "mutations_snapshot_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + # Ensure export sees a consistent snapshot at start time even if we mutate the source later + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + # Start export of 2020 + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table};" + ) + + # Drop a column that is required for the export + node.query(f"ALTER TABLE {mt_table} DROP COLUMN id") + + time.sleep(3) + # assert the mutation has been applied AND the data has not been exported yet + assert "Unknown expression identifier `id`" in node.query_and_get_error(f"SELECT id FROM {mt_table}"), "Column id is not removed" + + # Wait for export to finish and then verify destination still reflects the original snapshot (3 rows) + time.sleep(5) + assert node.query(f"SELECT count() FROM {s3_table} WHERE id >= 0") == '3\n', "Export did not preserve snapshot at start time after source mutation" + + +def test_add_column_during_export(cluster): + node = cluster.instances["node1"] + + mt_table = "add_column_during_export_mt_table" + s3_table = "add_column_during_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + # Ensure export sees a consistent snapshot at start time even if we mutate the source later + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + # Start export of 2020 + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table};" + ) + + node.query(f"ALTER TABLE {mt_table} ADD COLUMN id2 UInt64") + + time.sleep(3) + + # assert the mutation has been applied AND the data has not been exported yet + assert node.query(f"SELECT count(id2) FROM {mt_table}") == '4\n', "Column id2 is not added" + + # Wait for export to finish and then verify destination still reflects the original snapshot (3 rows) + time.sleep(5) + assert node.query(f"SELECT count() FROM {s3_table} WHERE id >= 0") == '3\n', "Export did not preserve snapshot at start time after source mutation" + assert "Unknown expression identifier `id2`" in node.query_and_get_error(f"SELECT id2 FROM {s3_table}"), "Column id2 is present in the exported data" + + +def test_pending_mutations_throw_before_export(cluster): + """Test that pending mutations before export throw an error with default settings.""" + node = cluster.instances["node1"] + + mt_table = "pending_mutations_throw_mt_table" + s3_table = "pending_mutations_throw_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0") + assert mutations.strip() != '0', "Mutation should be pending" + + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table} SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error, f"Expected error about pending mutations, got: {error}" + + +def test_pending_mutations_skip_before_export(cluster): + """Test that pending mutations before export are skipped with throw_on_pending_mutations=false.""" + node = cluster.instances["node1"] + + mt_table = "pending_mutations_skip_mt_table" + s3_table = "pending_mutations_skip_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0") + assert mutations.strip() != '0', "Mutation should be pending" + + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=false" + ) + + time.sleep(5) + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "101" not in result and "102" not in result and "103" not in result, \ + "Export should contain original data before mutation" + assert "1\n2\n3" in result, "Export should contain original data" + + +def test_data_mutations_after_export_started(cluster): + """Test that mutations applied after export starts don't affect the exported data.""" + node = cluster.instances["node1"] + + mt_table = "mutations_after_export_mt_table" + s3_table = "mutations_after_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + # Block traffic to MinIO to delay export + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + time.sleep(5) + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before mutation" + assert "101" not in result, "Export should not contain mutated data" + + +def test_pending_patch_parts_throw_before_export(cluster): + """Test that pending patch parts before export throw an error with default settings.""" + node = cluster.instances["node1"] + + mt_table = "pending_patches_throw_mt_table" + s3_table = "pending_patches_throw_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table}" + ) + + node.query(f"DROP TABLE {mt_table}") + + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error or "pending patch parts" in error.lower(), \ + f"Expected error about pending patch parts, got: {error}" + + +def test_pending_patch_parts_skip_before_export(cluster): + """Test that pending patch parts before export are skipped with throw_on_pending_patch_parts=false.""" + node = cluster.instances["node1"] + + mt_table = "pending_patches_skip_mt_table" + s3_table = "pending_patches_skip_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_patch_parts=false" + ) + + time.sleep(5) + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before patch" + + node.query(f"DROP TABLE {mt_table}") diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml new file mode 100644 index 000000000000..f8c5fab1a3be --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 1 + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml new file mode 100644 index 000000000000..ba6508ebd660 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 0 + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml new file mode 100644 index 000000000000..d46920b7ba88 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/data + minio + ClickHouse_Minio_P@ssw0rd + + + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml new file mode 100644 index 000000000000..518f29708929 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml @@ -0,0 +1,8 @@ + + + + 3 + + + + diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py new file mode 100644 index 000000000000..f229292c4ae3 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -0,0 +1,1074 @@ +import logging +import pytest +import random +import string +import time +from typing import Optional +import uuid + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + + +def wait_for_export_status( + node, + mt_table: str, + s3_table: str, + partition_id: str, + expected_status: str = "COMPLETED", + timeout: int = 30, + poll_interval: float = 0.5, +): + start_time = time.time() + last_status = None + while time.time() - start_time < timeout: + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '{partition_id}' + """ + ).strip() + + last_status = status + + if status and status == expected_status: + return status + + time.sleep(poll_interval) + + raise TimeoutError( + f"Export status did not reach '{expected_status}' within {timeout}s. Last status: '{last_status}'") + + +def wait_for_export_to_start( + node, + mt_table: str, + s3_table: str, + partition_id: str, + timeout: int = 10, + poll_interval: float = 0.2, +): + start_time = time.time() + while time.time() - start_time < timeout: + count = node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '{partition_id}' + """ + ).strip() + + if count != '0': + return True + + time.sleep(poll_interval) + + raise TimeoutError(f"Export did not start within {timeout}s. ") + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "replica1", + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + cluster.add_instance( + "replica2", + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + # node that does not participate in the export, but will have visibility over the s3 table + cluster.add_instance( + "watcher_node", + main_configs=["configs/named_collections.xml"], + user_configs=[], + with_minio=True, + ) + cluster.add_instance( + "replica_with_export_disabled", + main_configs=["configs/named_collections.xml", "configs/disable_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_s3_table(node, s3_table): + node.query(f"CREATE TABLE {s3_table} (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') PARTITION BY year") + + +def create_tables_and_insert_data(node, mt_table, s3_table, replica_name): + node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{mt_table}', '{replica_name}') PARTITION BY year ORDER BY tuple() SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1") + node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + + create_s3_table(node, s3_table) + + +def test_restart_nodes_during_export(cluster): + node = cluster.instances["replica1"] + node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "disaster_mt_table" + s3_table = "disaster_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + create_s3_table(watcher_node, s3_table) + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses_node1 = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses_node1) + + pm_rule_reject_responses_node2 = { + "instance": node2, + "destination": node2.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses_node2) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests_node1 = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests_node1) + + pm_rule_reject_requests_node2 = { + "instance": node2, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests_node2) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + """ + + node.query(export_queries) + + # wait for the exports to start + wait_for_export_to_start(node, mt_table, s3_table, "2020") + wait_for_export_to_start(node, mt_table, s3_table, "2021") + + node.stop_clickhouse(kill=True) + node2.stop_clickhouse(kill=True) + + assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2020") == '0\n', "Partition 2020 was written to S3 during network delay crash" + + assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2021") == '0\n', "Partition 2021 was written to S3 during network delay crash" + + # start the nodes, they should finish the export + node.start_clickhouse() + node2.start_clickhouse() + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") + + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") != f'0\n', "Export of partition 2020 did not resume after crash" + + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") != f'0\n', "Export of partition 2021 did not resume after crash" + + +def test_kill_export(cluster): + node = cluster.instances["replica1"] + node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "kill_export_mt_table" + s3_table = "kill_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + # Block responses from MinIO for node2 + pm_rule_reject_responses_node2 = { + "instance": node2, + "destination": node2.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses_node2) + + # Block requests to MinIO from node2 + pm_rule_reject_requests_node2 = { + "instance": node2, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests_node2) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + """ + + node.query(export_queries) + + # Kill only 2020 while S3 is blocked - retry mechanism keeps exports alive + # ZooKeeper operations (KILL) proceed quickly since only S3 is blocked + node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") + + # wait for 2021 to finish + wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") + + # checking for the commit file because maybe the data file was too fast? + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '0\n', "Partition 2020 was written to S3, it was not killed as expected" + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2021_*', format=LineAsString)") != f'0\n', "Partition 2021 was not written to S3, but it should have been" + + # check system.replicated_partition_exports for the export, status should be KILLED + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'KILLED\n', "Partition 2020 was not killed as expected" + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2021' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'COMPLETED\n', "Partition 2021 was not completed, this is unexpected" + + +def test_drop_source_table_during_export(cluster): + node = cluster.instances["replica1"] + # node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "drop_source_table_during_export_mt_table" + s3_table = "drop_source_table_during_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + # create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + create_s3_table(watcher_node, s3_table) + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table}; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table}; + """ + + node.query(export_queries) + + # This should kill the background operations and drop the table + node.query(f"DROP TABLE {mt_table}") + + # Sleep some time to let the export finish (assuming it was not properly cancelled) + time.sleep(10) + + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") == '0\n', "Background operations completed even with the table dropped" + + +def test_concurrent_exports_to_different_targets(cluster): + node = cluster.instances["replica1"] + + mt_table = "concurrent_diff_targets_mt_table" + s3_table_a = "concurrent_diff_targets_s3_a" + s3_table_b = "concurrent_diff_targets_s3_b" + + create_tables_and_insert_data(node, mt_table, s3_table_a, "replica1") + create_s3_table(node, s3_table_b) + + # Launch two exports of the same partition to two different S3 tables concurrently + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=1000) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_a}" + ) + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_b}" + ) + + wait_for_export_status(node, mt_table, s3_table_a, "2020", "COMPLETED") + wait_for_export_status(node, mt_table, s3_table_b, "2020", "COMPLETED") + + # Both targets should receive the same data independently + assert node.query(f"SELECT count() FROM {s3_table_a} WHERE year = 2020") == '3\n', "First target did not receive expected rows" + assert node.query(f"SELECT count() FROM {s3_table_b} WHERE year = 2020") == '3\n', "Second target did not receive expected rows" + + # And both should have a commit marker + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table_a}/commit_2020_*', format=LineAsString)" + ) != '0\n', "Commit file missing for first target" + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table_b}/commit_2020_*', format=LineAsString)" + ) != '0\n', "Commit file missing for second target" + + +def test_failure_is_logged_in_system_table(cluster): + node = cluster.instances["replica1"] + + mt_table = "failure_is_logged_in_system_table_mt_table" + s3_table = "failure_is_logged_in_system_table_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Also block requests to MinIO (destination: MinIO, destination_port: 9001) with REJECT to fail fast + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=1;" + ) + + # Wait so that the export fails + wait_for_export_status(node, mt_table, s3_table, "2020", "FAILED", timeout=10) + + # Network restored; verify the export is marked as FAILED in the system table + # Also verify we captured at least one exception and no commit file exists + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + + assert status.strip() == "FAILED", f"Expected FAILED status, got: {status!r}" + + exception_count = node.query( + f""" + SELECT any(exception_count) FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert int(exception_count.strip()) > 0, "Expected non-zero exception_count in system.replicated_partition_exports" + + # No commit should have been produced for this partition + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)" + ) == '0\n', "Commit file exists despite forced S3 failures" + + +def test_inject_short_living_failures(cluster): + node = cluster.instances["replica1"] + + mt_table = "inject_short_living_failures_mt_table" + s3_table = "inject_short_living_failures_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Also block requests to MinIO (destination: MinIO, destination_port: 9001) with REJECT to fail fast + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + # set big max_retries so that the export does not fail completely + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=100;" + ) + + # wait only for a second to get at least one failure, but not enough to finish the export + time.sleep(5) + + # wait for the export to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # Assert the export succeeded + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not succeed" + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + exception_count = node.query( + f""" + SELECT exception_count FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert int(exception_count.strip()) >= 1, "Expected at least one exception" + + +def test_export_ttl(cluster): + node = cluster.instances["replica1"] + + mt_table = "export_ttl_mt_table" + s3_table = "export_ttl_s3_table" + + expiration_time = 3 + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # start export + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_manifest_ttl={expiration_time};") + + # assert that I get an error when trying to export the same partition again, query_and_get_error + error = node.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};") + assert "Export with key" in error, "Expected error about expired export" + + # wait for the export to finish and for the manifest to expire + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + time.sleep(expiration_time * 2) + + # assert that the export succeeded, check the commit file + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # start export again + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}") + + # wait for the export to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # assert that the export succeeded, check the commit file + # there should be two commit files now, one for the first export and one for the second export + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '2\n', "Export did not succeed" + + +def test_export_partition_file_already_exists_policy(cluster): + node = cluster.instances["replica1"] + + mt_table = "export_partition_file_already_exists_policy_mt_table" + s3_table = "export_partition_file_already_exists_policy_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # stop merges so part names remain stable. it is important for the test. + node.query(f"SYSTEM STOP MERGES {mt_table}") + + # Export all parts + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + ) + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + # wait for the exports to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # try to export the partition + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1" + ) + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'COMPLETED' + """ + ) == '1\n', "Expected the export to be marked as COMPLETED" + + # overwrite policy + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='overwrite'" + ) + + # wait for the export to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # check system.replicated_partition_exports for the export + # ideally we would make sure the transaction id is different, but I do not have the time to do that now + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'COMPLETED' + """ + ) == '1\n', "Expected the export to be marked as COMPLETED" + + # last but not least, let's try with the error policy + # max retries = 1 so it fails fast + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='error', export_merge_tree_partition_max_retries=1", + ) + + # wait for the export to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "FAILED") + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'FAILED' + """ + ) == '1\n', "Expected the export to be marked as FAILED" + + +def test_export_partition_feature_is_disabled(cluster): + replica_with_export_disabled = cluster.instances["replica_with_export_disabled"] + + mt_table = "export_partition_feature_is_disabled_mt_table" + s3_table = "export_partition_feature_is_disabled_s3_table" + + create_tables_and_insert_data(replica_with_export_disabled, mt_table, s3_table, "replica1") + + error = replica_with_export_disabled.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};") + assert "experimental" in error, "Expected error about disabled feature" + + # make sure kill operation also throws + error = replica_with_export_disabled.query_and_get_error(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") + assert "experimental" in error, "Expected error about disabled feature" + + +def test_export_partition_permissions(cluster): + """Test that export partition validates permissions correctly: + - User needs ALTER permission on source table + - User needs INSERT permission on destination table + """ + node = cluster.instances["replica1"] + + mt_table = "permissions_mt_table" + s3_table = "permissions_s3_table" + + # Create tables as default user + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Create test users with specific permissions + node.query("CREATE USER IF NOT EXISTS user_no_alter IDENTIFIED WITH no_password") + node.query("CREATE USER IF NOT EXISTS user_no_insert IDENTIFIED WITH no_password") + node.query("CREATE USER IF NOT EXISTS user_with_permissions IDENTIFIED WITH no_password") + + # Grant basic access to all users + node.query(f"GRANT SELECT ON {mt_table} TO user_no_alter") + node.query(f"GRANT SELECT ON {s3_table} TO user_no_alter") + + # user_no_insert has ALTER on source but no INSERT on destination + node.query(f"GRANT ALTER ON {mt_table} TO user_no_insert") + node.query(f"GRANT SELECT ON {s3_table} TO user_no_insert") + + # user_with_permissions has both ALTER and INSERT + node.query(f"GRANT ALTER ON {mt_table} TO user_with_permissions") + node.query(f"GRANT INSERT ON {s3_table} TO user_with_permissions") + + # Test 1: User without ALTER permission should fail + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_no_alter" + ) + + assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ + f"Expected ACCESS_DENIED error for user without ALTER, got: {error}" + + # Test 2: User with ALTER but without INSERT permission should fail + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_no_insert" + ) + + assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ + f"Expected ACCESS_DENIED error for user without INSERT, got: {error}" + + # Test 3: User with both ALTER and INSERT should succeed + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_with_permissions" + ) + + # Wait for export to complete + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # Verify the export succeeded + result = node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") + assert result.strip() == "3", f"Expected 3 rows exported, got: {result}" + + # Verify system table shows COMPLETED status + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert status.strip() == "COMPLETED", f"Expected COMPLETED status, got: {status}" + + +# assert multiple exports within a single query are executed. They all share the same query id +# and previously the transaction id was the query id, which would cause problems +def test_multiple_exports_within_a_single_query(cluster): + node = cluster.instances["replica1"] + + mt_table = "multiple_exports_within_a_single_query_mt_table" + s3_table = "multiple_exports_within_a_single_query_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}, EXPORT PARTITION ID '2021' TO TABLE {s3_table};") + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") + + # assert the exports have been executed + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not succeed" + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") == '1\n', "Export did not succeed" + + # check system.replicated_partition_exports for the exports + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2021' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + +def test_pending_mutations_throw_before_export_partition(cluster): + """Test that pending mutations before export partition throw an error.""" + node = cluster.instances["replica1"] + + mt_table = "pending_mutations_throw_partition_mt_table" + s3_table = "pending_mutations_throw_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0") + assert mutations.strip() != '0', "Mutation should be pending" + + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error, f"Expected error about pending mutations, got: {error}" + + +def test_pending_mutations_skip_before_export_partition(cluster): + """Test that pending mutations before export partition are skipped with throw_on_pending_mutations=false.""" + node = cluster.instances["replica1"] + + mt_table = "pending_mutations_skip_partition_mt_table" + s3_table = "pending_mutations_skip_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0") + assert mutations.strip() != '0', "Mutation should be pending" + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=false" + ) + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "101" not in result and "102" not in result and "103" not in result, \ + "Export should contain original data before mutation" + assert "1\n2\n3" in result, "Export should contain original data" + + +def test_pending_patch_parts_throw_before_export_partition(cluster): + """Test that pending patch parts before export partition throw an error with default settings.""" + node = cluster.instances["replica1"] + + mt_table = "pending_patches_throw_partition_mt_table" + s3_table = "pending_patches_throw_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + ) + + node.query(f"DROP TABLE {mt_table}") + + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error or "pending patch parts" in error.lower(), \ + f"Expected error about pending patch parts, got: {error}" + + +def test_pending_patch_parts_skip_before_export_partition(cluster): + """Test that pending patch parts before export partition are skipped with throw_on_pending_patch_parts=false.""" + node = cluster.instances["replica1"] + + mt_table = "pending_patches_skip_partition_mt_table" + s3_table = "pending_patches_skip_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_patch_parts=false" + ) + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before patch" + + node.query(f"DROP TABLE {mt_table}") + + +def test_mutations_after_export_partition_started(cluster): + """Test that mutations applied after export partition starts don't affect the exported data.""" + node = cluster.instances["replica1"] + + mt_table = "mutations_after_export_partition_mt_table" + s3_table = "mutations_after_export_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to MinIO to delay export + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + # Wait for export to start + wait_for_export_to_start(node, mt_table, s3_table, "2020") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before mutation" + assert "101" not in result, "Export should not contain mutated data" + + +def test_patch_parts_after_export_partition_started(cluster): + """Test that patch parts created after export partition starts don't affect the exported data.""" + node = cluster.instances["replica1"] + + mt_table = "patches_after_export_partition_mt_table" + s3_table = "patches_after_export_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to MinIO to delay export + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + ) + + # Wait for export to start + wait_for_export_to_start(node, mt_table, s3_table, "2020") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before patch" + assert "101" not in result, "Export should not contain patched data" + + node.query(f"DROP TABLE {mt_table}") + + +def test_mutation_in_partition_clause(cluster): + """Test that mutations limited to specific partitions using IN PARTITION clause + allow exports of unaffected partitions to succeed.""" + node = cluster.instances["replica1"] + + mt_table = "mutation_in_partition_clause_mt_table" + s3_table = "mutation_in_partition_clause_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + # Issue a mutation that uses IN PARTITION to limit it to partition 2020 + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 IN PARTITION '2020' WHERE year = 2020") + + # Verify mutation is pending for 2020 + mutations = node.query( + f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0" + ) + assert mutations.strip() != '0', "Mutation should be pending" + + # Export of 2020 should fail (it has pending mutations) + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error, f"Expected error about pending mutations for partition 2020, got: {error}" + + # Export of 2021 should succeed (no mutations affecting it) + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2021 ORDER BY id") + assert "4" in result, "Export of partition 2021 should contain original data" + + +def test_export_partition_with_mixed_computed_columns(cluster): + """Test export partition with ALIAS, MATERIALIZED, and EPHEMERAL columns.""" + node = cluster.instances["replica1"] + + mt_table = "mixed_computed_mt_table" + s3_table = "mixed_computed_s3_table" + + node.query(f""" + CREATE TABLE {mt_table} ( + id UInt32, + value UInt32, + tag_input String EPHEMERAL, + doubled UInt64 ALIAS value * 2, + tripled UInt64 MATERIALIZED value * 3, + tag String DEFAULT upper(tag_input) + ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{mt_table}', 'replica1') + PARTITION BY id + ORDER BY id + SETTINGS index_granularity = 1 + """) + + # Create S3 destination table with regular columns (no EPHEMERAL) + node.query(f""" + CREATE TABLE {s3_table} ( + id UInt32, + value UInt32, + doubled UInt64, + tripled UInt64, + tag String + ) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') + PARTITION BY id + """) + + node.query(f"INSERT INTO {mt_table} (id, value, tag_input) VALUES (1, 5, 'test'), (1, 10, 'prod')") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '1' TO TABLE {s3_table}") + + wait_for_export_status(node, mt_table, s3_table, "1", "COMPLETED") + + # Verify source data (ALIAS computed, EPHEMERAL not stored) + source_result = node.query(f"SELECT id, value, doubled, tripled, tag FROM {mt_table} ORDER BY value") + expected = "1\t5\t10\t15\tTEST\n1\t10\t20\t30\tPROD\n" + assert source_result == expected, f"Source table data mismatch. Expected:\n{expected}\nGot:\n{source_result}" + + dest_result = node.query(f"SELECT id, value, doubled, tripled, tag FROM {s3_table} ORDER BY value") + assert dest_result == expected, f"Exported data mismatch. Expected:\n{expected}\nGot:\n{dest_result}" + + status = node.query(f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '1' + """) + assert status.strip() == "COMPLETED", f"Expected COMPLETED status, got: {status}" diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index b63f357d776a..1e2cedacddd2 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -44,6 +44,8 @@ ALTER MATERIALIZE TTL ['MATERIALIZE TTL'] TABLE ALTER TABLE ALTER REWRITE PARTS ['REWRITE PARTS'] TABLE ALTER TABLE ALTER SETTINGS ['ALTER SETTING','ALTER MODIFY SETTING','MODIFY SETTING','RESET SETTING'] TABLE ALTER TABLE ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTER TABLE +ALTER EXPORT PART ['ALTER EXPORT PART','EXPORT PART'] TABLE ALTER TABLE +ALTER EXPORT PARTITION ['ALTER EXPORT PARTITION','EXPORT PARTITION'] TABLE ALTER TABLE ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE ALTER UNLOCK SNAPSHOT ['UNLOCK SNAPSHOT'] TABLE ALTER TABLE diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index eea09bd06035..1b4fcf850453 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -18,6 +18,8 @@ columns columns creator_info creator_info +exports +exports failed_parts failed_parts flags diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index 0d6c21be132f..9a250fa65580 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -8,6 +8,7 @@ blocks columns columns creator_info +exports failed_parts flags host @@ -49,6 +50,7 @@ blocks columns columns creator_info +exports failed_parts flags host diff --git a/tests/queries/0_stateless/03413_experimental_settings_cannot_be_enabled_by_default.sql b/tests/queries/0_stateless/03413_experimental_settings_cannot_be_enabled_by_default.sql index 718eb63ad923..809185ed5e3e 100644 --- a/tests/queries/0_stateless/03413_experimental_settings_cannot_be_enabled_by_default.sql +++ b/tests/queries/0_stateless/03413_experimental_settings_cannot_be_enabled_by_default.sql @@ -4,5 +4,5 @@ -- However, some settings in the experimental tier are meant to control another experimental feature, and then they can be enabled as long as the feature itself is disabled. -- These are in the exceptions list inside NOT IN. -SELECT name, value FROM system.settings WHERE tier = 'Experimental' AND type = 'Bool' AND value != '0' AND name NOT IN ('throw_on_unsupported_query_inside_transaction'); +SELECT name, value FROM system.settings WHERE tier = 'Experimental' AND type = 'Bool' AND value != '0' AND name NOT IN ('throw_on_unsupported_query_inside_transaction', 'allow_experimental_export_merge_tree_part'); SELECT name, value FROM system.merge_tree_settings WHERE tier = 'Experimental' AND type = 'Bool' AND value != '0' AND name NOT IN ('remove_rolled_back_parts_immediately'); diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.reference b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.reference new file mode 100644 index 000000000000..883191745d91 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.reference @@ -0,0 +1,90 @@ +---- Export 2020_1_1_0 and 2021_2_2_0 +---- Both data parts should appear +1 2020 +2 2020 +3 2020 +4 2021 +---- Export the same part again, it should be idempotent +1 2020 +2 2020 +3 2020 +4 2021 +---- Data in roundtrip MergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 +---- Export 2020_1_1_0 and 2021_2_2_0 to wildcard table +---- Both data parts should appear +1 2020 +2 2020 +3 2020 +4 2021 +---- Export the same part again, it should be idempotent +1 2020 +2 2020 +3 2020 +4 2021 +---- Export 2020_1_1_0 and 2021_2_2_0 to wildcard table with partition expression with function +---- Both data parts should appear +1 2020 +2 2020 +3 2020 +4 2021 +---- Test max_bytes and max_rows per file +---- Count files in big_destination_max_bytes, should be 5 (4 parquet, 1 commit) +5 +---- Count rows in big_table and big_destination_max_bytes +4194304 +4194304 +---- Count files in big_destination_max_rows, should be 5 (4 parquet, 1 commit) +5 +---- Count rows in big_table and big_destination_max_rows +4194304 +4194304 +---- Table function with schema inheritance (no schema specified) +---- Data should be exported with inherited schema +100 test1 2022 +101 test2 2022 +---- Table function with explicit compatible schema +---- Data should be exported with explicit schema +102 test3 2023 +---- Test ALIAS columns export +---- Verify ALIAS column data in source table (arr_1 computed from arr[1]) +1 [1,2,3] 1 +1 [10,20,30] 10 +---- Verify ALIAS column data exported to S3 (should match source) +1 [1,2,3] 1 +1 [10,20,30] 10 +---- Test MATERIALIZED columns export +---- Verify MATERIALIZED column data in source table (arr_1 computed from arr[1]) +1 [1,2,3] 1 +1 [10,20,30] 10 +---- Verify MATERIALIZED column data exported to S3 (should match source) +1 [1,2,3] 1 +1 [10,20,30] 10 +---- Test EPHEMERAL column (not stored, ignored during export) +---- Verify data in source +1 ALICE +1 BOB +---- Verify exported data +1 ALICE +1 BOB +---- Test Mixed ALIAS, MATERIALIZED, and EPHEMERAL in same table +---- Verify mixed columns in source table +1 5 10 15 TEST +1 10 20 30 PROD +---- Verify mixed columns exported to S3 (should match source) +1 5 10 15 TEST +1 10 20 30 PROD +---- Test Export to Table Function with mixed columns +---- Verify mixed columns exported to S3 +1 5 10 15 TEST +1 10 20 30 PROD +---- Test Complex Expressions in computed columns +---- Verify complex expressions in source table +1 alice ALICE alice-1 +1 bob BOB bob-1 +---- Verify complex expressions exported to S3 (should match source) +1 alice ALICE alice-1 +1 bob BOB bob-1 diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh new file mode 100755 index 000000000000..f6e172800f04 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh @@ -0,0 +1,284 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires s3 storage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +mt_table="mt_table_${RANDOM}" +mt_table_partition_expression_with_function="mt_table_partition_expression_with_function_${RANDOM}" +s3_table="s3_table_${RANDOM}" +s3_table_wildcard="s3_table_wildcard_${RANDOM}" +s3_table_wildcard_partition_expression_with_function="s3_table_wildcard_partition_expression_with_function_${RANDOM}" +mt_table_roundtrip="mt_table_roundtrip_${RANDOM}" +big_table="big_table_${RANDOM}" +big_destination_max_bytes="big_destination_max_bytes_${RANDOM}" +big_destination_max_rows="big_destination_max_rows_${RANDOM}" +tf_schema_inherit="tf_schema_inherit_${RANDOM}" +tf_schema_explicit="tf_schema_explicit_${RANDOM}" +mt_table_tf="mt_table_tf_${RANDOM}" +mt_alias="mt_alias_${RANDOM}" +mt_materialized="mt_materialized_${RANDOM}" +s3_alias_export="s3_alias_export_${RANDOM}" +s3_materialized_export="s3_materialized_export_${RANDOM}" +mt_mixed="mt_mixed_${RANDOM}" +s3_mixed_export="s3_mixed_export_${RANDOM}" +mt_complex_expr="mt_complex_expr_${RANDOM}" +s3_complex_expr_export="s3_complex_expr_export_${RANDOM}" +mt_ephemeral="mt_ephemeral_${RANDOM}" +s3_ephemeral_export="s3_ephemeral_export_${RANDOM}" +s3_mixed_export_table_function="s3_mixed_export_table_function_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip, $s3_table_wildcard, $s3_table_wildcard_partition_expression_with_function, $mt_table_partition_expression_with_function, $mt_alias, $mt_materialized, $s3_alias_export, $s3_materialized_export, $mt_mixed, $s3_mixed_export, $mt_complex_expr, $s3_complex_expr_export, $mt_ephemeral, $s3_ephemeral_export" + +query "CREATE TABLE $mt_table (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple()" +query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table', format=Parquet, partition_strategy='hive') PARTITION BY year" + +query "INSERT INTO $mt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" +echo "---- Export 2020_1_1_0 and 2021_2_2_0" +query "ALTER TABLE $mt_table EXPORT PART '2020_1_1_0' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $mt_table EXPORT PART '2021_2_2_0' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Both data parts should appear" +query "SELECT * FROM $s3_table ORDER BY id" + +echo "---- Export the same part again, it should be idempotent" +query "ALTER TABLE $mt_table EXPORT PART '2020_1_1_0' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $mt_table_roundtrip ENGINE = MergeTree() PARTITION BY year ORDER BY tuple() AS SELECT * FROM $s3_table" + +echo "---- Data in roundtrip MergeTree table (should match s3_table)" +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $s3_table_wildcard (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table_wildcard/{_partition_id}/{_file}.parquet', format=Parquet, partition_strategy='wildcard') PARTITION BY year" + +echo "---- Export 2020_1_1_0 and 2021_2_2_0 to wildcard table" +query "ALTER TABLE $mt_table EXPORT PART '2020_1_1_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $mt_table EXPORT PART '2021_2_2_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Both data parts should appear" +query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard/**.parquet') ORDER BY id" + +echo "---- Export the same part again, it should be idempotent" +query "ALTER TABLE $mt_table EXPORT PART '2020_1_1_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard/**.parquet') ORDER BY id" + +query "CREATE TABLE $mt_table_partition_expression_with_function (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY toString(year) ORDER BY tuple()" +query "CREATE TABLE $s3_table_wildcard_partition_expression_with_function (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table_wildcard_partition_expression_with_function/{_partition_id}/{_file}.parquet', format=Parquet, partition_strategy='wildcard') PARTITION BY toString(year)" + +query "INSERT INTO $mt_table_partition_expression_with_function VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" + +echo "---- Export 2020_1_1_0 and 2021_2_2_0 to wildcard table with partition expression with function" +query "ALTER TABLE $mt_table_partition_expression_with_function EXPORT PART 'cb217c742dc7d143b61583011996a160_1_1_0' TO TABLE $s3_table_wildcard_partition_expression_with_function SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $mt_table_partition_expression_with_function EXPORT PART '3be6d49ecf9749a383964bc6fab22d10_2_2_0' TO TABLE $s3_table_wildcard_partition_expression_with_function SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 1 + +echo "---- Both data parts should appear" +query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard_partition_expression_with_function/**.parquet') ORDER BY id" + +echo "---- Test max_bytes and max_rows per file" + +query "CREATE TABLE $big_table (id UInt64, data String, year UInt16) Engine=MergeTree() order by id partition by year" + +query "CREATE TABLE $big_destination_max_bytes(id UInt64, data String, year UInt16) engine=S3(s3_conn, filename='$big_destination_max_bytes', partition_strategy='hive', format=Parquet) partition by year" + +query "CREATE TABLE $big_destination_max_rows(id UInt64, data String, year UInt16) engine=S3(s3_conn, filename='$big_destination_max_rows', partition_strategy='hive', format=Parquet) partition by year" + +# 4194304 is a number that came up during multiple iterations, it does not really mean anything (aside from the fact that the below numbers depend on it) +query "INSERT INTO $big_table SELECT number AS id, repeat('x', 100) AS data, 2025 AS year FROM numbers(4194304)" + +query "INSERT INTO $big_table SELECT number AS id, repeat('x', 100) AS data, 2026 AS year FROM numbers(4194304)" + +# make sure we have only one part +query "OPTIMIZE TABLE $big_table FINAL" + +big_part_max_bytes=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$big_table' AND partition_id = '2025' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') +big_part_max_rows=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$big_table' AND partition_id = '2026' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') + +# this should generate ~4 files +query "ALTER TABLE $big_table EXPORT PART '$big_part_max_bytes' TO TABLE $big_destination_max_bytes SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_max_bytes_per_file=3500000, output_format_parquet_row_group_size_bytes=1000000" +# export_merge_tree_part_max_rows_per_file = 1048576 (which is 4194304/4) to generate 4 files +query "ALTER TABLE $big_table EXPORT PART '$big_part_max_rows' TO TABLE $big_destination_max_rows SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_max_rows_per_file=1048576" + +# sleeping a little longer because it will write multiple files, trying not be flaky +sleep 20 + +echo "---- Count files in big_destination_max_bytes, should be 5 (4 parquet, 1 commit)" +query "SELECT count(_file) FROM s3(s3_conn, filename='$big_destination_max_bytes/**', format='One')" + +echo "---- Count rows in big_table and big_destination_max_bytes" +query "SELECT COUNT() from $big_table WHERE year = 2025" +query "SELECT COUNT() from $big_destination_max_bytes" + +echo "---- Count files in big_destination_max_rows, should be 5 (4 parquet, 1 commit)" +query "SELECT count(_file) FROM s3(s3_conn, filename='$big_destination_max_rows/**', format='One')" + +echo "---- Count rows in big_table and big_destination_max_rows" +query "SELECT COUNT() from $big_table WHERE year = 2026" +query "SELECT COUNT() from $big_destination_max_rows" + +echo "---- Table function with schema inheritance (no schema specified)" +query "CREATE TABLE $mt_table_tf (id UInt64, value String, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple()" +query "INSERT INTO $mt_table_tf VALUES (100, 'test1', 2022), (101, 'test2', 2022), (102, 'test3', 2023)" + +query "ALTER TABLE $mt_table_tf EXPORT PART '2022_1_1_0' TO TABLE FUNCTION s3(s3_conn, filename='$tf_schema_inherit', format='Parquet', partition_strategy='hive') PARTITION BY year SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Data should be exported with inherited schema" +query "SELECT * FROM s3(s3_conn, filename='$tf_schema_inherit/**.parquet') ORDER BY id" + +echo "---- Table function with explicit compatible schema" +query "ALTER TABLE $mt_table_tf EXPORT PART '2023_2_2_0' TO TABLE FUNCTION s3(s3_conn, filename='$tf_schema_explicit', format='Parquet', structure='id UInt64, value String, year UInt16', partition_strategy='hive') PARTITION BY year SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Data should be exported with explicit schema" +query "SELECT * FROM s3(s3_conn, filename='$tf_schema_explicit/**.parquet') ORDER BY id" + +echo "---- Test ALIAS columns export" +query "CREATE TABLE $mt_alias (a UInt32, arr Array(UInt64), arr_1 UInt64 ALIAS arr[1]) ENGINE = MergeTree() PARTITION BY a ORDER BY (a, arr[1]) SETTINGS index_granularity = 1" +query "CREATE TABLE $s3_alias_export (a UInt32, arr Array(UInt64), arr_1 UInt64) ENGINE = S3(s3_conn, filename='$s3_alias_export', format=Parquet, partition_strategy='hive') PARTITION BY a" + +query "INSERT INTO $mt_alias VALUES (1, [1, 2, 3]), (1, [10, 20, 30])" + +alias_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_alias' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') + +query "ALTER TABLE $mt_alias EXPORT PART '$alias_part' TO TABLE $s3_alias_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Verify ALIAS column data in source table (arr_1 computed from arr[1])" +query "SELECT a, arr, arr_1 FROM $mt_alias ORDER BY arr" + +echo "---- Verify ALIAS column data exported to S3 (should match source)" +query "SELECT a, arr, arr_1 FROM $s3_alias_export ORDER BY arr" + +echo "---- Test MATERIALIZED columns export" +query "CREATE TABLE $mt_materialized (a UInt32, arr Array(UInt64), arr_1 UInt64 MATERIALIZED arr[1]) ENGINE = MergeTree() PARTITION BY a ORDER BY (a, arr_1) SETTINGS index_granularity = 1" +query "CREATE TABLE $s3_materialized_export (a UInt32, arr Array(UInt64), arr_1 UInt64) ENGINE = S3(s3_conn, filename='$s3_materialized_export', format=Parquet, partition_strategy='hive') PARTITION BY a" + +query "INSERT INTO $mt_materialized VALUES (1, [1, 2, 3]), (1, [10, 20, 30])" + +materialized_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_materialized' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') + +query "ALTER TABLE $mt_materialized EXPORT PART '$materialized_part' TO TABLE $s3_materialized_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Verify MATERIALIZED column data in source table (arr_1 computed from arr[1])" +query "SELECT a, arr, arr_1 FROM $mt_materialized ORDER BY arr" + +echo "---- Verify MATERIALIZED column data exported to S3 (should match source)" +query "SELECT a, arr, arr_1 FROM $s3_materialized_export ORDER BY arr" + +echo "---- Test EPHEMERAL column (not stored, ignored during export)" +query "CREATE TABLE $mt_ephemeral ( + id UInt32, + name_input String EPHEMERAL, + name_upper String DEFAULT upper(name_input) +) ENGINE = MergeTree() PARTITION BY id ORDER BY id SETTINGS index_granularity = 1" + +query "CREATE TABLE $s3_ephemeral_export ( + id UInt32, + name_upper String +) ENGINE = S3(s3_conn, filename='$s3_ephemeral_export', format=Parquet, partition_strategy='hive') PARTITION BY id" + +query "INSERT INTO $mt_ephemeral (id, name_input) VALUES (1, 'alice'), (1, 'bob')" + +ephemeral_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_ephemeral' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') + +query "ALTER TABLE $mt_ephemeral EXPORT PART '$ephemeral_part' TO TABLE $s3_ephemeral_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Verify data in source" +query "SELECT id, name_upper FROM $mt_ephemeral ORDER BY name_upper" + +echo "---- Verify exported data" +query "SELECT id, name_upper FROM $s3_ephemeral_export ORDER BY name_upper" + +echo "---- Test Mixed ALIAS, MATERIALIZED, and EPHEMERAL in same table" +query "CREATE TABLE $mt_mixed ( + id UInt32, + value UInt32, + tag_input String EPHEMERAL, + doubled UInt64 ALIAS value * 2, + tripled UInt64 MATERIALIZED value * 3, + tag String DEFAULT upper(tag_input) +) ENGINE = MergeTree() PARTITION BY id ORDER BY id SETTINGS index_granularity = 1" + +query "CREATE TABLE $s3_mixed_export ( + id UInt32, + value UInt32, + doubled UInt64, + tripled UInt64, + tag String +) ENGINE = S3(s3_conn, filename='$s3_mixed_export', format=Parquet, partition_strategy='hive') PARTITION BY id" + +query "INSERT INTO $mt_mixed (id, value, tag_input) VALUES (1, 5, 'test'), (1, 10, 'prod')" + +mixed_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_mixed' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') + +query "ALTER TABLE $mt_mixed EXPORT PART '$mixed_part' TO TABLE $s3_mixed_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Verify mixed columns in source table" +query "SELECT id, value, doubled, tripled, tag FROM $mt_mixed ORDER BY value" + +echo "---- Verify mixed columns exported to S3 (should match source)" +query "SELECT id, value, doubled, tripled, tag FROM $s3_mixed_export ORDER BY value" + +echo "---- Test Export to Table Function with mixed columns" + +query "ALTER TABLE $mt_mixed EXPORT PART '$mixed_part' TO TABLE FUNCTION s3(s3_conn, filename='$s3_mixed_export_table_function', format=Parquet, partition_strategy='hive') PARTITION BY id SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Verify mixed columns exported to S3" +query "SELECT * FROM s3(s3_conn, filename='$s3_mixed_export_table_function/**.parquet', format=Parquet) ORDER BY value" + +echo "---- Test Complex Expressions in computed columns" +query "CREATE TABLE $mt_complex_expr ( + id UInt32, + name String, + upper_name String ALIAS upper(name), + concat_result String MATERIALIZED concat(name, '-', toString(id)) +) ENGINE = MergeTree() PARTITION BY id ORDER BY id SETTINGS index_granularity = 1" + +query "CREATE TABLE $s3_complex_expr_export ( + id UInt32, + name String, + upper_name String, + concat_result String +) ENGINE = S3(s3_conn, filename='$s3_complex_expr_export', format=Parquet, partition_strategy='hive') PARTITION BY id" + +query "INSERT INTO $mt_complex_expr (id, name) VALUES (1, 'alice'), (1, 'bob')" + +complex_expr_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_complex_expr' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') + +query "ALTER TABLE $mt_complex_expr EXPORT PART '$complex_expr_part' TO TABLE $s3_complex_expr_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +sleep 3 + +echo "---- Verify complex expressions in source table" +query "SELECT id, name, upper_name, concat_result FROM $mt_complex_expr ORDER BY name" + +echo "---- Verify complex expressions exported to S3 (should match source)" +query "SELECT id, name, upper_name, concat_result FROM $s3_complex_expr_export ORDER BY name" + +query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip, $mt_table_tf, $s3_table_wildcard, $s3_table_wildcard_partition_expression_with_function, $mt_table_partition_expression_with_function, $big_table, $big_destination_max_bytes, $big_destination_max_rows, $mt_alias, $mt_materialized, $s3_alias_export, $s3_materialized_export, $mt_ephemeral, $s3_ephemeral_export, $mt_mixed, $s3_mixed_export, $mt_complex_expr, $s3_complex_expr_export" diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.reference b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.sql b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.sql new file mode 100644 index 000000000000..7cb70af024a2 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.sql @@ -0,0 +1,39 @@ +-- Tags: no-parallel, no-fasttest + +DROP TABLE IF EXISTS 03572_mt_table, 03572_invalid_schema_table, 03572_ephemeral_mt_table, 03572_matching_ephemeral_s3_table; + +SET allow_experimental_export_merge_tree_part=1; + +CREATE TABLE 03572_mt_table (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +INSERT INTO 03572_mt_table VALUES (1, 2020); + +-- Create a table with a different partition key and export a partition to it. It should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table', format='Parquet', partition_strategy='hive') PARTITION BY x; + +ALTER TABLE 03572_mt_table EXPORT PART '2020_1_1_0' TO TABLE 03572_invalid_schema_table +SETTINGS allow_experimental_export_merge_tree_part = 1; -- {serverError INCOMPATIBLE_COLUMNS} + +DROP TABLE 03572_invalid_schema_table; + +-- The only partition strategy that supports exports is hive. Wildcard should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table/{_partition_id}', format='Parquet', partition_strategy='wildcard') PARTITION BY (id, year); + +ALTER TABLE 03572_mt_table EXPORT PART '2020_1_1_0' TO TABLE 03572_invalid_schema_table; -- {serverError NOT_IMPLEMENTED} + +-- Not a table function, should throw +ALTER TABLE 03572_mt_table EXPORT PART '2020_1_1_0' TO TABLE FUNCTION extractKeyValuePairs('name:ronaldo'); -- {serverError UNKNOWN_FUNCTION} + +-- It is a table function, but the engine does not support exports/imports, should throw +ALTER TABLE 03572_mt_table EXPORT PART '2020_1_1_0' TO TABLE FUNCTION url('a.parquet'); -- {serverError NOT_IMPLEMENTED} + +-- Test that destination table can not have a column that matches the source ephemeral +CREATE TABLE 03572_ephemeral_mt_table (id UInt64, year UInt16, name String EPHEMERAL) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +CREATE TABLE 03572_matching_ephemeral_s3_table (id UInt64, year UInt16, name String) ENGINE = S3(s3_conn, filename='03572_matching_ephemeral_s3_table', format='Parquet', partition_strategy='hive') PARTITION BY year; + +INSERT INTO 03572_ephemeral_mt_table (id, year, name) VALUES (1, 2020, 'alice'); + +ALTER TABLE 03572_ephemeral_mt_table EXPORT PART '2020_1_1_0' TO TABLE 03572_matching_ephemeral_s3_table; -- {serverError INCOMPATIBLE_COLUMNS} + +DROP TABLE IF EXISTS 03572_mt_table, 03572_invalid_schema_table, 03572_ephemeral_mt_table, 03572_matching_ephemeral_s3_table; diff --git a/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.reference b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.reference new file mode 100644 index 000000000000..07f1ec6376a6 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.reference @@ -0,0 +1,16 @@ +---- Get actual part names and export them +---- Both data parts should appear +1 2020 +2 2020 +3 2020 +4 2021 +---- Export the same part again, it should be idempotent +1 2020 +2 2020 +3 2020 +4 2021 +---- Data in roundtrip ReplicatedMergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 diff --git a/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.sh new file mode 100755 index 000000000000..a691d4bdf37a --- /dev/null +++ b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: replica, no-parallel, no-replicated-database, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +rmt_table="rmt_table_${RANDOM}" +s3_table="s3_table_${RANDOM}" +rmt_table_roundtrip="rmt_table_roundtrip_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" + +query "CREATE TABLE $rmt_table (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table', 'replica1') PARTITION BY year ORDER BY tuple()" +query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table', format=Parquet, partition_strategy='hive') PARTITION BY year" + +query "INSERT INTO $rmt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" + +echo "---- Get actual part names and export them" +part_2020=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$rmt_table' AND partition = '2020' ORDER BY name LIMIT 1" | tr -d '\n') +part_2021=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$rmt_table' AND partition = '2021' ORDER BY name LIMIT 1" | tr -d '\n') + +query "ALTER TABLE $rmt_table EXPORT PART '$part_2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $rmt_table EXPORT PART '$part_2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +echo "---- Both data parts should appear" +query "SELECT * FROM $s3_table ORDER BY id" + +echo "---- Export the same part again, it should be idempotent" +query "ALTER TABLE $rmt_table EXPORT PART '$part_2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $rmt_table_roundtrip (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table_roundtrip', 'replica1') PARTITION BY year ORDER BY tuple()" +query "INSERT INTO $rmt_table_roundtrip SELECT * FROM $s3_table" + +echo "---- Data in roundtrip ReplicatedMergeTree table (should match s3_table)" +query "SELECT * FROM $rmt_table_roundtrip ORDER BY id" + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" diff --git a/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.reference b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.sql b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.sql new file mode 100644 index 000000000000..f8f23532f0a7 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.sql @@ -0,0 +1,22 @@ +-- Tags: no-parallel, no-fasttest + +DROP TABLE IF EXISTS 03572_rmt_table, 03572_invalid_schema_table; + +CREATE TABLE 03572_rmt_table (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_03572_rmt/03572_rmt_table', 'replica1') PARTITION BY year ORDER BY tuple(); + +INSERT INTO 03572_rmt_table VALUES (1, 2020); + +-- Create a table with a different partition key and export a partition to it. It should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table', format='Parquet', partition_strategy='hive') PARTITION BY x; + +ALTER TABLE 03572_rmt_table EXPORT PART '2020_0_0_0' TO TABLE 03572_invalid_schema_table +SETTINGS allow_experimental_export_merge_tree_part = 1; -- {serverError INCOMPATIBLE_COLUMNS} + +DROP TABLE 03572_invalid_schema_table; + +-- The only partition strategy that supports exports is hive. Wildcard should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table/{_partition_id}', format='Parquet', partition_strategy='wildcard') PARTITION BY (id, year); + +ALTER TABLE 03572_rmt_table EXPORT PART '2020_0_0_0' TO TABLE 03572_invalid_schema_table SETTINGS allow_experimental_export_merge_tree_part = 1; -- {serverError NOT_IMPLEMENTED} + +DROP TABLE IF EXISTS 03572_rmt_table, 03572_invalid_schema_table; diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.reference b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference new file mode 100644 index 000000000000..d48023362b99 --- /dev/null +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference @@ -0,0 +1,31 @@ +Select from source table +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 +Select from destination table +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +Export partition 2022 +Select from destination table again +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 +---- Data in roundtrip ReplicatedMergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh new file mode 100755 index 000000000000..87503112aadb --- /dev/null +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, replica, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +rmt_table="rmt_table_${RANDOM}" +s3_table="s3_table_${RANDOM}" +rmt_table_roundtrip="rmt_table_roundtrip_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" + +query "CREATE TABLE $rmt_table (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table', 'replica1') PARTITION BY year ORDER BY tuple()" +query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table', format=Parquet, partition_strategy='hive') PARTITION BY year" + +query "INSERT INTO $rmt_table VALUES (1, 2020), (2, 2020), (4, 2021)" + +query "INSERT INTO $rmt_table VALUES (3, 2020), (5, 2021)" + +query "INSERT INTO $rmt_table VALUES (6, 2022), (7, 2022)" + +# sync replicas +query "SYSTEM SYNC REPLICA $rmt_table" + +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +# todo poll some kind of status +sleep 15 + +echo "Select from source table" +query "SELECT * FROM $rmt_table ORDER BY id" + +echo "Select from destination table" +query "SELECT * FROM $s3_table ORDER BY id" + +echo "Export partition 2022" +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2022' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +# todo poll some kind of status +sleep 5 + +echo "Select from destination table again" +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $rmt_table_roundtrip ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table_roundtrip', 'replica1') PARTITION BY year ORDER BY tuple() AS SELECT * FROM $s3_table" + +echo "---- Data in roundtrip ReplicatedMergeTree table (should match s3_table)" +query "SELECT * FROM $rmt_table_roundtrip ORDER BY id" + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" \ No newline at end of file diff --git a/tests/queries/0_stateless/03745_system_background_schedule_pool.reference b/tests/queries/0_stateless/03745_system_background_schedule_pool.reference index 9e04e463e5eb..8bcf2b1ae84e 100644 --- a/tests/queries/0_stateless/03745_system_background_schedule_pool.reference +++ b/tests/queries/0_stateless/03745_system_background_schedule_pool.reference @@ -1,4 +1,5 @@ 1 buffer_flush default test_buffer_03745 1 StorageBuffer (default.test_buffer_03745)/Bg schedule default test_merge_tree_03745 1 BackgroundJobsAssignee:DataProcessing +schedule default test_merge_tree_03745 1 BackgroundJobsAssignee:Moving distributed default test_distributed_03745 1 default.test_distributed_03745.DistributedInsertQueue.default/Bg diff --git a/tests/queries/0_stateless/03745_system_background_schedule_pool.sql b/tests/queries/0_stateless/03745_system_background_schedule_pool.sql index 83e04385563d..de2d253210a4 100644 --- a/tests/queries/0_stateless/03745_system_background_schedule_pool.sql +++ b/tests/queries/0_stateless/03745_system_background_schedule_pool.sql @@ -15,7 +15,7 @@ DROP TABLE test_table_03745; DROP TABLE IF EXISTS test_merge_tree_03745; CREATE TABLE test_merge_tree_03745 (x UInt64, y String) ENGINE = MergeTree() ORDER BY x; INSERT INTO test_merge_tree_03745 VALUES (1, 'a'), (2, 'b'); -SELECT pool, database, table, table_uuid != toUUIDOrDefault(0) AS has_uuid, log_name FROM system.background_schedule_pool WHERE database = currentDatabase(); +SELECT pool, database, table, table_uuid != toUUIDOrDefault(0) AS has_uuid, log_name FROM system.background_schedule_pool WHERE database = currentDatabase() ORDER BY (pool, database, table, log_name); DROP TABLE test_merge_tree_03745; -- Test 3: Distributed table (distributed pool)