diff --git a/.github/ISSUE_TEMPLATE/85_bug-report.md b/.github/ISSUE_TEMPLATE/85_bug-report.md index 3d2ed6148e3..08d03c284ca 100644 --- a/.github/ISSUE_TEMPLATE/85_bug-report.md +++ b/.github/ISSUE_TEMPLATE/85_bug-report.md @@ -13,6 +13,8 @@ assignees: '' > A clear and concise description of what works not as it is supposed to. +> A link to reproducer in [https://fiddle.clickhouse.com/](https://fiddle.clickhouse.com/). + **Does it reproduce on recent release?** [The list of releases](https://github.com/ClickHouse/ClickHouse/blob/master/utils/list-versions/version_date.tsv) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 3d43a960534..da84500559a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1056,6 +1056,23 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + MarkReleaseReady: + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderDebRelease + - BuilderDebAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Mark Commit Release Ready + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 mark_release_ready.py ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## @@ -3069,6 +3086,8 @@ jobs: needs: - DockerHubPush - BuilderReport + - BuilderSpecialReport + - MarkReleaseReady - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug2 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 09ca64977f0..d3580f4c4d7 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3579,6 +3579,7 @@ jobs: - DockerServerImages - CheckLabels - BuilderReport + - BuilderSpecialReport - FastTest - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index abe85d3e72d..8148905cec7 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -615,6 +615,23 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + MarkReleaseReady: + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderDebRelease + - BuilderDebAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Mark Commit Release Ready + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 mark_release_ready.py ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## @@ -1888,6 +1905,7 @@ jobs: - DockerServerImages - BuilderReport - BuilderSpecialReport + - MarkReleaseReady - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug2 diff --git a/CMakeLists.txt b/CMakeLists.txt index 7a04f347b2d..d10bc63c15e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -202,7 +202,7 @@ option(ADD_GDB_INDEX_FOR_GOLD "Add .gdb-index to resulting binaries for gold lin if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") # Can be lld or ld-lld or lld-13 or /path/to/lld. - if (LINKER_NAME MATCHES "lld") + if (LINKER_NAME MATCHES "lld" AND OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--gdb-index") message (STATUS "Adding .gdb-index via --gdb-index linker option.") @@ -248,7 +248,7 @@ endif () # Create BuildID when using lld. For other linkers it is created by default. # (NOTE: LINKER_NAME can be either path or name, and in different variants) -if (LINKER_NAME MATCHES "lld") +if (LINKER_NAME MATCHES "lld" AND OS_LINUX) # SHA1 is not cryptographically secure but it is the best what lld is offering. set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--build-id=sha1") endif () diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index eba12604b4d..44a9f979f99 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -10,6 +10,12 @@ #include "syscall.h" +#if defined(__has_feature) +#if __has_feature(memory_sanitizer) +#include +#endif +#endif + #define ARRAY_SIZE(a) sizeof((a))/sizeof((a[0])) /// Suppress TSan since it is possible for this code to be called from multiple threads, diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 8a17d97cf13..e8fecd9f30b 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -58,13 +58,19 @@ if (NOT LINKER_NAME) find_program (LLD_PATH NAMES "ld.lld") find_program (GOLD_PATH NAMES "ld.gold") elseif (COMPILER_CLANG) - find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld.lld" "lld") + # llvm lld is a generic driver. + # Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-ld (WebAssembly) instead + if (OS_LINUX) + find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "ld.lld") + elseif (OS_DARWIN) + find_program (LLD_PATH NAMES "ld64.lld-${COMPILER_VERSION_MAJOR}" "ld64.lld") + endif () find_program (GOLD_PATH NAMES "ld.gold" "gold") endif () endif() -if (OS_LINUX AND NOT LINKER_NAME) - # prefer lld linker over gold or ld on linux +if ((OS_LINUX OR OS_DARWIN) AND NOT LINKER_NAME) + # prefer lld linker over gold or ld on linux and macos if (LLD_PATH) if (COMPILER_GCC) # GCC driver requires one of supported linker names like "lld". diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index c8aa65bdd91..7dd43858416 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -77,15 +77,15 @@ While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = ## Virtual Columns {#virtual-columns} -When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. +When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. ### \_version -`_version` — Transaction counter. Type [UInt64](../../sql-reference/data-types/int-uint.md). +`_version` — Transaction counter. Type [UInt64](/docs/en/sql-reference/data-types/int-uint.md). ### \_sign -`_sign` — Deletion mark. Type [Int8](../../sql-reference/data-types/int-uint.md). Possible values: +`_sign` — Deletion mark. Type [Int8](/docs/en/sql-reference/data-types/int-uint.md). Possible values: - `1` — Row is not deleted, - `-1` — Row is deleted. @@ -93,29 +93,29 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree]( | MySQL | ClickHouse | |-------------------------|--------------------------------------------------------------| -| TINY | [Int8](../../sql-reference/data-types/int-uint.md) | -| SHORT | [Int16](../../sql-reference/data-types/int-uint.md) | -| INT24 | [Int32](../../sql-reference/data-types/int-uint.md) | -| LONG | [UInt32](../../sql-reference/data-types/int-uint.md) | -| LONGLONG | [UInt64](../../sql-reference/data-types/int-uint.md) | -| FLOAT | [Float32](../../sql-reference/data-types/float.md) | -| DOUBLE | [Float64](../../sql-reference/data-types/float.md) | -| DECIMAL, NEWDECIMAL | [Decimal](../../sql-reference/data-types/decimal.md) | -| DATE, NEWDATE | [Date](../../sql-reference/data-types/date.md) | -| DATETIME, TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) | -| DATETIME2, TIMESTAMP2 | [DateTime64](../../sql-reference/data-types/datetime64.md) | -| YEAR | [UInt16](../../sql-reference/data-types/int-uint.md) | -| TIME | [Int64](../../sql-reference/data-types/int-uint.md) | -| ENUM | [Enum](../../sql-reference/data-types/enum.md) | -| STRING | [String](../../sql-reference/data-types/string.md) | -| VARCHAR, VAR_STRING | [String](../../sql-reference/data-types/string.md) | -| BLOB | [String](../../sql-reference/data-types/string.md) | -| GEOMETRY | [String](../../sql-reference/data-types/string.md) | -| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | -| BIT | [UInt64](../../sql-reference/data-types/int-uint.md) | -| SET | [UInt64](../../sql-reference/data-types/int-uint.md) | +| TINY | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | +| SHORT | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | +| INT24 | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | +| LONG | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | +| LONGLONG | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | +| FLOAT | [Float32](/docs/en/sql-reference/data-types/float.md) | +| DOUBLE | [Float64](/docs/en/sql-reference/data-types/float.md) | +| DECIMAL, NEWDECIMAL | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | +| DATE, NEWDATE | [Date](/docs/en/sql-reference/data-types/date.md) | +| DATETIME, TIMESTAMP | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | +| DATETIME2, TIMESTAMP2 | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | +| YEAR | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | +| TIME | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | +| ENUM | [Enum](/docs/en/sql-reference/data-types/enum.md) | +| STRING | [String](/docs/en/sql-reference/data-types/string.md) | +| VARCHAR, VAR_STRING | [String](/docs/en/sql-reference/data-types/string.md) | +| BLOB | [String](/docs/en/sql-reference/data-types/string.md) | +| GEOMETRY | [String](/docs/en/sql-reference/data-types/string.md) | +| BINARY | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | +| BIT | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | +| SET | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | -[Nullable](../../sql-reference/data-types/nullable.md) is supported. +[Nullable](/docs/en/sql-reference/data-types/nullable.md) is supported. The data of TIME type in MySQL is converted to microseconds in ClickHouse. @@ -133,7 +133,7 @@ Apart of the data types limitations there are few restrictions comparing to `MyS ### DDL Queries {#ddl-queries} -MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop), [RENAME](../../sql-reference/statements/rename.md)). If ClickHouse cannot parse some DDL query, the query is ignored. +MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ALTER](/docs/en/sql-reference/statements/alter/index.md), [CREATE](/docs/en/sql-reference/statements/create/index.md), [DROP](/docs/en/sql-reference/statements/drop.md), [RENAME](/docs/en/sql-reference/statements/rename.md)). If ClickHouse cannot parse some DDL query, the query is ignored. ### Data Replication {#data-replication} @@ -151,7 +151,7 @@ MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ `SELECT` query from `MaterializedMySQL` tables has some specifics: - If `_version` is not specified in the `SELECT` query, the - [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier is used, so only rows with + [FINAL](/docs/en/sql-reference/statements/select/from.md/#select-from-final) modifier is used, so only rows with `MAX(_version)` are returned for each primary key value. - If `_sign` is not specified in the `SELECT` query, `WHERE _sign=1` is used by default. So the deleted rows are not @@ -164,7 +164,7 @@ MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ MySQL `PRIMARY KEY` and `INDEX` clauses are converted into `ORDER BY` tuples in ClickHouse tables. ClickHouse has only one physical order, which is determined by `ORDER BY` clause. To create a new physical order, use -[materialized views](../../sql-reference/statements/create/view.md#materialized). +[materialized views](/docs/en/sql-reference/statements/create/view.md/#materialized). **Notes** @@ -173,7 +173,7 @@ ClickHouse has only one physical order, which is determined by `ORDER BY` clause MySQL binlog. - Replication can be easily broken. - Manual operations on database and tables are forbidden. -- `MaterializedMySQL` is affected by the [optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert) +- `MaterializedMySQL` is affected by the [optimize_on_insert](/docs/en/operations/settings/settings.md/#optimize-on-insert) setting. Data is merged in the corresponding table in the `MaterializedMySQL` database when a table in the MySQL server changes. @@ -187,19 +187,19 @@ These are the schema conversion manipulations you can do with table overrides fo * Modify column type. Must be compatible with the original type, or replication will fail. For example, you can modify a UInt32 column to UInt64, but you can not modify a String column to Array(String). - * Modify [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl). - * Modify [column compression codec](../../sql-reference/statements/create/table/#codecs). - * Add [ALIAS columns](../../sql-reference/statements/create/table/#alias). - * Add [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes) - * Add [projections](../table-engines/mergetree-family/mergetree/#projections). Note that projection optimizations are + * Modify [column TTL](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-column-ttl). + * Modify [column compression codec](/docs/en/sql-reference/statements/create/table.md/#codecs). + * Add [ALIAS columns](/docs/en/sql-reference/statements/create/table.md/#alias). + * Add [skipping indexes](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-data_skipping-indexes) + * Add [projections](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#projections). Note that projection optimizations are disabled when using `SELECT ... FINAL` (which MaterializedMySQL does by default), so their utility is limited here. `INDEX ... TYPE hypothesis` as [described in the v21.12 blog post]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/) may be more useful in this case. - * Modify [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/) - * Modify [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Modify [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Add [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Add [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * Modify [PARTITION BY](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key/) + * Modify [ORDER BY](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Modify [PRIMARY KEY](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Add [SAMPLE BY](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Add [table TTL](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) ```sql CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) diff --git a/docs/en/engines/table-engines/log-family/index.md b/docs/en/engines/table-engines/log-family/index.md index 98bc4dbad04..486c41c2496 100644 --- a/docs/en/engines/table-engines/log-family/index.md +++ b/docs/en/engines/table-engines/log-family/index.md @@ -10,11 +10,11 @@ These engines were developed for scenarios when you need to quickly write many s Engines of the family: -- [StripeLog](../../../engines/table-engines/log-family/stripelog.md) -- [Log](../../../engines/table-engines/log-family/log.md) -- [TinyLog](../../../engines/table-engines/log-family/tinylog.md) +- [StripeLog](/docs/en/engines/table-engines/log-family/stripelog.md) +- [Log](/docs/en/engines/table-engines/log-family/log.md) +- [TinyLog](/docs/en/engines/table-engines/log-family/tinylog.md) -`Log` family table engines can store data to [HDFS](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-hdfs) or [S3](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-s3) distributed file systems. +`Log` family table engines can store data to [HDFS](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-hdfs) or [S3](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3) distributed file systems. ## Common Properties {#common-properties} @@ -28,7 +28,7 @@ Engines: During `INSERT` queries, the table is locked, and other queries for reading and writing data both wait for the table to unlock. If there are no data writing queries, any number of data reading queries can be performed concurrently. -- Do not support [mutations](../../../sql-reference/statements/alter/index.md#alter-mutations). +- Do not support [mutations](/docs/en/sql-reference/statements/alter/index.md/#alter-mutations). - Do not support indexes. diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 486baac2310..7dfb5a9fed7 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -16,20 +16,20 @@ Main features: This allows you to create a small sparse index that helps find data faster. -- Partitions can be used if the [partitioning key](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified. +- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified. ClickHouse supports certain operations with partitions that are more efficient than general operations on the same data with the same result. ClickHouse also automatically cuts off the partition data where the partitioning key is specified in the query. - Data replication support. - The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](../../../engines/table-engines/mergetree-family/replication.md). + The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - Data sampling support. If necessary, you can set the data sampling method in the table. :::info -The [Merge](../../../engines/table-engines/special/merge.md#merge) engine does not belong to the `*MergeTree` family. +The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family. ::: ## Creating a Table {#table_engine-mergetree-creating-a-table} @@ -57,7 +57,7 @@ ORDER BY expr [SETTINGS name=value, ...] ``` -For a description of parameters, see the [CREATE query description](../../../sql-reference/statements/create/table.md). +For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md). ### Query Clauses {#mergetree-query-clauses} @@ -77,9 +77,9 @@ Use the `ORDER BY tuple()` syntax, if you do not need sorting. See [Selecting th #### PARTITION BY -`PARTITION BY` — The [partitioning key](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases you don't need partition key, and in most other cases you don't need partition key more granular than by months. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead make client identifier or name the first column in the ORDER BY expression). +`PARTITION BY` — The [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases you don't need partition key, and in most other cases you don't need partition key more granular than by months. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead make client identifier or name the first column in the ORDER BY expression). -For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](../../../sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. +For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. #### PRIMARY KEY @@ -127,7 +127,7 @@ Additional parameters that control the behavior of the `MergeTree` (optional): #### use_minimalistic_part_header_in_zookeeper -`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”. +`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”. #### min_merge_bytes_to_use_direct_io @@ -166,15 +166,15 @@ Additional parameters that control the behavior of the `MergeTree` (optional): #### max_compress_block_size -`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](../../../operations/settings/settings.md#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. +`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. #### min_compress_block_size -`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](../../../operations/settings/settings.md#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. +`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. #### max_partitions_to_read -`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](../../../operations/settings/merge-tree-settings.md#max-partitions-to-read) in the global setting. +`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. **Example of Sections Setting** @@ -184,7 +184,7 @@ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDa In the example, we set partitioning by month. -We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If you define a [SAMPLE](../../../sql-reference/statements/select/sample.md#select-sample-clause) clause when selecting the data, ClickHouse will return an evenly pseudorandom data sample for a subset of users. +We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If you define a [SAMPLE](/docs/en/sql-reference/statements/select/sample.md/#select-sample-clause) clause when selecting the data, ClickHouse will return an evenly pseudorandom data sample for a subset of users. The `index_granularity` setting can be omitted because 8192 is the default value. @@ -207,9 +207,9 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **MergeTree() Parameters** -- `date-column` — The name of a column of the [Date](../../../sql-reference/data-types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format. +- `date-column` — The name of a column of the [Date](/docs/en/sql-reference/data-types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format. - `sampling_expression` — An expression for sampling. -- `(primary, key)` — Primary key. Type: [Tuple()](../../../sql-reference/data-types/tuple.md) +- `(primary, key)` — Primary key. Type: [Tuple()](/docs/en/sql-reference/data-types/tuple.md) - `index_granularity` — The granularity of an index. The number of data rows between the “marks” of an index. The value 8192 is appropriate for most tasks. **Example** @@ -262,7 +262,7 @@ Sparse indexes allow you to work with a very large number of table rows, because ClickHouse does not require a unique primary key. You can insert multiple rows with the same primary key. -You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses but it is strongly discouraged. To allow this feature, turn on the [allow_nullable_key](../../../operations/settings/settings.md#allow-nullable-key) setting. The [NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. +You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses but it is strongly discouraged. To allow this feature, turn on the [allow_nullable_key](/docs/en/operations/settings/settings.md/#allow-nullable-key) setting. The [NULLS_LAST](/docs/en/sql-reference/statements/select/order-by.md/#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. ### Selecting the Primary Key {#selecting-the-primary-key} @@ -279,26 +279,26 @@ The number of columns in the primary key is not explicitly limited. Depending on ClickHouse sorts data by primary key, so the higher the consistency, the better the compression. -- Provide additional logic when merging data parts in the [CollapsingMergeTree](../../../engines/table-engines/mergetree-family/collapsingmergetree.md#table_engine-collapsingmergetree) and [SummingMergeTree](../../../engines/table-engines/mergetree-family/summingmergetree.md) engines. +- Provide additional logic when merging data parts in the [CollapsingMergeTree](/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md/#table_engine-collapsingmergetree) and [SummingMergeTree](/docs/en/engines/table-engines/mergetree-family/summingmergetree.md) engines. In this case it makes sense to specify the *sorting key* that is different from the primary key. A long primary key will negatively affect the insert performance and memory consumption, but extra columns in the primary key do not affect ClickHouse performance during `SELECT` queries. -You can create a table without a primary key using the `ORDER BY tuple()` syntax. In this case, ClickHouse stores data in the order of inserting. If you want to save data order when inserting data by `INSERT ... SELECT` queries, set [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads). +You can create a table without a primary key using the `ORDER BY tuple()` syntax. In this case, ClickHouse stores data in the order of inserting. If you want to save data order when inserting data by `INSERT ... SELECT` queries, set [max_insert_threads = 1](/docs/en/operations/settings/settings.md/#settings-max-insert-threads). -To select data in the initial order, use [single-threaded](../../../operations/settings/settings.md#settings-max_threads) `SELECT` queries. +To select data in the initial order, use [single-threaded](/docs/en/operations/settings/settings.md/#settings-max_threads) `SELECT` queries. ### Choosing a Primary Key that Differs from the Sorting Key {#choosing-a-primary-key-that-differs-from-the-sorting-key} It is possible to specify a primary key (an expression with values that are written in the index file for each mark) that is different from the sorting key (an expression for sorting the rows in data parts). In this case the primary key expression tuple must be a prefix of the sorting key expression tuple. -This feature is helpful when using the [SummingMergeTree](../../../engines/table-engines/mergetree-family/summingmergetree.md) and -[AggregatingMergeTree](../../../engines/table-engines/mergetree-family/aggregatingmergetree.md) table engines. In a common case when using these engines, the table has two types of columns: *dimensions* and *measures*. Typical queries aggregate values of measure columns with arbitrary `GROUP BY` and filtering by dimensions. Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key, it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns and this list must be frequently updated with newly added dimensions. +This feature is helpful when using the [SummingMergeTree](/docs/en/engines/table-engines/mergetree-family/summingmergetree.md) and +[AggregatingMergeTree](/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md) table engines. In a common case when using these engines, the table has two types of columns: *dimensions* and *measures*. Typical queries aggregate values of measure columns with arbitrary `GROUP BY` and filtering by dimensions. Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key, it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns and this list must be frequently updated with newly added dimensions. In this case it makes sense to leave only a few columns in the primary key that will provide efficient range scans and add the remaining dimension columns to the sorting key tuple. -[ALTER](../../../sql-reference/statements/alter/index.md) of the sorting key is a lightweight operation because when a new column is simultaneously added to the table and to the sorting key, existing data parts do not need to be changed. Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, the data is sorted by both the old and new sorting keys at the moment of table modification. +[ALTER](/docs/en/sql-reference/statements/alter/index.md) of the sorting key is a lightweight operation because when a new column is simultaneously added to the table and to the sorting key, existing data parts do not need to be changed. Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, the data is sorted by both the old and new sorting keys at the moment of table modification. ### Use of Indexes and Partitions in Queries {#use-of-indexes-and-partitions-in-queries} @@ -342,7 +342,7 @@ In the example below, the index can’t be used. SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) and [force_primary_key](../../../operations/settings/settings.md#force-primary-key). +To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](/docs/en/operations/settings/settings.md/#settings-force_index_by_date) and [force_primary_key](/docs/en/operations/settings/settings.md/#force-primary-key). The key for partitioning by month allows reading only those data blocks which contain dates from the proper range. In this case, the data block may contain data for many dates (up to an entire month). Within a block, data is sorted by primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date. @@ -400,7 +400,7 @@ Stores unique values of the specified expression (no more than `max_rows` rows, #### `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` -Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with datatypes: [String](../../../sql-reference/data-types/string.md), [FixedString](../../../sql-reference/data-types/fixedstring.md) and [Map](../../../sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions. +Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with datatypes: [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) and [Map](/docs/en/sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions. - `n` — ngram size, - `size_of_bloom_filter_in_bytes` — Bloom filter size in bytes (you can use large values here, for example, 256 or 512, because it can be compressed well). @@ -417,11 +417,11 @@ The optional `false_positive` parameter is the probability of receiving a false Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID`, `Map`. -For `Map` data type client can specify if index should be created for keys or values using [mapKeys](../../../sql-reference/functions/tuple-map-functions.md#mapkeys) or [mapValues](../../../sql-reference/functions/tuple-map-functions.md#mapvalues) function. +For `Map` data type client can specify if index should be created for keys or values using [mapKeys](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapkeys) or [mapValues](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapvalues) function. There are also special-purpose and experimental indexes to support approximate nearest neighbor (ANN) queries. See [here](annindexes.md) for details. -The following functions can use the filter: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions), [notIn](../../../sql-reference/functions/in-functions), [has](../../../sql-reference/functions/array-functions#hasarr-elem), [hasAny](../../../sql-reference/functions/array-functions#hasany), [hasAll](../../../sql-reference/functions/array-functions#hasall). +The following functions can use the filter: [equals](/docs/en/sql-reference/functions/comparison-functions.md), [notEquals](/docs/en/sql-reference/functions/comparison-functions.md), [in](/docs/en/sql-reference/functions/in-functions), [notIn](/docs/en/sql-reference/functions/in-functions), [has](/docs/en/sql-reference/functions/array-functions#hasarr-elem), [hasAny](/docs/en/sql-reference/functions/array-functions#hasany), [hasAll](/docs/en/sql-reference/functions/array-functions#hasall). Example of index creation for `Map` data type @@ -445,21 +445,21 @@ The `set` index can be used with all functions. Function subsets for other index | Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | |------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| -| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | -| [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | -| [in](../../../sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notIn](../../../sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greater (>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [lessOrEquals (<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greaterOrEquals (>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [empty](../../../sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [notEmpty](../../../sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | +| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | +| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | | hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | Functions with a constant argument that is less than ngram size can’t be used by `ngrambf_v1` for query optimization. @@ -485,16 +485,16 @@ For example: ## Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex} -In addition to skip indices, there are also [Approximate Nearest Neighbor Search Indexes](../../../engines/table-engines/mergetree-family/annindexes.md). +In addition to skip indices, there are also [Approximate Nearest Neighbor Search Indexes](/docs/en/engines/table-engines/mergetree-family/annindexes.md). ## Projections {#projections} -Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +Projections are like [materialized views](/docs/en/sql-reference/statements/create/view.md/#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. :::note -When you are implementing projections you should also consider the [force_optimize_projection](../../../operations/settings/settings.md#force-optimize-projection) setting. +When you are implementing projections you should also consider the [force_optimize_projection](/docs/en/operations/settings/settings.md/#force-optimize-projection) setting. ::: -Projections are not supported in the `SELECT` statements with the [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. +Projections are not supported in the `SELECT` statements with the [FINAL](/docs/en/sql-reference/statements/select/from.md/#select-from-final) modifier. ### Projection Query {#projection-query} A projection query is what defines a projection. It implicitly selects data from the parent table. @@ -504,7 +504,7 @@ A projection query is what defines a projection. It implicitly selects data from SELECT [GROUP BY] [ORDER BY] ``` -Projections can be modified or dropped with the [ALTER](../../../sql-reference/statements/alter/projection.md) statement. +Projections can be modified or dropped with the [ALTER](/docs/en/sql-reference/statements/alter/projection.md) statement. ### Projection Storage {#projection-storage} Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous `MergeTree` table's part. The table is induced by the definition query of the projection. If there is a `GROUP BY` clause, the underlying storage engine becomes [AggregatingMergeTree](aggregatingmergetree.md), and all aggregate functions are converted to `AggregateFunction`. If there is an `ORDER BY` clause, the `MergeTree` table uses it as its primary key expression. During the merge process the projection part is merged via its storage's merge routine. The checksum of the parent table's part is combined with the projection's part. Other maintenance jobs are similar to skip indices. @@ -526,7 +526,7 @@ Determines the lifetime of values. The `TTL` clause can be set for the whole table and for each individual column. Table-level `TTL` can also specify the logic of automatic moving data between disks and volumes, or recompressing parts where all the data has been expired. -Expressions must evaluate to [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md) data type. +Expressions must evaluate to [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type. **Syntax** @@ -537,7 +537,7 @@ TTL time_column TTL time_column + interval ``` -To define `interval`, use [time interval](../../../sql-reference/operators/index.md#operators-datetime) operators, for example: +To define `interval`, use [time interval](/docs/en/sql-reference/operators/index.md/#operators-datetime) operators, for example: ``` sql TTL date_time + INTERVAL 1 MONTH @@ -684,11 +684,11 @@ Data with an expired `TTL` is removed when ClickHouse merges data parts. When ClickHouse detects that data is expired, it performs an off-schedule merge. To control the frequency of such merges, you can set `merge_with_ttl_timeout`. If the value is too low, it will perform many off-schedule merges that may consume a lot of resources. -If you perform the `SELECT` query between merges, you may get expired data. To avoid it, use the [OPTIMIZE](../../../sql-reference/statements/optimize.md) query before `SELECT`. +If you perform the `SELECT` query between merges, you may get expired data. To avoid it, use the [OPTIMIZE](/docs/en/sql-reference/statements/optimize.md) query before `SELECT`. **See Also** -- [ttl_only_drop_parts](../../../operations/settings/settings.md#ttl_only_drop_parts) setting +- [ttl_only_drop_parts](/docs/en/operations/settings/settings.md/#ttl_only_drop_parts) setting ## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes} @@ -697,16 +697,16 @@ If you perform the `SELECT` query between merges, you may get expired data. To a `MergeTree` family table engines can store data on multiple block devices. For example, it can be useful when the data of a certain table are implicitly split into “hot” and “cold”. The most recent data is regularly requested but requires only a small amount of space. On the contrary, the fat-tailed historical data is requested rarely. If several disks are available, the “hot” data may be located on fast disks (for example, NVMe SSDs or in memory), while the “cold” data - on relatively slow ones (for example, HDD). -Data part is the minimum movable unit for `MergeTree`-engine tables. The data belonging to one part are stored on one disk. Data parts can be moved between disks in the background (according to user settings) as well as by means of the [ALTER](../../../sql-reference/statements/alter/partition.md#alter_move-partition) queries. +Data part is the minimum movable unit for `MergeTree`-engine tables. The data belonging to one part are stored on one disk. Data parts can be moved between disks in the background (according to user settings) as well as by means of the [ALTER](/docs/en/sql-reference/statements/alter/partition.md/#alter_move-partition) queries. ### Terms {#terms} - Disk — Block device mounted to the filesystem. -- Default disk — Disk that stores the path specified in the [path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) server setting. +- Default disk — Disk that stores the path specified in the [path](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-path) server setting. - Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). - Storage policy — Set of volumes and the rules for moving data between them. -The names given to the described entities can be found in the system tables, [system.storage_policies](../../../operations/system-tables/storage_policies.md#system_tables-storage_policies) and [system.disks](../../../operations/system-tables/disks.md#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables. +The names given to the described entities can be found in the system tables, [system.storage_policies](/docs/en/operations/system-tables/storage_policies.md/#system_tables-storage_policies) and [system.disks](/docs/en/operations/system-tables/disks.md/#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables. ### Configuration {#table_engine-mergetree-multiple-volumes_configure} @@ -853,16 +853,16 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' The `default` storage policy implies using only one volume, which consists of only one disk given in ``. You could change storage policy after table creation with [ALTER TABLE ... MODIFY SETTING] query, new policy should include all old disks and volumes with same names. -The number of threads performing background moves of data parts can be changed by [background_move_pool_size](../../../operations/settings/settings.md#background_move_pool_size) setting. +The number of threads performing background moves of data parts can be changed by [background_move_pool_size](/docs/en/operations/settings/settings.md/#background_move_pool_size) setting. ### Details {#details} In the case of `MergeTree` tables, data is getting to disk in different ways: - As a result of an insert (`INSERT` query). -- During background merges and [mutations](../../../sql-reference/statements/alter/index.md#alter-mutations). +- During background merges and [mutations](/docs/en/sql-reference/statements/alter/index.md/#alter-mutations). - When downloading from another replica. -- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter/partition.md#alter_freeze-partition). +- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](/docs/en/sql-reference/statements/alter/partition.md/#alter_freeze-partition). In all these cases except for mutations and partition freezing, a part is stored on a volume and a disk according to the given storage policy: @@ -872,16 +872,16 @@ In all these cases except for mutations and partition freezing, a part is stored Under the hood, mutations and partition freezing make use of [hard links](https://en.wikipedia.org/wiki/Hard_link). Hard links between different disks are not supported, therefore in such cases the resulting parts are stored on the same disks as the initial ones. In the background, parts are moved between volumes on the basis of the amount of free space (`move_factor` parameter) according to the order the volumes are declared in the configuration file. -Data is never transferred from the last one and into the first one. One may use system tables [system.part_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (field `type = MOVE_PART`) and [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (fields `path` and `disk`) to monitor background moves. Also, the detailed information can be found in server logs. +Data is never transferred from the last one and into the first one. One may use system tables [system.part_log](/docs/en/operations/system-tables/part_log.md/#system_tables-part-log) (field `type = MOVE_PART`) and [system.parts](/docs/en/operations/system-tables/parts.md/#system_tables-parts) (fields `path` and `disk`) to monitor background moves. Also, the detailed information can be found in server logs. -User can force moving a part or a partition from one volume to another using the query [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter/partition.md#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met. +User can force moving a part or a partition from one volume to another using the query [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](/docs/en/sql-reference/statements/alter/partition.md/#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met. Moving data does not interfere with data replication. Therefore, different storage policies can be specified for the same table on different replicas. After the completion of background merges and mutations, old parts are removed only after a certain amount of time (`old_parts_lifetime`). During this time, they are not moved to other volumes or disks. Therefore, until the parts are finally removed, they are still taken into account for evaluation of the occupied disk space. -User can assign new big parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way using the [min_bytes_to_rebalance_partition_over_jbod](../../../operations/settings/merge-tree-settings.md#min-bytes-to-rebalance-partition-over-jbod) setting. +User can assign new big parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way using the [min_bytes_to_rebalance_partition_over_jbod](/docs/en/operations/settings/merge-tree-settings.md/#min-bytes-to-rebalance-partition-over-jbod) setting. ## Using S3 for Data Storage {#table_engine-mergetree-s3} diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 06faceab8ec..67b595d0fa0 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -20,7 +20,7 @@ Replication works at the level of an individual table, not the entire server. A Replication does not depend on sharding. Each shard has its own independent replication. -Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](../../../sql-reference/statements/alter/index.md#query_language_queries_alter)). +Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](/docs/en/sql-reference/statements/alter/index.md/#query_language_queries_alter)). `CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated: @@ -28,9 +28,9 @@ Compressed data for `INSERT` and `ALTER` queries is replicated (for more informa - The `DROP TABLE` query deletes the replica located on the server where the query is run. - The `RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas. -ClickHouse uses [ClickHouse Keeper](../../../guides/sre/keeper/clickhouse-keeper.md) for storing replicas meta information. It is possible to use ZooKeeper version 3.4.5 or newer, but ClickHouse Keeper is recommended. +ClickHouse uses [ClickHouse Keeper](/docs/en/guides/sre/keeper/clickhouse-keeper.md) for storing replicas meta information. It is possible to use ZooKeeper version 3.4.5 or newer, but ClickHouse Keeper is recommended. -To use replication, set parameters in the [zookeeper](../../../operations/server-configuration-parameters/settings.md#server-settings_zookeeper) server configuration section. +To use replication, set parameters in the [zookeeper](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings_zookeeper) server configuration section. :::warning Don’t neglect the security setting. ClickHouse supports the `digest` [ACL scheme](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) of the ZooKeeper security subsystem. @@ -95,21 +95,21 @@ You can specify any existing ZooKeeper cluster and the system will use a directo If ZooKeeper isn’t set in the config file, you can’t create replicated tables, and any existing replicated tables will be read-only. -ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). +ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](/docs/en/operations/settings/settings.md/#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](/docs/en/operations/settings/settings.md/#settings-fallback_to_stale_replicas_for_distributed_queries). For each `INSERT` query, approximately ten entries are added to ZooKeeper through several transactions. (To be more precise, this is for each inserted block of data; an INSERT query contains one block or one block per `max_insert_block_size = 1048576` rows.) This leads to slightly longer latencies for `INSERT` compared to non-replicated tables. But if you follow the recommendations to insert data in batches of no more than one `INSERT` per second, it does not create any problems. The entire ClickHouse cluster used for coordinating one ZooKeeper cluster has a total of several hundred `INSERTs` per second. The throughput on data inserts (the number of rows per second) is just as high as for non-replicated data. For very large clusters, you can use different ZooKeeper clusters for different shards. However, from our experience this has not proven necessary based on production clusters with approximately 300 servers. -Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. The number of threads performing background tasks for replicated tables can be set by [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) setting. +Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. The number of threads performing background tasks for replicated tables can be set by [background_schedule_pool_size](/docs/en/operations/settings/settings.md/#background_schedule_pool_size) setting. -`ReplicatedMergeTree` engine uses a separate thread pool for replicated fetches. Size of the pool is limited by the [background_fetches_pool_size](../../../operations/settings/settings.md#background_fetches_pool_size) setting which can be tuned with a server restart. +`ReplicatedMergeTree` engine uses a separate thread pool for replicated fetches. Size of the pool is limited by the [background_fetches_pool_size](/docs/en/operations/settings/settings.md/#background_fetches_pool_size) setting which can be tuned with a server restart. By default, an INSERT query waits for confirmation of writing the data from only one replica. If the data was successfully written to only one replica and the server with this replica ceases to exist, the stored data will be lost. To enable getting confirmation of data writes from multiple replicas, use the `insert_quorum` option. Each block of data is written atomically. The INSERT query is divided into blocks up to `max_insert_block_size = 1048576` rows. In other words, if the `INSERT` query has less than 1048576 rows, it is made atomically. -Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application does not know if the data was written to the DB, so the `INSERT` query can simply be repeated. It does not matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) server settings. +Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application does not know if the data was written to the DB, so the `INSERT` query can simply be repeated. It does not matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-merge_tree) server settings. During replication, only the source data to insert is transferred over the network. Further data transformation (merging) is coordinated and performed on all the replicas in the same way. This minimizes network usage, which means that replication works well when replicas reside in different datacenters. (Note that duplicating data in different datacenters is the main goal of replication.) @@ -165,7 +165,7 @@ CREATE TABLE table_name -As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the [macros](../../../operations/server-configuration-parameters/settings.md#macros) section of the configuration file. +As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the [macros](/docs/en/operations/server-configuration-parameters/settings.md/#macros) section of the configuration file. Example: @@ -295,10 +295,10 @@ If the data in ClickHouse Keeper was lost or damaged, you can save data by movin **See Also** -- [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) -- [background_fetches_pool_size](../../../operations/settings/settings.md#background_fetches_pool_size) -- [execute_merges_on_single_replica_time_threshold](../../../operations/settings/settings.md#execute-merges-on-single-replica-time-threshold) -- [max_replicated_fetches_network_bandwidth](../../../operations/settings/merge-tree-settings.md#max_replicated_fetches_network_bandwidth) -- [max_replicated_sends_network_bandwidth](../../../operations/settings/merge-tree-settings.md#max_replicated_sends_network_bandwidth) +- [background_schedule_pool_size](/docs/en/operations/settings/settings.md/#background_schedule_pool_size) +- [background_fetches_pool_size](/docs/en/operations/settings/settings.md/#background_fetches_pool_size) +- [execute_merges_on_single_replica_time_threshold](/docs/en/operations/settings/settings.md/#execute-merges-on-single-replica-time-threshold) +- [max_replicated_fetches_network_bandwidth](/docs/en/operations/settings/merge-tree-settings.md/#max_replicated_fetches_network_bandwidth) +- [max_replicated_sends_network_bandwidth](/docs/en/operations/settings/merge-tree-settings.md/#max_replicated_sends_network_bandwidth) [Original article](https://clickhouse.com/docs/en/operations/table_engines/replication/) diff --git a/docs/en/engines/table-engines/special/join.md b/docs/en/engines/table-engines/special/join.md index 161896e5550..0e51a8b7696 100644 --- a/docs/en/engines/table-engines/special/join.md +++ b/docs/en/engines/table-engines/special/join.md @@ -6,10 +6,10 @@ sidebar_label: Join # Join Table Engine -Optional prepared data structure for usage in [JOIN](../../../sql-reference/statements/select/join.md#select-join) operations. +Optional prepared data structure for usage in [JOIN](/docs/en/sql-reference/statements/select/join.md/#select-join) operations. :::note -This is not an article about the [JOIN clause](../../../sql-reference/statements/select/join.md#select-join) itself. +This is not an article about the [JOIN clause](/docs/en/sql-reference/statements/select/join.md/#select-join) itself. ::: ## Creating a Table {#creating-a-table} @@ -22,17 +22,17 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE = Join(join_strictness, join_type, k1[, k2, ...]) ``` -See the detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. +See the detailed description of the [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md/#create-table-query) query. ## Engine Parameters ### join_strictness -`join_strictness` – [JOIN strictness](../../../sql-reference/statements/select/join.md#select-join-types). +`join_strictness` – [JOIN strictness](/docs/en/sql-reference/statements/select/join.md/#select-join-types). ### join_type -`join_type` – [JOIN type](../../../sql-reference/statements/select/join.md#select-join-types). +`join_type` – [JOIN type](/docs/en/sql-reference/statements/select/join.md/#select-join-types). ### Key columns @@ -55,11 +55,11 @@ You can use `INSERT` queries to add data to the `Join`-engine tables. If the tab Main use-cases for `Join`-engine tables are following: - Place the table to the right side in a `JOIN` clause. -- Call the [joinGet](../../../sql-reference/functions/other-functions.md#joinget) function, which lets you extract data from the table the same way as from a dictionary. +- Call the [joinGet](/docs/en/sql-reference/functions/other-functions.md/#joinget) function, which lets you extract data from the table the same way as from a dictionary. ### Deleting Data {#deleting-data} -`ALTER DELETE` queries for `Join`-engine tables are implemented as [mutations](../../../sql-reference/statements/alter/index.md#mutations). `DELETE` mutation reads filtered data and overwrites data of memory and disk. +`ALTER DELETE` queries for `Join`-engine tables are implemented as [mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations). `DELETE` mutation reads filtered data and overwrites data of memory and disk. ### Limitations and Settings {#join-limitations-and-settings} @@ -67,30 +67,30 @@ When creating a table, the following settings are applied: #### join_use_nulls -[join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +[join_use_nulls](/docs/en/operations/settings/settings.md/#join_use_nulls) #### max_rows_in_join -[max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +[max_rows_in_join](/docs/en/operations/settings/query-complexity.md/#settings-max_rows_in_join) #### max_bytes_in_join -[max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +[max_bytes_in_join](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join) #### join_overflow_mode -[join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +[join_overflow_mode](/docs/en/operations/settings/query-complexity.md/#settings-join_overflow_mode) #### join_any_take_last_row -[join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +[join_any_take_last_row](/docs/en/operations/settings/settings.md/#settings-join_any_take_last_row) #### join_use_nulls -[persistent](../../../operations/settings/settings.md#persistent) +[persistent](/docs/en/operations/settings/settings.md/#persistent) The `Join`-engine tables can’t be used in `GLOBAL JOIN` operations. -The `Join`-engine allows to specify [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) setting in the `CREATE TABLE` statement. [SELECT](../../../sql-reference/statements/select/index.md) query should have the same `join_use_nulls` value. +The `Join`-engine allows to specify [join_use_nulls](/docs/en/operations/settings/settings.md/#join_use_nulls) setting in the `CREATE TABLE` statement. [SELECT](/docs/en/sql-reference/statements/select/index.md) query should have the same `join_use_nulls` value. ## Usage Examples {#example} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7f7c14817ba..d2f0f46f637 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -70,7 +70,7 @@ Another use case of `prefer_global_in_and_join` is accessing tables created by **See also:** -- [Distributed subqueries](../../sql-reference/operators/in.md#select-distributed-subqueries) for more information on how to use `GLOBAL IN`/`GLOBAL JOIN` +- [Distributed subqueries](../../sql-reference/operators/in.md/#select-distributed-subqueries) for more information on how to use `GLOBAL IN`/`GLOBAL JOIN` ## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} @@ -170,7 +170,7 @@ It makes sense to disable it if the server has millions of tiny tables that are ## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} -Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md#range). Defines the maximum number of values generated by function per block of data (sum of array sizes for every row in a block). +Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md/#range). Defines the maximum number of values generated by function per block of data (sum of array sizes for every row in a block). Possible values: @@ -273,10 +273,10 @@ Default value: 0. ## insert_null_as_default {#insert_null_as_default} -Enables or disables the insertion of [default values](../../sql-reference/statements/create/table.md#create-default-values) instead of [NULL](../../sql-reference/syntax.md#null-literal) into columns with not [nullable](../../sql-reference/data-types/nullable.md#data_type-nullable) data type. +Enables or disables the insertion of [default values](../../sql-reference/statements/create/table.md/#create-default-values) instead of [NULL](../../sql-reference/syntax.md/#null-literal) into columns with not [nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable) data type. If column type is not nullable and this setting is disabled, then inserting `NULL` causes an exception. If column type is nullable, then `NULL` values are inserted as is, regardless of this setting. -This setting is applicable to [INSERT ... SELECT](../../sql-reference/statements/insert-into.md#insert_query_insert-select) queries. Note that `SELECT` subqueries may be concatenated with `UNION ALL` clause. +This setting is applicable to [INSERT ... SELECT](../../sql-reference/statements/insert-into.md/#insert_query_insert-select) queries. Note that `SELECT` subqueries may be concatenated with `UNION ALL` clause. Possible values: @@ -287,7 +287,7 @@ Default value: `1`. ## join_default_strictness {#settings-join_default_strictness} -Sets default strictness for [JOIN clauses](../../sql-reference/statements/select/join.md#select-join). +Sets default strictness for [JOIN clauses](../../sql-reference/statements/select/join.md/#select-join). Possible values: @@ -322,7 +322,7 @@ When using `partial_merge` algorithm, ClickHouse sorts the data and dumps it to - `direct` - can be applied when the right storage supports key-value requests. -The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs. +The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md/#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs. - `auto` — try `hash` join and switch on the fly to another algorithm if the memory limit is violated. @@ -348,7 +348,7 @@ Default value: 0. See also: -- [JOIN clause](../../sql-reference/statements/select/join.md#select-join) +- [JOIN clause](../../sql-reference/statements/select/join.md/#select-join) - [Join table engine](../../engines/table-engines/special/join.md) - [join_default_strictness](#settings-join_default_strictness) @@ -359,7 +359,7 @@ Sets the type of [JOIN](../../sql-reference/statements/select/join.md) behaviour Possible values: - 0 — The empty cells are filled with the default value of the corresponding field type. -- 1 — `JOIN` behaves the same way as in standard SQL. The type of the corresponding field is converted to [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable), and empty cells are filled with [NULL](../../sql-reference/syntax.md). +- 1 — `JOIN` behaves the same way as in standard SQL. The type of the corresponding field is converted to [Nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable), and empty cells are filled with [NULL](../../sql-reference/syntax.md). Default value: 0. @@ -431,7 +431,7 @@ Default value: 0. See also: -- [JOIN strictness](../../sql-reference/statements/select/join.md#join-settings) +- [JOIN strictness](../../sql-reference/statements/select/join.md/#join-settings) ## temporary_files_codec {#temporary_files_codec} @@ -532,7 +532,7 @@ Default value: 8. If ClickHouse should read more than `merge_tree_max_rows_to_use_cache` rows in one query, it does not use the cache of uncompressed blocks. -The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. +The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md/#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. Possible values: @@ -544,7 +544,7 @@ Default value: 128 ✕ 8192. If ClickHouse should read more than `merge_tree_max_bytes_to_use_cache` bytes in one query, it does not use the cache of uncompressed blocks. -The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. +The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md/#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. Possible values: @@ -594,7 +594,7 @@ Default value: `1`. Setting up query logging. -Queries sent to ClickHouse with this setup are logged according to the rules in the [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server configuration parameter. +Queries sent to ClickHouse with this setup are logged according to the rules in the [query_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query-log) server configuration parameter. Example: @@ -639,7 +639,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' Setting up query threads logging. -Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. +Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter. Possible values: @@ -658,7 +658,7 @@ log_query_threads=1 Setting up query views logging. -When a query run by ClickHouse with this setup on has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) server configuration parameter. +When a query run by ClickHouse with this setup on has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_views_log) server configuration parameter. Example: @@ -884,7 +884,7 @@ Default value: `5`. ## max_replicated_fetches_network_bandwidth_for_server {#max_replicated_fetches_network_bandwidth_for_server} -Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) fetches for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_fetches_network_bandwidth](../../operations/settings/merge-tree-settings.md#max_replicated_fetches_network_bandwidth) setting. +Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) fetches for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_fetches_network_bandwidth](../../operations/settings/merge-tree-settings.md/#max_replicated_fetches_network_bandwidth) setting. The setting isn't followed perfectly accurately. @@ -905,7 +905,7 @@ Could be used for throttling speed when replicating the data to add or replace n ## max_replicated_sends_network_bandwidth_for_server {#max_replicated_sends_network_bandwidth_for_server} -Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) sends for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_sends_network_bandwidth](../../operations/settings/merge-tree-settings.md#max_replicated_sends_network_bandwidth) setting. +Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) sends for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_sends_network_bandwidth](../../operations/settings/merge-tree-settings.md/#max_replicated_sends_network_bandwidth) setting. The setting isn't followed perfectly accurately. @@ -955,7 +955,7 @@ For more information, see the section “Extreme values”. ## kafka_max_wait_ms {#kafka-max-wait-ms} -The wait time in milliseconds for reading messages from [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) before retry. +The wait time in milliseconds for reading messages from [Kafka](../../engines/table-engines/integrations/kafka.md/#kafka) before retry. Possible values: @@ -977,7 +977,7 @@ Default value: false. ## use_uncompressed_cache {#setting-use_uncompressed_cache} Whether to use a cache of uncompressed blocks. Accepts 0 or 1. By default, 0 (disabled). -Using the uncompressed cache (only for tables in the MergeTree family) can significantly reduce latency and increase throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +Using the uncompressed cache (only for tables in the MergeTree family) can significantly reduce latency and increase throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md/#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically to save space for truly small queries. This means that you can keep the ‘use_uncompressed_cache’ setting always set to 1. @@ -1124,7 +1124,7 @@ This setting is useful for replicated tables with a sampling key. A query may be - The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. :::warning -This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md#max_parallel_replica-subqueries) for more details. +This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. ::: ## compile_expressions {#compile-expressions} @@ -1261,7 +1261,7 @@ Possible values: Default value: 1. By default, blocks inserted into replicated tables by the `INSERT` statement are deduplicated (see [Data Replication](../../engines/table-engines/mergetree-family/replication.md)). -For the replicated tables by default the only 100 of the most recent blocks for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). +For the replicated tables by default the only 100 of the most recent blocks for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md/#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). For not replicated tables see [non_replicated_deduplication_window](merge-tree-settings.md/#non-replicated-deduplication-window). ## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} @@ -1296,7 +1296,7 @@ Default value: empty string (disabled) `insert_deduplication_token` is used for deduplication _only_ when not empty. -For the replicated tables by default the only 100 of the most recent inserts for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). +For the replicated tables by default the only 100 of the most recent inserts for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md/#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). For not replicated tables see [non_replicated_deduplication_window](merge-tree-settings.md/#non-replicated-deduplication-window). Example: @@ -1373,15 +1373,15 @@ Default value: 0. ## count_distinct_implementation {#settings-count_distinct_implementation} -Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) construction. +Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md/#agg_function-count) construction. Possible values: -- [uniq](../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) -- [uniqCombined](../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined) -- [uniqCombined64](../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) -- [uniqHLL12](../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12) -- [uniqExact](../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) +- [uniq](../../sql-reference/aggregate-functions/reference/uniq.md/#agg_function-uniq) +- [uniqCombined](../../sql-reference/aggregate-functions/reference/uniqcombined.md/#agg_function-uniqcombined) +- [uniqCombined64](../../sql-reference/aggregate-functions/reference/uniqcombined64.md/#agg_function-uniqcombined64) +- [uniqHLL12](../../sql-reference/aggregate-functions/reference/uniqhll12.md/#agg_function-uniqhll12) +- [uniqExact](../../sql-reference/aggregate-functions/reference/uniqexact.md/#agg_function-uniqexact) Default value: `uniqExact`. @@ -1616,14 +1616,14 @@ Enables or disables optimization by transforming some functions to reading subco These functions can be transformed: -- [length](../../sql-reference/functions/array-functions.md#array_functions-length) to read the [size0](../../sql-reference/data-types/array.md#array-size) subcolumn. -- [empty](../../sql-reference/functions/array-functions.md#function-empty) to read the [size0](../../sql-reference/data-types/array.md#array-size) subcolumn. -- [notEmpty](../../sql-reference/functions/array-functions.md#function-notempty) to read the [size0](../../sql-reference/data-types/array.md#array-size) subcolumn. -- [isNull](../../sql-reference/operators/index.md#operator-is-null) to read the [null](../../sql-reference/data-types/nullable.md#finding-null) subcolumn. -- [isNotNull](../../sql-reference/operators/index.md#is-not-null) to read the [null](../../sql-reference/data-types/nullable.md#finding-null) subcolumn. -- [count](../../sql-reference/aggregate-functions/reference/count.md) to read the [null](../../sql-reference/data-types/nullable.md#finding-null) subcolumn. -- [mapKeys](../../sql-reference/functions/tuple-map-functions.md#mapkeys) to read the [keys](../../sql-reference/data-types/map.md#map-subcolumns) subcolumn. -- [mapValues](../../sql-reference/functions/tuple-map-functions.md#mapvalues) to read the [values](../../sql-reference/data-types/map.md#map-subcolumns) subcolumn. +- [length](../../sql-reference/functions/array-functions.md/#array_functions-length) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. +- [empty](../../sql-reference/functions/array-functions.md/#function-empty) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. +- [notEmpty](../../sql-reference/functions/array-functions.md/#function-notempty) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. +- [isNull](../../sql-reference/operators/index.md/#operator-is-null) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. +- [isNotNull](../../sql-reference/operators/index.md/#is-not-null) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. +- [count](../../sql-reference/aggregate-functions/reference/count.md) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. +- [mapKeys](../../sql-reference/functions/tuple-map-functions.md/#mapkeys) to read the [keys](../../sql-reference/data-types/map.md/#map-subcolumns) subcolumn. +- [mapValues](../../sql-reference/functions/tuple-map-functions.md/#mapvalues) to read the [values](../../sql-reference/data-types/map.md/#map-subcolumns) subcolumn. Possible values: @@ -1782,7 +1782,7 @@ Default value: 1000000000 nanoseconds (once a second). See also: -- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) ## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} @@ -1805,7 +1805,7 @@ Default value: 1000000000 nanoseconds. See also: -- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) ## allow_introspection_functions {#settings-allow_introspection_functions} @@ -1821,11 +1821,11 @@ Default value: 0. **See Also** - [Sampling Query Profiler](../../operations/optimizing-performance/sampling-query-profiler.md) -- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) ## input_format_parallel_parsing {#input-format-parallel-parsing} -Enables or disables order-preserving parallel parsing of data formats. Supported only for [TSV](../../interfaces/formats.md#tabseparated), [TKSV](../../interfaces/formats.md#tskv), [CSV](../../interfaces/formats.md#csv) and [JSONEachRow](../../interfaces/formats.md#jsoneachrow) formats. +Enables or disables order-preserving parallel parsing of data formats. Supported only for [TSV](../../interfaces/formats.md/#tabseparated), [TKSV](../../interfaces/formats.md/#tskv), [CSV](../../interfaces/formats.md/#csv) and [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) formats. Possible values: @@ -1836,7 +1836,7 @@ Default value: `1`. ## output_format_parallel_formatting {#output-format-parallel-formatting} -Enables or disables parallel formatting of data formats. Supported only for [TSV](../../interfaces/formats.md#tabseparated), [TKSV](../../interfaces/formats.md#tskv), [CSV](../../interfaces/formats.md#csv) and [JSONEachRow](../../interfaces/formats.md#jsoneachrow) formats. +Enables or disables parallel formatting of data formats. Supported only for [TSV](../../interfaces/formats.md/#tabseparated), [TKSV](../../interfaces/formats.md/#tskv), [CSV](../../interfaces/formats.md/#csv) and [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) formats. Possible values: @@ -1878,7 +1878,7 @@ Default value: 0. ## insert_distributed_sync {#insert_distributed_sync} -Enables or disables synchronous data insertion into a [Distributed](../../engines/table-engines/special/distributed.md#distributed) table. +Enables or disables synchronous data insertion into a [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table. By default, when inserting data into a `Distributed` table, the ClickHouse server sends data to cluster nodes in asynchronous mode. When `insert_distributed_sync=1`, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true). @@ -1891,12 +1891,12 @@ Default value: `0`. **See Also** -- [Distributed Table Engine](../../engines/table-engines/special/distributed.md#distributed) -- [Managing Distributed Tables](../../sql-reference/statements/system.md#query-language-system-distributed) +- [Distributed Table Engine](../../engines/table-engines/special/distributed.md/#distributed) +- [Managing Distributed Tables](../../sql-reference/statements/system.md/#query-language-system-distributed) ## insert_shard_id {#insert_shard_id} -If not `0`, specifies the shard of [Distributed](../../engines/table-engines/special/distributed.md#distributed) table into which the data will be inserted synchronously. +If not `0`, specifies the shard of [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table into which the data will be inserted synchronously. If `insert_shard_id` value is incorrect, the server will throw an exception. @@ -1909,7 +1909,7 @@ SELECT uniq(shard_num) FROM system.clusters WHERE cluster = 'requested_cluster'; Possible values: - 0 — Disabled. -- Any number from `1` to `shards_num` of corresponding [Distributed](../../engines/table-engines/special/distributed.md#distributed) table. +- Any number from `1` to `shards_num` of corresponding [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table. Default value: `0`. @@ -1969,7 +1969,7 @@ Default value: 16. ## background_move_pool_size {#background_move_pool_size} -Sets the number of threads performing background moves of data parts for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes)-engine tables. This setting is applied at the ClickHouse server start and can’t be changed in a user session. +Sets the number of threads performing background moves of data parts for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-multiple-volumes)-engine tables. This setting is applied at the ClickHouse server start and can’t be changed in a user session. Possible values: @@ -1979,7 +1979,7 @@ Default value: 8. ## background_schedule_pool_size {#background_schedule_pool_size} -Sets the number of threads performing background tasks for [replicated](../../engines/table-engines/mergetree-family/replication.md) tables, [Kafka](../../engines/table-engines/integrations/kafka.md) streaming, [DNS cache updates](../../operations/server-configuration-parameters/settings.md#server-settings-dns-cache-update-period). This setting is applied at ClickHouse server start and can’t be changed in a user session. +Sets the number of threads performing background tasks for [replicated](../../engines/table-engines/mergetree-family/replication.md) tables, [Kafka](../../engines/table-engines/integrations/kafka.md) streaming, [DNS cache updates](../../operations/server-configuration-parameters/settings.md/#server-settings-dns-cache-update-period). This setting is applied at ClickHouse server start and can’t be changed in a user session. Possible values: @@ -2036,12 +2036,12 @@ Default value: 16. **See Also** -- [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) engine. -- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine) engine. +- [Kafka](../../engines/table-engines/integrations/kafka.md/#kafka) engine. +- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md/#rabbitmq-engine) engine. ## validate_polygons {#validate_polygons} -Enables or disables throwing an exception in the [pointInPolygon](../../sql-reference/functions/geo/index.md#pointinpolygon) function, if the polygon is self-intersecting or self-tangent. +Enables or disables throwing an exception in the [pointInPolygon](../../sql-reference/functions/geo/index.md/#pointinpolygon) function, if the polygon is self-intersecting or self-tangent. Possible values: @@ -2052,7 +2052,7 @@ Default value: 1. ## transform_null_in {#transform_null_in} -Enables equality of [NULL](../../sql-reference/syntax.md#null-literal) values for [IN](../../sql-reference/operators/in.md) operator. +Enables equality of [NULL](../../sql-reference/syntax.md/#null-literal) values for [IN](../../sql-reference/operators/in.md) operator. By default, `NULL` values can’t be compared because `NULL` means undefined value. Thus, comparison `expr = NULL` must always return `false`. With this setting `NULL = NULL` returns `true` for `IN` operator. @@ -2106,7 +2106,7 @@ Result: **See Also** -- [NULL Processing in IN Operators](../../sql-reference/operators/in.md#in-null-processing) +- [NULL Processing in IN Operators](../../sql-reference/operators/in.md/#in-null-processing) ## low_cardinality_max_dictionary_size {#low_cardinality_max_dictionary_size} @@ -2133,7 +2133,7 @@ Default value: 0. ## low_cardinality_allow_in_native_format {#low_cardinality_allow_in_native_format} -Allows or restricts using the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type with the [Native](../../interfaces/formats.md#native) format. +Allows or restricts using the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type with the [Native](../../interfaces/formats.md/#native) format. If usage of `LowCardinality` is restricted, ClickHouse server converts `LowCardinality`-columns to ordinary ones for `SELECT` queries, and convert ordinary columns to `LowCardinality`-columns for `INSERT` queries. @@ -2197,7 +2197,7 @@ Default value: 268435456. ## optimize_read_in_order {#optimize_read_in_order} -Enables [ORDER BY](../../sql-reference/statements/select/order-by.md#optimize_read_in_order) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries for reading data from [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables. +Enables [ORDER BY](../../sql-reference/statements/select/order-by.md/#optimize_read_in_order) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries for reading data from [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables. Possible values: @@ -2208,7 +2208,7 @@ Default value: `1`. **See Also** -- [ORDER BY Clause](../../sql-reference/statements/select/order-by.md#optimize_read_in_order) +- [ORDER BY Clause](../../sql-reference/statements/select/order-by.md/#optimize_read_in_order) ## optimize_aggregation_in_order {#optimize_aggregation_in_order} @@ -2223,11 +2223,11 @@ Default value: `0`. **See Also** -- [GROUP BY optimization](../../sql-reference/statements/select/group-by.md#aggregation-in-order) +- [GROUP BY optimization](../../sql-reference/statements/select/group-by.md/#aggregation-in-order) ## mutations_sync {#mutations_sync} -Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql-reference/statements/alter/index.md#mutations)) synchronously. +Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql-reference/statements/alter/index.md/#mutations)) synchronously. Possible values: @@ -2239,8 +2239,8 @@ Default value: `0`. **See Also** -- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) -- [Mutations](../../sql-reference/statements/alter/index.md#mutations) +- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) +- [Mutations](../../sql-reference/statements/alter/index.md/#mutations) ## ttl_only_drop_parts {#ttl_only_drop_parts} @@ -2261,8 +2261,8 @@ Default value: `0`. **See Also** -- [CREATE TABLE query clauses and settings](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) (`merge_with_ttl_timeout` setting) -- [Table TTL](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) +- [CREATE TABLE query clauses and settings](../../engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) (`merge_with_ttl_timeout` setting) +- [Table TTL](../../engines/table-engines/mergetree-family/mergetree.md/#mergetree-table-ttl) ## lock_acquire_timeout {#lock_acquire_timeout} @@ -2279,7 +2279,7 @@ Default value: `120` seconds. ## cast_keep_nullable {#cast_keep_nullable} -Enables or disables keeping of the `Nullable` data type in [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) operations. +Enables or disables keeping of the `Nullable` data type in [CAST](../../sql-reference/functions/type-conversion-functions.md/#type_conversion_function-cast) operations. When the setting is enabled and the argument of `CAST` function is `Nullable`, the result is also transformed to `Nullable` type. When the setting is disabled, the result always has the destination type exactly. @@ -2324,7 +2324,7 @@ Result: **See Also** -- [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function +- [CAST](../../sql-reference/functions/type-conversion-functions.md/#type_conversion_function-cast) function ## system_events_show_zero_values {#system_events_show_zero_values} @@ -2369,7 +2369,7 @@ Result ## persistent {#persistent} -Disables persistency for the [Set](../../engines/table-engines/special/set.md#set) and [Join](../../engines/table-engines/special/join.md#join) table engines. +Disables persistency for the [Set](../../engines/table-engines/special/set.md/#set) and [Join](../../engines/table-engines/special/join.md/#join) table engines. Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence. @@ -2382,7 +2382,7 @@ Default value: `1`. ## allow_nullable_key {#allow-nullable-key} -Allows using of the [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree) tables. +Allows using of the [Nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md/#table_engines-mergetree) tables. Possible values: @@ -2401,7 +2401,7 @@ Do not enable this feature in version `<= 21.8`. It's not properly implemented a ## aggregate_functions_null_for_empty {#aggregate_functions_null_for_empty} -Enables or disables rewriting all aggregate functions in a query, adding [-OrNull](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-ornull) suffix to them. Enable it for SQL standard compatibility. +Enables or disables rewriting all aggregate functions in a query, adding [-OrNull](../../sql-reference/aggregate-functions/combinators.md/#agg-functions-combinator-ornull) suffix to them. Enable it for SQL standard compatibility. It is implemented via query rewrite (similar to [count_distinct_implementation](#settings-count_distinct_implementation) setting) to get consistent results for distributed queries. Possible values: @@ -2448,7 +2448,7 @@ See examples in [UNION](../../sql-reference/statements/select/union.md). ## data_type_default_nullable {#data_type_default_nullable} -Allows data types without explicit modifiers [NULL or NOT NULL](../../sql-reference/statements/create/table.md#null-modifiers) in column definition will be [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable). +Allows data types without explicit modifiers [NULL or NOT NULL](../../sql-reference/statements/create/table.md/#null-modifiers) in column definition will be [Nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable). Possible values: @@ -2478,7 +2478,7 @@ It can be useful when merges are CPU bounded not IO bounded (performing heavy da ## max_final_threads {#max-final-threads} -Sets the maximum number of parallel threads for the `SELECT` query data read phase with the [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier. +Sets the maximum number of parallel threads for the `SELECT` query data read phase with the [FINAL](../../sql-reference/statements/select/from.md/#select-from-final) modifier. Possible values: @@ -2551,7 +2551,7 @@ Result: └─────────────┘ ``` -Note that this setting influences [Materialized view](../../sql-reference/statements/create/view.md#materialized) and [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md) behaviour. +Note that this setting influences [Materialized view](../../sql-reference/statements/create/view.md/#materialized) and [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md) behaviour. ## engine_file_empty_if_not_exists {#engine-file-empty_if-not-exists} @@ -2608,7 +2608,7 @@ Default value: `0`. ## allow_experimental_live_view {#allow-experimental-live-view} -Allows creation of experimental [live views](../../sql-reference/statements/create/view.md#live-view). +Allows creation of experimental [live views](../../sql-reference/statements/create/view.md/#live-view). Possible values: @@ -2619,19 +2619,19 @@ Default value: `0`. ## live_view_heartbeat_interval {#live-view-heartbeat-interval} -Sets the heartbeat interval in seconds to indicate [live view](../../sql-reference/statements/create/view.md#live-view) is alive . +Sets the heartbeat interval in seconds to indicate [live view](../../sql-reference/statements/create/view.md/#live-view) is alive . Default value: `15`. ## max_live_view_insert_blocks_before_refresh {#max-live-view-insert-blocks-before-refresh} -Sets the maximum number of inserted blocks after which mergeable blocks are dropped and query for [live view](../../sql-reference/statements/create/view.md#live-view) is re-executed. +Sets the maximum number of inserted blocks after which mergeable blocks are dropped and query for [live view](../../sql-reference/statements/create/view.md/#live-view) is re-executed. Default value: `64`. ## periodic_live_view_refresh {#periodic-live-view-refresh} -Sets the interval in seconds after which periodically refreshed [live view](../../sql-reference/statements/create/view.md#live-view) is forced to refresh. +Sets the interval in seconds after which periodically refreshed [live view](../../sql-reference/statements/create/view.md/#live-view) is forced to refresh. Default value: `60`. @@ -2670,7 +2670,7 @@ Default value: 180. ## check_query_single_value_result {#check_query_single_value_result} -Defines the level of detail for the [CHECK TABLE](../../sql-reference/statements/check-table.md#checking-mergetree-tables) query result for `MergeTree` family engines . +Defines the level of detail for the [CHECK TABLE](../../sql-reference/statements/check-table.md/#checking-mergetree-tables) query result for `MergeTree` family engines . Possible values: @@ -2681,7 +2681,7 @@ Default value: `0`. ## prefer_column_name_to_alias {#prefer-column-name-to-alias} -Enables or disables using the original column names instead of aliases in query expressions and clauses. It especially matters when alias is the same as the column name, see [Expression Aliases](../../sql-reference/syntax.md#notes-on-usage). Enable this setting to make aliases syntax rules in ClickHouse more compatible with most other database engines. +Enables or disables using the original column names instead of aliases in query expressions and clauses. It especially matters when alias is the same as the column name, see [Expression Aliases](../../sql-reference/syntax.md/#notes-on-usage). Enable this setting to make aliases syntax rules in ClickHouse more compatible with most other database engines. Possible values: @@ -2725,7 +2725,7 @@ Result: ## limit {#limit} -Sets the maximum number of rows to get from the query result. It adjusts the value set by the [LIMIT](../../sql-reference/statements/select/limit.md#limit-clause) clause, so that the limit, specified in the query, cannot exceed the limit, set by this setting. +Sets the maximum number of rows to get from the query result. It adjusts the value set by the [LIMIT](../../sql-reference/statements/select/limit.md/#limit-clause) clause, so that the limit, specified in the query, cannot exceed the limit, set by this setting. Possible values: @@ -2736,7 +2736,7 @@ Default value: `0`. ## offset {#offset} -Sets the number of rows to skip before starting to return rows from the query. It adjusts the offset set by the [OFFSET](../../sql-reference/statements/select/offset.md#offset-fetch) clause, so that these two values are summarized. +Sets the number of rows to skip before starting to return rows from the query. It adjusts the offset set by the [OFFSET](../../sql-reference/statements/select/offset.md/#offset-fetch) clause, so that these two values are summarized. Possible values: @@ -2773,7 +2773,7 @@ Result: ## optimize_syntax_fuse_functions {#optimize_syntax_fuse_functions} -Enables to fuse aggregate functions with identical argument. It rewrites query contains at least two aggregate functions from [sum](../../sql-reference/aggregate-functions/reference/sum.md#agg_function-sum), [count](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) or [avg](../../sql-reference/aggregate-functions/reference/avg.md#agg_function-avg) with identical argument to [sumCount](../../sql-reference/aggregate-functions/reference/sumcount.md#agg_function-sumCount). +Enables to fuse aggregate functions with identical argument. It rewrites query contains at least two aggregate functions from [sum](../../sql-reference/aggregate-functions/reference/sum.md/#agg_function-sum), [count](../../sql-reference/aggregate-functions/reference/count.md/#agg_function-count) or [avg](../../sql-reference/aggregate-functions/reference/avg.md/#agg_function-avg) with identical argument to [sumCount](../../sql-reference/aggregate-functions/reference/sumcount.md/#agg_function-sumCount). Possible values: @@ -2932,7 +2932,7 @@ If the setting is set to `0`, the table function does not make Nullable columns ## allow_experimental_projection_optimization {#allow-experimental-projection-optimization} -Enables or disables [projection](../../engines/table-engines/mergetree-family/mergetree.md#projections) optimization when processing `SELECT` queries. +Enables or disables [projection](../../engines/table-engines/mergetree-family/mergetree.md/#projections) optimization when processing `SELECT` queries. Possible values: @@ -2943,7 +2943,7 @@ Default value: `1`. ## force_optimize_projection {#force-optimize-projection} -Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow-experimental-projection-optimization) setting). +Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md/#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow-experimental-projection-optimization) setting). Possible values: @@ -2978,7 +2978,7 @@ Default value: `120` seconds. ## regexp_max_matches_per_row {#regexp-max-matches-per-row} -Sets the maximum number of matches for a single regular expression per row. Use it to protect against memory overload when using greedy regular expression in the [extractAllGroupsHorizontal](../../sql-reference/functions/string-search-functions.md#extractallgroups-horizontal) function. +Sets the maximum number of matches for a single regular expression per row. Use it to protect against memory overload when using greedy regular expression in the [extractAllGroupsHorizontal](../../sql-reference/functions/string-search-functions.md/#extractallgroups-horizontal) function. Possible values: @@ -3010,7 +3010,7 @@ Default value: `1`. ## short_circuit_function_evaluation {#short-circuit-function-evaluation} -Allows calculating the [if](../../sql-reference/functions/conditional-functions.md#if), [multiIf](../../sql-reference/functions/conditional-functions.md#multiif), [and](../../sql-reference/functions/logical-functions.md#logical-and-function), and [or](../../sql-reference/functions/logical-functions.md#logical-or-function) functions according to a [short scheme](https://en.wikipedia.org/wiki/Short-circuit_evaluation). This helps optimize the execution of complex expressions in these functions and prevent possible exceptions (such as division by zero when it is not expected). +Allows calculating the [if](../../sql-reference/functions/conditional-functions.md/#if), [multiIf](../../sql-reference/functions/conditional-functions.md/#multiif), [and](../../sql-reference/functions/logical-functions.md/#logical-and-function), and [or](../../sql-reference/functions/logical-functions.md/#logical-or-function) functions according to a [short scheme](https://en.wikipedia.org/wiki/Short-circuit_evaluation). This helps optimize the execution of complex expressions in these functions and prevent possible exceptions (such as division by zero when it is not expected). Possible values: @@ -3022,7 +3022,7 @@ Default value: `enable`. ## max_hyperscan_regexp_length {#max-hyperscan-regexp-length} -Defines the maximum length for each regular expression in the [hyperscan multi-match functions](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn). +Defines the maximum length for each regular expression in the [hyperscan multi-match functions](../../sql-reference/functions/string-search-functions.md/#multimatchanyhaystack-pattern1-pattern2-patternn). Possible values: @@ -3065,7 +3065,7 @@ Exception: Regexp length too large. ## max_hyperscan_regexp_total_length {#max-hyperscan-regexp-total-length} -Sets the maximum length total of all regular expressions in each [hyperscan multi-match function](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn). +Sets the maximum length total of all regular expressions in each [hyperscan multi-match function](../../sql-reference/functions/string-search-functions.md/#multimatchanyhaystack-pattern1-pattern2-patternn). Possible values: @@ -3142,8 +3142,8 @@ Result: ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Enables or disables returning results of type: -- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). -- `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). +- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md/#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md/#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md/#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md/#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md/#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md/#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md/#tolastdayofmonth). +- `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md/#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md/#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md/#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md/#timeslot). Possible values: @@ -3167,7 +3167,7 @@ Default value: `1`. ## optimize_move_to_prewhere_if_final {#optimize_move_to_prewhere_if_final} -Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries with [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier. +Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries with [FINAL](../../sql-reference/statements/select/from.md/#select-from-final) modifier. Works only for [*MergeTree](../../engines/table-engines/mergetree-family/index.md) tables. @@ -3184,7 +3184,7 @@ Default value: `0`. ## describe_include_subcolumns {#describe_include_subcolumns} -Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) or an [Array](../../sql-reference/data-types/array.md#array-size) data type. +Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md/#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md/#finding-null) or an [Array](../../sql-reference/data-types/array.md/#array-size) data type. Possible values: @@ -3283,7 +3283,7 @@ Default value: `0`. ## alter_partition_verbose_result {#alter-partition-verbose-result} Enables or disables the display of information about the parts to which the manipulation operations with partitions and parts have been successfully applied. -Applicable to [ATTACH PARTITION|PART](../../sql-reference/statements/alter/partition.md#alter_attach-partition) and to [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md#alter_freeze-partition). +Applicable to [ATTACH PARTITION|PART](../../sql-reference/statements/alter/partition.md/#alter_attach-partition) and to [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md/#alter_freeze-partition). Possible values: @@ -3418,11 +3418,11 @@ When writing data, ClickHouse throws an exception if input data contain columns Supported formats: -- [JSONEachRow](../../interfaces/formats.md#jsoneachrow) -- [TSKV](../../interfaces/formats.md#tskv) +- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) +- [TSKV](../../interfaces/formats.md/#tskv) - All formats with suffixes WithNames/WithNamesAndTypes -- [JSONColumns](../../interfaces/formats.md#jsoncolumns) -- [MySQLDump](../../interfaces/formats.md#mysqldump) +- [JSONColumns](../../interfaces/formats.md/#jsoncolumns) +- [MySQLDump](../../interfaces/formats.md/#mysqldump) Possible values: @@ -3439,18 +3439,18 @@ To improve insert performance, we recommend disabling this check if you are sure Supported formats: -- [CSVWithNames](../../interfaces/formats.md#csvwithnames) -- [CSVWithNamesAndTypes](../../interfaces/formats.md#csvwithnamesandtypes) -- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) -- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md#tabseparatedwithnamesandtypes) -- [JSONCompactEachRowWithNames](../../interfaces/formats.md#jsoncompacteachrowwithnames) -- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompacteachrowwithnamesandtypes) -- [JSONCompactStringsEachRowWithNames](../../interfaces/formats.md#jsoncompactstringseachrowwithnames) -- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompactstringseachrowwithnamesandtypes) -- [RowBinaryWithNames](../../interfaces/formats.md#rowbinarywithnames) -- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes) -- [CustomSeparatedWithNames](../../interfaces/formats.md#customseparatedwithnames) -- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md#customseparatedwithnamesandtypes) +- [CSVWithNames](../../interfaces/formats.md/#csvwithnames) +- [CSVWithNamesAndTypes](../../interfaces/formats.md/#csvwithnamesandtypes) +- [TabSeparatedWithNames](../../interfaces/formats.md/#tabseparatedwithnames) +- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md/#tabseparatedwithnamesandtypes) +- [JSONCompactEachRowWithNames](../../interfaces/formats.md/#jsoncompacteachrowwithnames) +- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md/#jsoncompacteachrowwithnamesandtypes) +- [JSONCompactStringsEachRowWithNames](../../interfaces/formats.md/#jsoncompactstringseachrowwithnames) +- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md/#jsoncompactstringseachrowwithnamesandtypes) +- [RowBinaryWithNames](../../interfaces/formats.md/#rowbinarywithnames) +- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md/#rowbinarywithnamesandtypes) +- [CustomSeparatedWithNames](../../interfaces/formats.md/#customseparatedwithnames) +- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md/#customseparatedwithnamesandtypes) Possible values: @@ -3465,12 +3465,12 @@ Controls whether format parser should check if data types from the input data ma Supported formats: -- [CSVWithNamesAndTypes](../../interfaces/formats.md#csvwithnamesandtypes) -- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md#tabseparatedwithnamesandtypes) -- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompacteachrowwithnamesandtypes) -- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompactstringseachrowwithnamesandtypes) -- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes-rowbinarywithnamesandtypes) -- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md#customseparatedwithnamesandtypes) +- [CSVWithNamesAndTypes](../../interfaces/formats.md/#csvwithnamesandtypes) +- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md/#tabseparatedwithnamesandtypes) +- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md/#jsoncompacteachrowwithnamesandtypes) +- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md/#jsoncompactstringseachrowwithnamesandtypes) +- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md/#rowbinarywithnamesandtypes-rowbinarywithnamesandtypes) +- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md/#customseparatedwithnamesandtypes) Possible values: @@ -3481,7 +3481,7 @@ Default value: 1. ## input_format_defaults_for_omitted_fields {#input_format_defaults_for_omitted_fields} -When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv), [TabSeparated](../../interfaces/formats.md#tabseparated) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes. +When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow), [CSV](../../interfaces/formats.md/#csv), [TabSeparated](../../interfaces/formats.md/#tabseparated) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes. :::note When this option is enabled, extended table metadata are sent from server to client. It consumes additional computing resources on the server and can reduce performance. @@ -3496,7 +3496,7 @@ Default value: 1. ## input_format_null_as_default {#input_format_null_as_default} -Enables or disables the initialization of [NULL](../../sql-reference/syntax.md#null-literal) fields with [default values](../../sql-reference/statements/create/table.md#create-default-values), if data type of these fields is not [nullable](../../sql-reference/data-types/nullable.md#data_type-nullable). +Enables or disables the initialization of [NULL](../../sql-reference/syntax.md/#null-literal) fields with [default values](../../sql-reference/statements/create/table.md/#create-default-values), if data type of these fields is not [nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable). If column type is not nullable and this setting is disabled, then inserting `NULL` causes an exception. If column type is nullable, then `NULL` values are inserted as is, regardless of this setting. This setting is applicable to [INSERT ... VALUES](../../sql-reference/statements/insert-into.md) queries for text input formats. @@ -3663,7 +3663,7 @@ Enabled by default ## insert_distributed_one_random_shard {#insert_distributed_one_random_shard} -Enables or disables random shard insertion into a [Distributed](../../engines/table-engines/special/distributed.md#distributed) table when there is no distributed key. +Enables or disables random shard insertion into a [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table when there is no distributed key. By default, when inserting data into a `Distributed` table with more than one shard, the ClickHouse server will reject any insertion request if there is no distributed key. When `insert_distributed_one_random_shard = 1`, insertions are allowed and data is forwarded randomly among all shards. @@ -3682,7 +3682,7 @@ Enables or disables the insertion of JSON data with nested objects. Supported formats: -- [JSONEachRow](../../interfaces/formats.md#jsoneachrow) +- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) Possible values: @@ -3693,7 +3693,7 @@ Default value: 0. See also: -- [Usage of Nested Structures](../../interfaces/formats.md#jsoneachrow-nested) with the `JSONEachRow` format. +- [Usage of Nested Structures](../../interfaces/formats.md/#jsoneachrow-nested) with the `JSONEachRow` format. ### input_format_json_read_bools_as_numbers {#input_format_json_read_bools_as_numbers} @@ -3716,7 +3716,7 @@ Enabled by default. ### output_format_json_quote_64bit_integers {#output_format_json_quote_64bit_integers} -Controls quoting of 64-bit or bigger [integers](../../sql-reference/data-types/int-uint.md) (like `UInt64` or `Int128`) when they are output in a [JSON](../../interfaces/formats.md#json) format. +Controls quoting of 64-bit or bigger [integers](../../sql-reference/data-types/int-uint.md) (like `UInt64` or `Int128`) when they are output in a [JSON](../../interfaces/formats.md/#json) format. Such integers are enclosed in quotes by default. This behavior is compatible with most JavaScript implementations. Possible values: @@ -3734,7 +3734,7 @@ Disabled by default. ### output_format_json_quote_denormals {#output_format_json_quote_denormals} -Enables `+nan`, `-nan`, `+inf`, `-inf` outputs in [JSON](../../interfaces/formats.md#json) output format. +Enables `+nan`, `-nan`, `+inf`, `-inf` outputs in [JSON](../../interfaces/formats.md/#json) output format. Possible values: @@ -3851,7 +3851,7 @@ Disabled by default. ### output_format_json_array_of_rows {#output_format_json_array_of_rows} -Enables the ability to output all rows as a JSON array in the [JSONEachRow](../../interfaces/formats.md#jsoneachrow) format. +Enables the ability to output all rows as a JSON array in the [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) format. Possible values: @@ -3904,7 +3904,7 @@ Disabled by default. ### format_json_object_each_row_column_for_object_name {#format_json_object_each_row_column_for_object_name} -The name of column that will be used for storing/writing object names in [JSONObjectEachRow](../../interfaces/formats.md#jsonobjecteachrow) format. +The name of column that will be used for storing/writing object names in [JSONObjectEachRow](../../interfaces/formats.md/#jsonobjecteachrow) format. Column type should be String. If value is empty, default names `row_{i}`will be used for object names. Default value: ''. @@ -4005,7 +4005,7 @@ Disabled by default. ### format_tsv_null_representation {#format_tsv_null_representation} -Defines the representation of `NULL` for [TSV](../../interfaces/formats.md#tabseparated) output and input formats. User can set any string as a value, for example, `My NULL`. +Defines the representation of `NULL` for [TSV](../../interfaces/formats.md/#tabseparated) output and input formats. User can set any string as a value, for example, `My NULL`. Default value: `\N`. @@ -4159,7 +4159,7 @@ Default value: `0`. ### format_csv_null_representation {#format_csv_null_representation} -Defines the representation of `NULL` for [CSV](../../interfaces/formats.md#csv) output and input formats. User can set any string as a value, for example, `My NULL`. +Defines the representation of `NULL` for [CSV](../../interfaces/formats.md/#csv) output and input formats. User can set any string as a value, for example, `My NULL`. Default value: `\N`. @@ -4198,7 +4198,7 @@ My NULL ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} -Enables or disables the full SQL parser if the fast stream parser can’t parse the data. This setting is used only for the [Values](../../interfaces/formats.md#data-format-values) format at the data insertion. For more information about syntax parsing, see the [Syntax](../../sql-reference/syntax.md) section. +Enables or disables the full SQL parser if the fast stream parser can’t parse the data. This setting is used only for the [Values](../../interfaces/formats.md/#data-format-values) format at the data insertion. For more information about syntax parsing, see the [Syntax](../../sql-reference/syntax.md) section. Possible values: @@ -4248,7 +4248,7 @@ Ok. ### input_format_values_deduce_templates_of_expressions {#input_format_values_deduce_templates_of_expressions} -Enables or disables template deduction for SQL expressions in [Values](../../interfaces/formats.md#data-format-values) format. It allows parsing and interpreting expressions in `Values` much faster if expressions in consecutive rows have the same structure. ClickHouse tries to deduce the template of an expression, parse the following rows using this template and evaluate the expression on a batch of successfully parsed rows. +Enables or disables template deduction for SQL expressions in [Values](../../interfaces/formats.md/#data-format-values) format. It allows parsing and interpreting expressions in `Values` much faster if expressions in consecutive rows have the same structure. ClickHouse tries to deduce the template of an expression, parse the following rows using this template and evaluate the expression on a batch of successfully parsed rows. Possible values: @@ -4293,7 +4293,7 @@ Default value: 1. ### input_format_arrow_import_nested {#input_format_arrow_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Arrow](../../interfaces/formats.md#data_types-matching-arrow) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Arrow](../../interfaces/formats.md/#data_types-matching-arrow) input format. Possible values: @@ -4322,7 +4322,7 @@ Disabled by default. ### output_format_arrow_low_cardinality_as_dictionary {#output_format_arrow_low_cardinality_as_dictionary} -Allows to convert the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) type to the `DICTIONARY` type of the [Arrow](../../interfaces/formats.md#data-format-arrow) format for `SELECT` queries. +Allows to convert the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) type to the `DICTIONARY` type of the [Arrow](../../interfaces/formats.md/#data-format-arrow) format for `SELECT` queries. Possible values: @@ -4341,7 +4341,7 @@ Disabled by default. ### input_format_orc_import_nested {#input_format_orc_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [ORC](../../interfaces/formats.md#data-format-orc) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [ORC](../../interfaces/formats.md/#data-format-orc) input format. Possible values: @@ -4384,7 +4384,7 @@ Disabled by default. ## input_format_parquet_import_nested {#input_format_parquet_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Parquet](../../interfaces/formats.md#data-format-parquet) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Parquet](../../interfaces/formats.md/#data-format-parquet) input format. Possible values: @@ -4481,7 +4481,7 @@ Disabled by default. ### input_format_avro_allow_missing_fields {#input_format_avro_allow_missing_fields} -Enables using fields that are not specified in [Avro](../../interfaces/formats.md#data-format-avro) or [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) format schema. When a field is not found in the schema, ClickHouse uses the default value instead of throwing an exception. +Enables using fields that are not specified in [Avro](../../interfaces/formats.md/#data-format-avro) or [AvroConfluent](../../interfaces/formats.md/#data-format-avro-confluent) format schema. When a field is not found in the schema, ClickHouse uses the default value instead of throwing an exception. Possible values: @@ -4492,7 +4492,7 @@ Default value: 0. ### format_avro_schema_registry_url {#format_avro_schema_registry_url} -Sets [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html) URL to use with [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) format. +Sets [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html) URL to use with [AvroConfluent](../../interfaces/formats.md/#data-format-avro-confluent) format. Default value: `Empty`. @@ -4549,7 +4549,7 @@ Default value: `250`. ### output_format_pretty_max_value_width {#output_format_pretty_max_value_width} -Limits the width of value displayed in [Pretty](../../interfaces/formats.md#pretty) formats. If the value width exceeds the limit, the value is cut. +Limits the width of value displayed in [Pretty](../../interfaces/formats.md/#pretty) formats. If the value width exceeds the limit, the value is cut. Possible values: @@ -4625,7 +4625,7 @@ SELECT * FROM a; ### output_format_pretty_row_numbers {#output_format_pretty_row_numbers} -Adds row numbers to output in the [Pretty](../../interfaces/formats.md#pretty) format. +Adds row numbers to output in the [Pretty](../../interfaces/formats.md/#pretty) format. Possible values: @@ -4670,52 +4670,52 @@ Delimiter between rows (for Template format). ### format_custom_escaping_rule {#format_custom_escaping_rule} -Sets the field escaping rule for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the field escaping rule for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Possible values: -- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md#tabseparated). -- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md#data-format-values). -- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md#csv). -- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md#jsoneachrow). -- `'XML'` — Similarly to [XML](../../interfaces/formats.md#xml). -- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md#tabseparatedraw). +- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md/#tabseparated). +- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md/#data-format-values). +- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md/#csv). +- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow). +- `'XML'` — Similarly to [XML](../../interfaces/formats.md/#xml). +- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md/#tabseparatedraw). Default value: `'Escaped'`. ### format_custom_field_delimiter {#format_custom_field_delimiter} -Sets the character that is interpreted as a delimiter between the fields for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a delimiter between the fields for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `'\t'`. ### format_custom_row_before_delimiter {#format_custom_row_before_delimiter} -Sets the character that is interpreted as a delimiter before the field of the first column for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a delimiter before the field of the first column for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `''`. ### format_custom_row_after_delimiter {#format_custom_row_after_delimiter} -Sets the character that is interpreted as a delimiter after the field of the last column for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a delimiter after the field of the last column for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `'\n'`. ### format_custom_row_between_delimiter {#format_custom_row_between_delimiter} -Sets the character that is interpreted as a delimiter between the rows for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a delimiter between the rows for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `''`. ### format_custom_result_before_delimiter {#format_custom_result_before_delimiter} -Sets the character that is interpreted as a prefix before the result set for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a prefix before the result set for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `''`. ### format_custom_result_after_delimiter {#format_custom_result_after_delimiter} -Sets the character that is interpreted as a suffix after the result set for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a suffix after the result set for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `''`. @@ -4727,12 +4727,12 @@ Field escaping rule. Possible values: -- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md#tabseparated). -- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md#data-format-values). -- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md#csv). -- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md#jsoneachrow). -- `'XML'` — Similarly to [XML](../../interfaces/formats.md#xml). -- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md#tabseparatedraw). +- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md/#tabseparated). +- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md/#data-format-values). +- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md/#csv). +- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow). +- `'XML'` — Similarly to [XML](../../interfaces/formats.md/#xml). +- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md/#tabseparatedraw). Default value: `Raw`. @@ -4746,7 +4746,7 @@ Disabled by default. ### format_capn_proto_enum_comparising_mode {#format_capn_proto_enum_comparising_mode} -Determines how to map ClickHouse `Enum` data type and [CapnProto](../../interfaces/formats.md#capnproto) `Enum` data type from schema. +Determines how to map ClickHouse `Enum` data type and [CapnProto](../../interfaces/formats.md/#capnproto) `Enum` data type from schema. Possible values: diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 43623577e66..203fe4e42d2 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -7,13 +7,13 @@ title: "External Disks for Storing Data" Data, processed in ClickHouse, is usually stored in the local file system — on the same machine with the ClickHouse server. That requires large-capacity disks, which can be expensive enough. To avoid that you can store the data remotely — on [Amazon S3](https://aws.amazon.com/s3/) disks or in the Hadoop Distributed File System ([HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)). -To work with data stored on `Amazon S3` disks use [S3](../engines/table-engines/integrations/s3.md) table engine, and to work with data in the Hadoop Distributed File System — [HDFS](../engines/table-engines/integrations/hdfs.md) table engine. +To work with data stored on `Amazon S3` disks use [S3](/docs/en/engines/table-engines/integrations/s3.md) table engine, and to work with data in the Hadoop Distributed File System — [HDFS](/docs/en/engines/table-engines/integrations/hdfs.md) table engine. To load data from a web server with static files use a disk with type [web](#storing-data-on-webserver). ## Configuring HDFS {#configuring-hdfs} -[MergeTree](../engines/table-engines/mergetree-family/mergetree.md) and [Log](../engines/table-engines/log-family/log.md) family table engines can store data to HDFS using a disk with type `HDFS`. +[MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) and [Log](/docs/en/engines/table-engines/log-family/log.md) family table engines can store data to HDFS using a disk with type `HDFS`. Configuration markup: @@ -53,7 +53,7 @@ Optional parameters: ## Using Virtual File System for Data Encryption {#encrypted-virtual-file-system} -You can encrypt the data stored on [S3](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-s3), or [HDFS](#configuring-hdfs) external disks, or on a local disk. To turn on the encryption mode, in the configuration file you must define a disk with the type `encrypted` and choose a disk on which the data will be saved. An `encrypted` disk ciphers all written files on the fly, and when you read files from an `encrypted` disk it deciphers them automatically. So you can work with an `encrypted` disk like with a normal one. +You can encrypt the data stored on [S3](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3), or [HDFS](#configuring-hdfs) external disks, or on a local disk. To turn on the encryption mode, in the configuration file you must define a disk with the type `encrypted` and choose a disk on which the data will be saved. An `encrypted` disk ciphers all written files on the fly, and when you read files from an `encrypted` disk it deciphers them automatically. So you can work with an `encrypted` disk like with a normal one. Example of disk configuration: @@ -80,14 +80,14 @@ Required parameters: - `type` — `encrypted`. Otherwise the encrypted disk is not created. - `disk` — Type of disk for data storage. -- `key` — The key for encryption and decryption. Type: [Uint64](../sql-reference/data-types/int-uint.md). You can use `key_hex` parameter to encrypt in hexadecimal form. +- `key` — The key for encryption and decryption. Type: [Uint64](/docs/en/sql-reference/data-types/int-uint.md). You can use `key_hex` parameter to encrypt in hexadecimal form. You can specify multiple keys using the `id` attribute (see example above). Optional parameters: - `path` — Path to the location on the disk where the data will be saved. If not specified, the data will be saved in the root directory. - `current_key_id` — The key used for encryption. All the specified keys can be used for decryption, and you can always switch to another key while maintaining access to previously encrypted data. -- `algorithm` — [Algorithm](../sql-reference/statements/create/table.md#create-query-encryption-codecs) for encryption. Possible values: `AES_128_CTR`, `AES_192_CTR` or `AES_256_CTR`. Default value: `AES_128_CTR`. The key length depends on the algorithm: `AES_128_CTR` — 16 bytes, `AES_192_CTR` — 24 bytes, `AES_256_CTR` — 32 bytes. +- `algorithm` — [Algorithm](/docs/en/sql-reference/statements/create/table.md/#create-query-encryption-codecs) for encryption. Possible values: `AES_128_CTR`, `AES_192_CTR` or `AES_256_CTR`. Default value: `AES_128_CTR`. The key length depends on the algorithm: `AES_128_CTR` — 16 bytes, `AES_192_CTR` — 24 bytes, `AES_256_CTR` — 32 bytes. Example of disk configuration: @@ -265,9 +265,9 @@ Cache profile events: There is a tool `clickhouse-static-files-uploader`, which prepares a data directory for a given table (`SELECT data_paths FROM system.tables WHERE name = 'table_name'`). For each table you need, you get a directory of files. These files can be uploaded to, for example, a web server with static files. After this preparation, you can load this table into any ClickHouse server via `DiskWeb`. -This is a read-only disk. Its data is only read and never modified. A new table is loaded to this disk via `ATTACH TABLE` query (see example below). Local disk is not actually used, each `SELECT` query will result in a `http` request to fetch required data. All modification of the table data will result in an exception, i.e. the following types of queries are not allowed: [CREATE TABLE](../sql-reference/statements/create/table.md), [ALTER TABLE](../sql-reference/statements/alter/index.md), [RENAME TABLE](../sql-reference/statements/rename.md#misc_operations-rename_table), [DETACH TABLE](../sql-reference/statements/detach.md) and [TRUNCATE TABLE](../sql-reference/statements/truncate.md). +This is a read-only disk. Its data is only read and never modified. A new table is loaded to this disk via `ATTACH TABLE` query (see example below). Local disk is not actually used, each `SELECT` query will result in a `http` request to fetch required data. All modification of the table data will result in an exception, i.e. the following types of queries are not allowed: [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md), [ALTER TABLE](/docs/en/sql-reference/statements/alter/index.md), [RENAME TABLE](/docs/en/sql-reference/statements/rename.md/#misc_operations-rename_table), [DETACH TABLE](/docs/en/sql-reference/statements/detach.md) and [TRUNCATE TABLE](/docs/en/sql-reference/statements/truncate.md). -Web server storage is supported only for the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) and [Log](../engines/table-engines/log-family/log.md) engine families. To access the data stored on a `web` disk, use the [storage_policy](../engines/table-engines/mergetree-family/mergetree.md#terms) setting when executing the query. For example, `ATTACH TABLE table_web UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'`. +Web server storage is supported only for the [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) and [Log](/docs/en/engines/table-engines/log-family/log.md) engine families. To access the data stored on a `web` disk, use the [storage_policy](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#terms) setting when executing the query. For example, `ATTACH TABLE table_web UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'`. A ready test case. You need to add this configuration to config: @@ -451,7 +451,7 @@ Optional parameters: - `remote_fs_read_backoff_threashold` — The maximum wait time when trying to read data for remote disk. Default value: `10000` seconds. - `remote_fs_read_backoff_max_tries` — The maximum number of attempts to read with backoff. Default value: `5`. -If a query fails with an exception `DB:Exception Unreachable URL`, then you can try to adjust the settings: [http_connection_timeout](../operations/settings/settings.md#http_connection_timeout), [http_receive_timeout](../operations/settings/settings.md#http_receive_timeout), [keep_alive_timeout](../operations/server-configuration-parameters/settings.md#keep-alive-timeout). +If a query fails with an exception `DB:Exception Unreachable URL`, then you can try to adjust the settings: [http_connection_timeout](/docs/en/operations/settings/settings.md/#http_connection_timeout), [http_receive_timeout](/docs/en/operations/settings/settings.md/#http_receive_timeout), [keep_alive_timeout](/docs/en/operations/server-configuration-parameters/settings.md/#keep-alive-timeout). To get files for upload run: `clickhouse static-files-disk-uploader --metadata-path --output-dir ` (`--metadata-path` can be found in query `SELECT data_paths FROM system.tables WHERE name = 'table_name'`). @@ -460,7 +460,7 @@ When loading files by `endpoint`, they must be loaded into `/store/` p If URL is not reachable on disk load when the server is starting up tables, then all errors are caught. If in this case there were errors, tables can be reloaded (become visible) via `DETACH TABLE table_name` -> `ATTACH TABLE table_name`. If metadata was successfully loaded at server startup, then tables are available straight away. -Use [http_max_single_read_retries](../operations/settings/settings.md#http-max-single-read-retries) setting to limit the maximum number of retries during a single HTTP read. +Use [http_max_single_read_retries](/docs/en/operations/settings/settings.md/#http-max-single-read-retries) setting to limit the maximum number of retries during a single HTTP read. ## Zero-copy Replication (not ready for production) {#zero-copy} diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index e08a727a62a..5fc302cad34 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -1,7 +1,8 @@ --- slug: /en/operations/system-tables/ sidebar_position: 52 -sidebar_label: System Tables +sidebar_label: Overview +pagination_next: 'en/operations/system-tables/asynchronous_metric_log' --- # System Tables @@ -72,4 +73,3 @@ If procfs is supported and enabled on the system, ClickHouse server collects the - `OSReadBytes` - `OSWriteBytes` -[Original article](https://clickhouse.com/docs/en/operations/system-tables/) diff --git a/docs/en/operations/system-tables/mutations.md b/docs/en/operations/system-tables/mutations.md index 45447f3644e..782d7c42ad2 100644 --- a/docs/en/operations/system-tables/mutations.md +++ b/docs/en/operations/system-tables/mutations.md @@ -3,31 +3,31 @@ slug: /en/operations/system-tables/mutations --- # mutations -The table contains information about [mutations](../../sql-reference/statements/alter/index.md#mutations) of [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row. +The table contains information about [mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) of [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row. Columns: -- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database to which the mutation was applied. +- `database` ([String](/docs/en/sql-reference/data-types/string.md)) — The name of the database to which the mutation was applied. -- `table` ([String](../../sql-reference/data-types/string.md)) — The name of the table to which the mutation was applied. +- `table` ([String](/docs/en/sql-reference/data-types/string.md)) — The name of the table to which the mutation was applied. -- `mutation_id` ([String](../../sql-reference/data-types/string.md)) — The ID of the mutation. For replicated tables these IDs correspond to znode names in the `/mutations/` directory in ClickHouse Keeper. For non-replicated tables the IDs correspond to file names in the data directory of the table. +- `mutation_id` ([String](/docs/en/sql-reference/data-types/string.md)) — The ID of the mutation. For replicated tables these IDs correspond to znode names in the `/mutations/` directory in ClickHouse Keeper. For non-replicated tables the IDs correspond to file names in the data directory of the table. -- `command` ([String](../../sql-reference/data-types/string.md)) — The mutation command string (the part of the query after `ALTER TABLE [db.]table`). +- `command` ([String](/docs/en/sql-reference/data-types/string.md)) — The mutation command string (the part of the query after `ALTER TABLE [db.]table`). -- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the mutation command was submitted for execution. +- `create_time` ([Datetime](/docs/en/sql-reference/data-types/datetime.md)) — Date and time when the mutation command was submitted for execution. -- `block_numbers.partition_id` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — For mutations of replicated tables, the array contains the partitions' IDs (one record for each partition). For mutations of non-replicated tables the array is empty. +- `block_numbers.partition_id` ([Array](/docs/en/sql-reference/data-types/array.md)([String](/docs/en/sql-reference/data-types/string.md))) — For mutations of replicated tables, the array contains the partitions' IDs (one record for each partition). For mutations of non-replicated tables the array is empty. -- `block_numbers.number` ([Array](../../sql-reference/data-types/array.md)([Int64](../../sql-reference/data-types/int-uint.md))) — For mutations of replicated tables, the array contains one record for each partition, with the block number that was acquired by the mutation. Only parts that contain blocks with numbers less than this number will be mutated in the partition. +- `block_numbers.number` ([Array](/docs/en/sql-reference/data-types/array.md)([Int64](/docs/en/sql-reference/data-types/int-uint.md))) — For mutations of replicated tables, the array contains one record for each partition, with the block number that was acquired by the mutation. Only parts that contain blocks with numbers less than this number will be mutated in the partition. In non-replicated tables, block numbers in all partitions form a single sequence. This means that for mutations of non-replicated tables, the column will contain one record with a single block number acquired by the mutation. -- `parts_to_do_names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — An array of names of data parts that need to be mutated for the mutation to complete. +- `parts_to_do_names` ([Array](/docs/en/sql-reference/data-types/array.md)([String](/docs/en/sql-reference/data-types/string.md))) — An array of names of data parts that need to be mutated for the mutation to complete. -- `parts_to_do` ([Int64](../../sql-reference/data-types/int-uint.md)) — The number of data parts that need to be mutated for the mutation to complete. +- `parts_to_do` ([Int64](/docs/en/sql-reference/data-types/int-uint.md)) — The number of data parts that need to be mutated for the mutation to complete. -- `is_done` ([UInt8](../../sql-reference/data-types/int-uint.md)) — The flag whether the mutation is done or not. Possible values: +- `is_done` ([UInt8](/docs/en/sql-reference/data-types/int-uint.md)) — The flag whether the mutation is done or not. Possible values: - `1` if the mutation is completed, - `0` if the mutation is still in process. @@ -37,16 +37,16 @@ Even if `parts_to_do = 0` it is possible that a mutation of a replicated table i If there were problems with mutating some data parts, the following columns contain additional information: -- `latest_failed_part` ([String](../../sql-reference/data-types/string.md)) — The name of the most recent part that could not be mutated. +- `latest_failed_part` ([String](/docs/en/sql-reference/data-types/string.md)) — The name of the most recent part that could not be mutated. -- `latest_fail_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — The date and time of the most recent part mutation failure. +- `latest_fail_time` ([Datetime](/docs/en/sql-reference/data-types/datetime.md)) — The date and time of the most recent part mutation failure. -- `latest_fail_reason` ([String](../../sql-reference/data-types/string.md)) — The exception message that caused the most recent part mutation failure. +- `latest_fail_reason` ([String](/docs/en/sql-reference/data-types/string.md)) — The exception message that caused the most recent part mutation failure. **See Also** -- [Mutations](../../sql-reference/statements/alter/index.md#mutations) -- [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine -- [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) family +- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) +- [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) table engine +- [ReplicatedMergeTree](/docs/en/engines/table-engines/mergetree-family/replication.md) family [Original article](https://clickhouse.com/docs/en/operations/system-tables/mutations) diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index f1d60896a2e..cbabd9b27b1 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -9,7 +9,7 @@ Each row describes one data part. Columns: -- `partition` ([String](../../sql-reference/data-types/string.md)) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter) query. +- `partition` ([String](../../sql-reference/data-types/string.md)) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md/#query_language_queries_alter) query. Formats: @@ -75,7 +75,7 @@ Columns: - `primary_key_bytes_in_memory_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The amount of memory (in bytes) reserved for primary key values. -- `is_frozen` ([UInt8](../../sql-reference/data-types/int-uint.md)) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup does not exist. For more details, see [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md#alter_freeze-partition) +- `is_frozen` ([UInt8](../../sql-reference/data-types/int-uint.md)) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup does not exist. For more details, see [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md/#alter_freeze-partition) - `database` ([String](../../sql-reference/data-types/string.md)) – Name of the database. @@ -87,25 +87,25 @@ Columns: - `disk_name` ([String](../../sql-reference/data-types/string.md)) – Name of a disk that stores the data part. -- `hash_of_all_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) of compressed files. +- `hash_of_all_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md/#hash_functions-siphash128) of compressed files. -- `hash_of_uncompressed_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) of uncompressed files (files with marks, index file etc.). +- `hash_of_uncompressed_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md/#hash_functions-siphash128) of uncompressed files (files with marks, index file etc.). -- `uncompressed_hash_of_compressed_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) of data in the compressed files as if they were uncompressed. +- `uncompressed_hash_of_compressed_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md/#hash_functions-siphash128) of data in the compressed files as if they were uncompressed. -- `delete_ttl_info_min` ([DateTime](../../sql-reference/data-types/datetime.md)) — The minimum value of the date and time key for [TTL DELETE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `delete_ttl_info_min` ([DateTime](../../sql-reference/data-types/datetime.md)) — The minimum value of the date and time key for [TTL DELETE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -- `delete_ttl_info_max` ([DateTime](../../sql-reference/data-types/datetime.md)) — The maximum value of the date and time key for [TTL DELETE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `delete_ttl_info_max` ([DateTime](../../sql-reference/data-types/datetime.md)) — The maximum value of the date and time key for [TTL DELETE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -- `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). :::warning The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simpliest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. ::: -- `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the minimum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the minimum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -- `move_ttl_info.max` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the maximum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `move_ttl_info.max` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the maximum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). - `bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Alias for `bytes_on_disk`. @@ -166,6 +166,6 @@ move_ttl_info.max: [] **See Also** - [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md) -- [TTL for Columns and Tables](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) +- [TTL for Columns and Tables](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl) [Original article](https://clickhouse.com/docs/en/operations/system-tables/parts) diff --git a/docs/en/operations/system-tables/parts_columns.md b/docs/en/operations/system-tables/parts_columns.md index 68757ddfbff..d934e01f245 100644 --- a/docs/en/operations/system-tables/parts_columns.md +++ b/docs/en/operations/system-tables/parts_columns.md @@ -9,7 +9,7 @@ Each row describes one data part. Columns: -- `partition` ([String](../../sql-reference/data-types/string.md)) — The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter) query. +- `partition` ([String](../../sql-reference/data-types/string.md)) — The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md/#query_language_queries_alter) query. Formats: diff --git a/docs/en/operations/utilities/index.md b/docs/en/operations/utilities/index.md index df4af30768c..9de68923ea4 100644 --- a/docs/en/operations/utilities/index.md +++ b/docs/en/operations/utilities/index.md @@ -1,10 +1,11 @@ --- slug: /en/operations/utilities/ sidebar_position: 56 -sidebar_label: Utilities +sidebar_label: Overview +pagination_next: 'en/operations/utilities/clickhouse-copier' --- -# ClickHouse Utility +# ClickHouse Utilities - [clickhouse-local](../../operations/utilities/clickhouse-local.md) — Allows running SQL queries on data without starting the ClickHouse server, similar to how `awk` does this. - [clickhouse-copier](../../operations/utilities/clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index cc278465437..6bca0dbff42 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -35,11 +35,11 @@ These actions are described in detail below. ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after | FIRST] ``` -Adds a new column to the table with the specified `name`, `type`, [`codec`](../create/table.md#codecs) and `default_expr` (see the section [Default expressions](../../../sql-reference/statements/create/table.md#create-default-values)). +Adds a new column to the table with the specified `name`, `type`, [`codec`](../create/table.md/#codecs) and `default_expr` (see the section [Default expressions](/docs/en/sql-reference/statements/create/table.md/#create-default-values)). If the `IF NOT EXISTS` clause is included, the query won’t return an error if the column already exists. If you specify `AFTER name_after` (the name of another column), the column is added after the specified one in the list of table columns. If you want to add a column to the beginning of the table use the `FIRST` clause. Otherwise, the column is added to the end of the table. For a chain of actions, `name_after` can be the name of a column that is added in one of the previous actions. -Adding a column just changes the table structure, without performing any actions with data. The data does not appear on the disk after `ALTER`. If the data is missing for a column when reading from the table, it is filled in with default values (by performing the default expression if there is one, or using zeros or empty strings). The column appears on the disk after merging data parts (see [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)). +Adding a column just changes the table structure, without performing any actions with data. The data does not appear on the disk after `ALTER`. If the data is missing for a column when reading from the table, it is filled in with default values (by performing the default expression if there is one, or using zeros or empty strings). The column appears on the disk after merging data parts (see [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md)). This approach allows us to complete the `ALTER` query instantly, without increasing the volume of old data. @@ -76,7 +76,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. :::warning -You can’t delete a column if it is referenced by [materialized view](../../../sql-reference/statements/create/view.md#materialized). Otherwise, it returns an error. +You can’t delete a column if it is referenced by [materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. ::: Example: @@ -107,7 +107,7 @@ ALTER TABLE visits RENAME COLUMN webBrowser TO browser CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name ``` -Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](partition.md#how-to-set-partition-expression). +Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](partition.md/#how-to-set-partition-expression). If the `IF EXISTS` clause is specified, the query won’t return an error if the column does not exist. @@ -127,7 +127,7 @@ Adds a comment to the column. If the `IF EXISTS` clause is specified, the query Each column can have one comment. If a comment already exists for the column, a new comment overwrites the previous comment. -Comments are stored in the `comment_expression` column returned by the [DESCRIBE TABLE](../../../sql-reference/statements/describe-table.md) query. +Comments are stored in the `comment_expression` column returned by the [DESCRIBE TABLE](/docs/en/sql-reference/statements/describe-table.md) query. Example: @@ -152,15 +152,15 @@ This query changes the `name` column properties: - TTL -For examples of columns compression CODECS modifying, see [Column Compression Codecs](../create/table.md#codecs). +For examples of columns compression CODECS modifying, see [Column Compression Codecs](../create/table.md/#codecs). -For examples of columns TTL modifying, see [Column TTL](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-column-ttl). +For examples of columns TTL modifying, see [Column TTL](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-column-ttl). If the `IF EXISTS` clause is specified, the query won’t return an error if the column does not exist. The query also can change the order of the columns using `FIRST | AFTER` clause, see [ADD COLUMN](#alter_add-column) description. -When changing the type, values are converted as if the [toType](../../../sql-reference/functions/type-conversion-functions.md) functions were applied to them. If only the default expression is changed, the query does not do anything complex, and is completed almost instantly. +When changing the type, values are converted as if the [toType](/docs/en/sql-reference/functions/type-conversion-functions.md) functions were applied to them. If only the default expression is changed, the query does not do anything complex, and is completed almost instantly. Example: @@ -246,7 +246,7 @@ SELECT groupArray(x), groupArray(s) FROM tmp; **See Also** -- [MATERIALIZED](../../statements/create/table.md#materialized). +- [MATERIALIZED](/docs/en/sql-reference/statements/create/table.md/#materialized). ## Limitations @@ -254,8 +254,8 @@ The `ALTER` query lets you create and delete separate elements (columns) in nest There is no support for deleting columns in the primary key or the sampling key (columns that are used in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, you are allowed to add values to an Enum or to change a type from `DateTime` to `UInt32`). -If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](../../../sql-reference/statements/insert-into.md#insert_query_insert-select) query, then switch the tables using the [RENAME](../../../sql-reference/statements/rename.md#rename-table) query and delete the old table. You can use the [clickhouse-copier](../../../operations/utilities/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query. +If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](/docs/en/sql-reference/statements/insert-into.md/#insert_query_insert-select) query, then switch the tables using the [RENAME](/docs/en/sql-reference/statements/rename.md/#rename-table) query and delete the old table. You can use the [clickhouse-copier](/docs/en/operations/utilities/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query. The `ALTER` query blocks all reads and writes for the table. In other words, if a long `SELECT` is running at the time of the `ALTER` query, the `ALTER` query will wait for it to complete. At the same time, all new queries to the same table will wait while this `ALTER` is running. -For tables that do not store data themselves (such as [Merge](../../../sql-reference/statements/alter/index.md) and [Distributed](../../../sql-reference/statements/alter/index.md)), `ALTER` just changes the table structure, and does not change the structure of subordinate tables. For example, when running ALTER for a `Distributed` table, you will also need to run `ALTER` for the tables on all remote servers. +For tables that do not store data themselves (such as [Merge](/docs/en/sql-reference/statements/alter/index.md) and [Distributed](/docs/en/sql-reference/statements/alter/index.md)), `ALTER` just changes the table structure, and does not change the structure of subordinate tables. For example, when running ALTER for a `Distributed` table, you will also need to run `ALTER` for the tables on all remote servers. diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index ba5d01d9b4d..4dcab030d13 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -10,21 +10,21 @@ sidebar_label: DELETE ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr ``` -Deletes data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +Deletes data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). :::note -The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. `ALTER TABLE` is considered a heavyweight operation that requires the underlying data to be merged before it is deleted. For MergeTree tables, consider using the [`DELETE FROM` query](../delete.md), which performs a lightweight delete and can be considerably faster. +The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. `ALTER TABLE` is considered a heavyweight operation that requires the underlying data to be merged before it is deleted. For MergeTree tables, consider using the [`DELETE FROM` query](/docs/en/sql-reference/statements/delete.md), which performs a lightweight delete and can be considerably faster. ::: The `filter_expr` must be of type `UInt8`. The query deletes rows in the table for which this expression takes a non-zero value. One query can contain several commands separated by commas. -The synchronicity of the query processing is defined by the [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting. By default, it is asynchronous. +The synchronicity of the query processing is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting. By default, it is asynchronous. **See also** -- [Mutations](../../../sql-reference/statements/alter/index.md#mutations) -- [Synchronicity of ALTER Queries](../../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) -- [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting +- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) +- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) +- [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/index.md index 4027429cf0d..1c4d62f3190 100644 --- a/docs/en/sql-reference/statements/alter/index.md +++ b/docs/en/sql-reference/statements/alter/index.md @@ -8,43 +8,43 @@ sidebar_label: ALTER Most `ALTER TABLE` queries modify table settings or data: -- [COLUMN](../../../sql-reference/statements/alter/column.md) -- [PARTITION](../../../sql-reference/statements/alter/partition.md) -- [DELETE](../../../sql-reference/statements/alter/delete.md) -- [UPDATE](../../../sql-reference/statements/alter/update.md) -- [ORDER BY](../../../sql-reference/statements/alter/order-by.md) -- [INDEX](../../../sql-reference/statements/alter/index/index.md) -- [CONSTRAINT](../../../sql-reference/statements/alter/constraint.md) -- [TTL](../../../sql-reference/statements/alter/ttl.md) +- [COLUMN](/docs/en/sql-reference/statements/alter/column.md) +- [PARTITION](/docs/en/sql-reference/statements/alter/partition.md) +- [DELETE](/docs/en/sql-reference/statements/alter/delete.md) +- [UPDATE](/docs/en/sql-reference/statements/alter/update.md) +- [ORDER BY](/docs/en/sql-reference/statements/alter/order-by.md) +- [INDEX](/docs/en/sql-reference/statements/alter/skipping-index.md) +- [CONSTRAINT](/docs/en/sql-reference/statements/alter/constraint.md) +- [TTL](/docs/en/sql-reference/statements/alter/ttl.md) :::note -Most `ALTER TABLE` queries are supported only for [\*MergeTree](../../../engines/table-engines/mergetree-family/index.md) tables, as well as [Merge](../../../engines/table-engines/special/merge.md) and [Distributed](../../../engines/table-engines/special/distributed.md). +Most `ALTER TABLE` queries are supported only for [\*MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables, as well as [Merge](/docs/en/engines/table-engines/special/merge.md) and [Distributed](/docs/en/engines/table-engines/special/distributed.md). ::: These `ALTER` statements manipulate views: -- [ALTER TABLE ... MODIFY QUERY](../../../sql-reference/statements/alter/view.md) — Modifies a [Materialized view](../create/view.md#materialized) structure. -- [ALTER LIVE VIEW](../../../sql-reference/statements/alter/view.md#alter-live-view) — Refreshes a [Live view](../create/view.md#live-view). +- [ALTER TABLE ... MODIFY QUERY](/docs/en/sql-reference/statements/alter/view.md) — Modifies a [Materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized) structure. +- [ALTER LIVE VIEW](/docs/en/sql-reference/statements/alter/view.md/#alter-live-view) — Refreshes a [Live view](/docs/en/sql-reference/statements/create/view.md/#live-view). These `ALTER` statements modify entities related to role-based access control: -- [USER](../../../sql-reference/statements/alter/user.md) -- [ROLE](../../../sql-reference/statements/alter/role.md) -- [QUOTA](../../../sql-reference/statements/alter/quota.md) -- [ROW POLICY](../../../sql-reference/statements/alter/row-policy.md) -- [SETTINGS PROFILE](../../../sql-reference/statements/alter/settings-profile.md) +- [USER](/docs/en/sql-reference/statements/alter/user.md) +- [ROLE](/docs/en/sql-reference/statements/alter/role.md) +- [QUOTA](/docs/en/sql-reference/statements/alter/quota.md) +- [ROW POLICY](/docs/en/sql-reference/statements/alter/row-policy.md) +- [SETTINGS PROFILE](/docs/en/sql-reference/statements/alter/settings-profile.md) -[ALTER TABLE ... MODIFY COMMENT](../../../sql-reference/statements/alter/comment.md) statement adds, modifies, or removes comments to the table, regardless if it was set before or not. +[ALTER TABLE ... MODIFY COMMENT](/docs/en/sql-reference/statements/alter/comment.md) statement adds, modifies, or removes comments to the table, regardless if it was set before or not. ## Mutations -`ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE … DELETE](../../../sql-reference/statements/alter/delete.md) and [ALTER TABLE … UPDATE](../../../sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](../../../engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts. +`ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE … DELETE](/docs/en/sql-reference/statements/alter/delete.md) and [ALTER TABLE … UPDATE](/docs/en/sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts. For `*MergeTree` tables mutations execute by **rewriting whole data parts**. There is no atomicity - parts are substituted for mutated parts as soon as they are ready and a `SELECT` query that started executing during a mutation will see data from parts that have already been mutated along with data from parts that have not been mutated yet. Mutations are totally ordered by their creation order and are applied to each part in that order. Mutations are also partially ordered with `INSERT INTO` queries: data that was inserted into the table before the mutation was submitted will be mutated and data that was inserted after that will not be mutated. Note that mutations do not block inserts in any way. -A mutation query returns immediately after the mutation entry is added (in case of replicated tables to ZooKeeper, for non-replicated tables - to the filesystem). The mutation itself executes asynchronously using the system profile settings. To track the progress of mutations you can use the [`system.mutations`](../../../operations/system-tables/mutations.md#system_tables-mutations) table. A mutation that was successfully submitted will continue to execute even if ClickHouse servers are restarted. There is no way to roll back the mutation once it is submitted, but if the mutation is stuck for some reason it can be cancelled with the [`KILL MUTATION`](../../../sql-reference/statements/kill.md#kill-mutation) query. +A mutation query returns immediately after the mutation entry is added (in case of replicated tables to ZooKeeper, for non-replicated tables - to the filesystem). The mutation itself executes asynchronously using the system profile settings. To track the progress of mutations you can use the [`system.mutations`](/docs/en/operations/system-tables/mutations.md/#system_tables-mutations) table. A mutation that was successfully submitted will continue to execute even if ClickHouse servers are restarted. There is no way to roll back the mutation once it is submitted, but if the mutation is stuck for some reason it can be cancelled with the [`KILL MUTATION`](/docs/en/sql-reference/statements/kill.md/#kill-mutation) query. Entries for finished mutations are not deleted right away (the number of preserved entries is determined by the `finished_mutations_to_keep` storage engine parameter). Older mutation entries are deleted. @@ -52,12 +52,12 @@ Entries for finished mutations are not deleted right away (the number of preserv For non-replicated tables, all `ALTER` queries are performed synchronously. For replicated tables, the query just adds instructions for the appropriate actions to `ZooKeeper`, and the actions themselves are performed as soon as possible. However, the query can wait for these actions to be completed on all the replicas. -For all `ALTER` queries, you can use the [replication_alter_partitions_sync](../../../operations/settings/settings.md#replication-alter-partitions-sync) setting to set up waiting. +For all `ALTER` queries, you can use the [replication_alter_partitions_sync](/docs/en/operations/settings/settings.md/#replication-alter-partitions-sync) setting to set up waiting. -You can specify how long (in seconds) to wait for inactive replicas to execute all `ALTER` queries with the [replication_wait_for_inactive_replica_timeout](../../../operations/settings/settings.md#replication-wait-for-inactive-replica-timeout) setting. +You can specify how long (in seconds) to wait for inactive replicas to execute all `ALTER` queries with the [replication_wait_for_inactive_replica_timeout](/docs/en/operations/settings/settings.md/#replication-wait-for-inactive-replica-timeout) setting. :::note For all `ALTER` queries, if `replication_alter_partitions_sync = 2` and some replicas are not active for more than the time, specified in the `replication_wait_for_inactive_replica_timeout` setting, then an exception `UNFINISHED` is thrown. ::: -For `ALTER TABLE ... UPDATE|DELETE` queries the synchronicity is defined by the [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting. +For `ALTER TABLE ... UPDATE|DELETE` queries the synchronicity is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting. diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 2d89c1d5d18..146c15e776e 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -5,7 +5,7 @@ sidebar_label: PARTITION title: "Manipulating Partitions and Parts" --- -The following operations with [partitions](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md) are available: +The following operations with [partitions](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) are available: - [DETACH PARTITION\|PART](#detach-partitionpart) — Moves a partition or part to the `detached` directory and forget it. - [DROP PARTITION\|PART](#drop-partitionpart) — Deletes a partition or part. @@ -43,7 +43,7 @@ Read about setting the partition expression in a section [How to set the partiti After the query is executed, you can do whatever you want with the data in the `detached` directory — delete it from the file system, or just leave it. -This query is replicated – it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](../../../operations/system-tables/replicas.md#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replicas (as multiple leaders are allowed). +This query is replicated – it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](/docs/en/operations/system-tables/replicas.md/#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replicas (as multiple leaders are allowed). ## DROP PARTITION\|PART @@ -175,7 +175,7 @@ At the time of execution, for a data snapshot, the query creates hardlinks to a - if the `WITH NAME` parameter is specified, then the value of the `'backup_name'` parameter is used instead of the incremental number. :::note -If you use [a set of disks for data storage in a table](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression. +If you use [a set of disks for data storage in a table](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression. ::: The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs `chmod` for all files, forbidding writing into them. @@ -249,7 +249,7 @@ Although the query is called `ALTER TABLE`, it does not change the table structu ## MOVE PARTITION\|PART -Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). +Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-multiple-volumes). ``` sql ALTER TABLE table_name [ON CLUSTER cluster] MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' @@ -270,7 +270,7 @@ ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' ## UPDATE IN PARTITION -Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). Syntax: @@ -286,11 +286,11 @@ ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2; ### See Also -- [UPDATE](../../../sql-reference/statements/alter/update.md#alter-table-update-statements) +- [UPDATE](/docs/en/sql-reference/statements/alter/update.md/#alter-table-update-statements) ## DELETE IN PARTITION -Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). Syntax: @@ -306,7 +306,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; ### See Also -- [DELETE](../../../sql-reference/statements/alter/delete.md#alter-mutations) +- [DELETE](/docs/en/sql-reference/statements/alter/delete.md/#alter-mutations) ## How to Set Partition Expression @@ -315,16 +315,16 @@ You can specify the partition expression in `ALTER ... PARTITION` queries in dif - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. - As a tuple of expressions or constants that matches (in types) the table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. Usage of quotes when specifying the partition depends on the type of partition expression. For example, for the `String` type, you have to specify its name in quotes (`'`). For the `Date` and `Int*` types no quotes are needed. -All the rules above are also true for the [OPTIMIZE](../../../sql-reference/statements/optimize.md) query. If you need to specify the only partition when optimizing a non-partitioned table, set the expression `PARTITION tuple()`. For example: +All the rules above are also true for the [OPTIMIZE](/docs/en/sql-reference/statements/optimize.md) query. If you need to specify the only partition when optimizing a non-partitioned table, set the expression `PARTITION tuple()`. For example: ``` sql OPTIMIZE TABLE table_not_partitioned PARTITION tuple() FINAL; ``` -`IN PARTITION` specifies the partition to which the [UPDATE](../../../sql-reference/statements/alter/update.md#alter-table-update-statements) or [DELETE](../../../sql-reference/statements/alter/delete.md#alter-mutations) expressions are applied as a result of the `ALTER TABLE` query. New parts are created only from the specified partition. In this way, `IN PARTITION` helps to reduce the load when the table is divided into many partitions, and you only need to update the data point-by-point. +`IN PARTITION` specifies the partition to which the [UPDATE](/docs/en/sql-reference/statements/alter/update.md/#alter-table-update-statements) or [DELETE](/docs/en/sql-reference/statements/alter/delete.md/#alter-mutations) expressions are applied as a result of the `ALTER TABLE` query. New parts are created only from the specified partition. In this way, `IN PARTITION` helps to reduce the load when the table is divided into many partitions, and you only need to update the data point-by-point. The examples of `ALTER ... PARTITION` queries are demonstrated in the tests [`00502_custom_partitioning_local`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index ff8ecf3a77f..7913c7cb7e4 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -5,7 +5,7 @@ sidebar_label: PROJECTION title: "Manipulating Projections" --- -The following operations with [projections](../../../engines/table-engines/mergetree-family/mergetree.md#projections) are available: +The following operations with [projections](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#projections) are available: ## ADD PROJECTION @@ -13,15 +13,15 @@ The following operations with [projections](../../../engines/table-engines/merge ## DROP PROJECTION -`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). ## MATERIALIZE PROJECTION -`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). ## CLEAR PROJECTION -`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only change metadata or remove files. @@ -29,5 +29,5 @@ The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only Also, they are replicated, syncing projections metadata via ClickHouse Keeper or ZooKeeper. :::note -Projection manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +Projection manipulation is supported only for tables with [`*MergeTree`](/docs/en/engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](/docs/en/engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/docs/en/sql-reference/statements/alter/index/index.md b/docs/en/sql-reference/statements/alter/skipping-index.md similarity index 70% rename from docs/en/sql-reference/statements/alter/index/index.md rename to docs/en/sql-reference/statements/alter/skipping-index.md index 03d4bd47e71..2dadffc4527 100644 --- a/docs/en/sql-reference/statements/alter/index/index.md +++ b/docs/en/sql-reference/statements/alter/skipping-index.md @@ -1,5 +1,6 @@ --- -slug: /en/sql-reference/statements/alter/index +slug: /en/sql-reference/statements/alter/skipping-index + toc_hidden_folder: true sidebar_position: 42 sidebar_label: INDEX @@ -13,12 +14,12 @@ The following operations are available: - `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. -- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. +- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing indices metadata via ZooKeeper. :::note -Index manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). +Index manipulation is supported only for tables with [`*MergeTree`](/docs/en/engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](/docs/en/engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index e4fb872ae24..f40b72f7ab3 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -10,7 +10,7 @@ sidebar_label: UPDATE ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` -Manipulates data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +Manipulates data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). :::note The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. @@ -20,11 +20,11 @@ The `filter_expr` must be of type `UInt8`. This query updates values of specifie One query can contain several commands separated by commas. -The synchronicity of the query processing is defined by the [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting. By default, it is asynchronous. +The synchronicity of the query processing is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting. By default, it is asynchronous. **See also** -- [Mutations](../../../sql-reference/statements/alter/index.md#mutations) -- [Synchronicity of ALTER Queries](../../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) -- [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting +- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) +- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) +- [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting diff --git a/docs/en/sql-reference/statements/index.md b/docs/en/sql-reference/statements/index.md index bfb90f4a89f..b286d8c932d 100644 --- a/docs/en/sql-reference/statements/index.md +++ b/docs/en/sql-reference/statements/index.md @@ -8,25 +8,25 @@ sidebar_label: Statements Statements represent various kinds of action you can perform using SQL queries. Each kind of statement has it’s own syntax and usage details that are described separately: -- [SELECT](../../sql-reference/statements/select/index.md) -- [INSERT INTO](../../sql-reference/statements/insert-into.md) -- [CREATE](../../sql-reference/statements/create/index.md) -- [ALTER](../../sql-reference/statements/alter/index.md) -- [SYSTEM](../../sql-reference/statements/system.md) -- [SHOW](../../sql-reference/statements/show.md) -- [GRANT](../../sql-reference/statements/grant.md) -- [REVOKE](../../sql-reference/statements/revoke.md) -- [ATTACH](../../sql-reference/statements/attach.md) -- [CHECK TABLE](../../sql-reference/statements/check-table.md) -- [DESCRIBE TABLE](../../sql-reference/statements/describe-table.md) -- [DETACH](../../sql-reference/statements/detach.md) -- [DROP](../../sql-reference/statements/drop.md) -- [EXISTS](../../sql-reference/statements/exists.md) -- [KILL](../../sql-reference/statements/kill.md) -- [OPTIMIZE](../../sql-reference/statements/optimize.md) -- [RENAME](../../sql-reference/statements/rename.md) -- [SET](../../sql-reference/statements/set.md) -- [SET ROLE](../../sql-reference/statements/set-role.md) -- [TRUNCATE](../../sql-reference/statements/truncate.md) -- [USE](../../sql-reference/statements/use.md) -- [EXPLAIN](../../sql-reference/statements/explain.md) +- [SELECT](/docs/en/sql-reference/statements/select/index.md) +- [INSERT INTO](/docs/en/sql-reference/statements/insert-into.md) +- [CREATE](/docs/en/sql-reference/statements/create/index.md) +- [ALTER](/docs/en/sql-reference/statements/alter/index.md) +- [SYSTEM](/docs/en/sql-reference/statements/system.md) +- [SHOW](/docs/en/sql-reference/statements/show.md) +- [GRANT](/docs/en/sql-reference/statements/grant.md) +- [REVOKE](/docs/en/sql-reference/statements/revoke.md) +- [ATTACH](/docs/en/sql-reference/statements/attach.md) +- [CHECK TABLE](/docs/en/sql-reference/statements/check-table.md) +- [DESCRIBE TABLE](/docs/en/sql-reference/statements/describe-table.md) +- [DETACH](/docs/en/sql-reference/statements/detach.md) +- [DROP](/docs/en/sql-reference/statements/drop.md) +- [EXISTS](/docs/en/sql-reference/statements/exists.md) +- [KILL](/docs/en/sql-reference/statements/kill.md) +- [OPTIMIZE](/docs/en/sql-reference/statements/optimize.md) +- [RENAME](/docs/en/sql-reference/statements/rename.md) +- [SET](/docs/en/sql-reference/statements/set.md) +- [SET ROLE](/docs/en/sql-reference/statements/set-role.md) +- [TRUNCATE](/docs/en/sql-reference/statements/truncate.md) +- [USE](/docs/en/sql-reference/statements/use.md) +- [EXPLAIN](/docs/en/sql-reference/statements/explain.md) diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index d09adcd13d6..94b23bc695c 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -39,3 +39,7 @@ You can’t use table functions if the [allow_ddl](../../operations/settings/per | [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. | | [sqlite](../../sql-reference/table-functions/sqlite.md) | Creates a [sqlite](../../engines/table-engines/integrations/sqlite.md)-engine table. | +:::note +Only these table functions are enabled in readonly mode : +null, view, viewIfPermitted, numbers, numbers_mt, generateRandom, values, cluster, clusterAllReplicas +::: \ No newline at end of file diff --git a/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp b/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp index 14850da2ebf..c60d4a4e5cc 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp +++ b/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp @@ -1,7 +1,6 @@ #include "ExternalDictionaryLibraryHandler.h" #include -#include #include #include @@ -113,7 +112,7 @@ Block ExternalDictionaryLibraryHandler::loadAll() Block ExternalDictionaryLibraryHandler::loadIds(const std::vector & ids) { - const ExternalDictionaryLibraryAPI::VectorUInt64 ids_data{bit_cast(ids.data()), ids.size()}; + const ExternalDictionaryLibraryAPI::VectorUInt64 ids_data{std::bit_cast(ids.data()), ids.size()}; auto columns_holder = std::make_unique(attributes_names.size()); ExternalDictionaryLibraryAPI::CStrings columns_pass{static_cast(columns_holder.get()), attributes_names.size()}; diff --git a/programs/library-bridge/ExternalDictionaryLibraryUtils.h b/programs/library-bridge/ExternalDictionaryLibraryUtils.h index e813efab2a6..c9d03d27f75 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryUtils.h +++ b/programs/library-bridge/ExternalDictionaryLibraryUtils.h @@ -2,7 +2,6 @@ #include #include -#include #include #include "ExternalDictionaryLibraryAPI.h" diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 7fdc5a54d8a..b6952ad6cb0 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -32,7 +32,6 @@ #include #include #include -#include #include #include #include @@ -278,9 +277,9 @@ Float transformFloatMantissa(Float x, UInt64 seed) using UInt = std::conditional_t, UInt32, UInt64>; constexpr size_t mantissa_num_bits = std::is_same_v ? 23 : 52; - UInt x_uint = bit_cast(x); + UInt x_uint = std::bit_cast(x); x_uint = static_cast(feistelNetwork(x_uint, mantissa_num_bits, seed)); - return bit_cast(x_uint); + return std::bit_cast(x_uint); } diff --git a/src/AggregateFunctions/QuantileBFloat16Histogram.h b/src/AggregateFunctions/QuantileBFloat16Histogram.h index 8ec325e238d..de9f61e01a2 100644 --- a/src/AggregateFunctions/QuantileBFloat16Histogram.h +++ b/src/AggregateFunctions/QuantileBFloat16Histogram.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -104,13 +103,13 @@ private: /// Take the most significant 16 bits of the floating point number. BFloat16 toBFloat16(const Value & x) const { - return bit_cast(static_cast(x)) >> 16; + return std::bit_cast(static_cast(x)) >> 16; } /// Put the bits into most significant 16 bits of the floating point number and fill other bits with zeros. Float32 toFloat32(const BFloat16 & x) const { - return bit_cast(x << 16); + return std::bit_cast(x << 16); } using Pair = PairNoInit; diff --git a/src/Analyzer/IQueryTreePass.h b/src/Analyzer/IQueryTreePass.h index 39b3d743ed3..4293934c32d 100644 --- a/src/Analyzer/IQueryTreePass.h +++ b/src/Analyzer/IQueryTreePass.h @@ -16,7 +16,7 @@ namespace DB * Dependencies between passes must be avoided. */ class IQueryTreePass; -using QueryTreePassPtr = std::shared_ptr; +using QueryTreePassPtr = std::unique_ptr; using QueryTreePasses = std::vector; class IQueryTreePass diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 9db2d66d99d..4f24051df79 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5527,9 +5527,15 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier * 3. Check that there are no columns that are not specified in GROUP BY keys. * 4. Validate GROUP BY modifiers. */ - assertNoAggregateFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); - assertNoGroupingFunction(query_node_typed.getJoinTree(), "in JOIN TREE"); - assertNoWindowFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + auto join_tree_node_type = query_node_typed.getJoinTree()->getNodeType(); + bool join_tree_is_subquery = join_tree_node_type == QueryTreeNodeType::QUERY || join_tree_node_type == QueryTreeNodeType::UNION; + + if (!join_tree_is_subquery) + { + assertNoAggregateFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + assertNoGroupingFunction(query_node_typed.getJoinTree(), "in JOIN TREE"); + assertNoWindowFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + } if (query_node_typed.hasWhere()) { diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 853b4a23f38..9dc9cda44a3 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -18,6 +18,9 @@ #include #include +#include +#include +#include namespace DB { @@ -25,6 +28,38 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; +} + +namespace +{ + +#ifndef NDEBUG + +/** This visitor checks if Query Tree structure is valid after each pass + * in debug build. + */ +class ValidationChecker : public InDepthQueryTreeVisitor +{ + String pass_name; +public: + explicit ValidationChecker(String pass_name_) + : pass_name(std::move(pass_name_)) + {} + + void visitImpl(QueryTreeNodePtr & node) const + { + auto * column = node->as(); + if (!column) + return; + if (column->getColumnSourceOrNull() == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column {} {} query tree node does not have valid source node after running {} pass", + column->getColumnName(), column->getColumnType(), pass_name); + } +}; +#endif + } /** ClickHouse query tree pass manager. @@ -61,7 +96,12 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node) size_t passes_size = passes.size(); for (size_t i = 0; i < passes_size; ++i) + { passes[i]->run(query_tree_node, current_context); +#ifndef NDEBUG + ValidationChecker(passes[i]->getName()).visit(query_tree_node); +#endif + } } void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index) @@ -75,7 +115,12 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pa auto current_context = getContext(); for (size_t i = 0; i < up_to_pass_index; ++i) + { passes[i]->run(query_tree_node, current_context); +#ifndef NDEBUG + ValidationChecker(passes[i]->getName()).visit(query_tree_node); +#endif + } } void QueryTreePassManager::dump(WriteBuffer & buffer) @@ -114,38 +159,38 @@ void addQueryTreePasses(QueryTreePassManager & manager) auto context = manager.getContext(); const auto & settings = context->getSettingsRef(); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_functions_to_subcolumns) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.count_distinct_optimization) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_rewrite_sum_if_to_count_if) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_normalize_count_variants) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_arithmetic_operations_in_aggregate_functions) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_injective_functions_inside_uniq) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_multiif_to_if) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_if_chain_to_multiif) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); } } diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 3afbb6735dc..2f8aa487621 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -1331,7 +1331,7 @@ public: } template - inline auto addQuarters(DateOrTime d, Int64 delta) const + inline auto NO_SANITIZE_UNDEFINED addQuarters(DateOrTime d, Int64 delta) const { return addMonths(d, delta * 3); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 4db944c8c3f..7440896b4c9 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -411,6 +411,7 @@ inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); } inline bool isArray(const DataTypePtr & data_type) { return WhichDataType(data_type).isArray(); } inline bool isMap(const DataTypePtr & data_type) {return WhichDataType(data_type).isMap(); } +inline bool isInterval(const DataTypePtr & data_type) {return WhichDataType(data_type).isInterval(); } inline bool isNothing(const DataTypePtr & data_type) { return WhichDataType(data_type).isNothing(); } inline bool isUUID(const DataTypePtr & data_type) { return WhichDataType(data_type).isUUID(); } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index ed9199a359f..29f5719e3ed 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index c921b0425d3..7e4dc387b66 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -642,7 +643,8 @@ class FunctionBinaryArithmetic : public IFunction DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, DataTypeDecimal32, DataTypeDecimal64, DataTypeDecimal128, DataTypeDecimal256, DataTypeDate, DataTypeDateTime, - DataTypeFixedString, DataTypeString>; + DataTypeFixedString, DataTypeString, + DataTypeInterval>; using Floats = TypeList; @@ -717,6 +719,82 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name, context); } + static FunctionOverloadResolverPtr + getFunctionForDateTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + { + bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1); + + /// Exactly one argument must be Date or DateTime + if (first_is_date_or_datetime == second_is_date_or_datetime) + return {}; + + if (!isTuple(type0) && !isTuple(type1)) + return {}; + + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + /// We construct another function and call it. + if constexpr (!is_plus && !is_minus) + return {}; + + if (isTuple(type0) && second_is_date_or_datetime && is_minus) + throw Exception("Wrong order of arguments for function " + String(name) + ": argument of Tuple type cannot be first", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + std::string function_name; + if (is_plus) + { + function_name = "addTupleOfIntervals"; + } + else + { + function_name = "subtractTupleOfIntervals"; + } + + return FunctionFactory::instance().get(function_name, context); + } + + static FunctionOverloadResolverPtr + getFunctionForMergeIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + { + /// Special case when the function is plus or minus, first argument is Interval or Tuple of Intervals + /// and the second argument is the Interval of a different kind. + /// We construct another function (example: addIntervals) and call it + + if constexpr (!is_plus && !is_minus) + return {}; + + const auto * tuple_data_type_0 = checkAndGetDataType(type0.get()); + const auto * interval_data_type_0 = checkAndGetDataType(type0.get()); + const auto * interval_data_type_1 = checkAndGetDataType(type1.get()); + + if ((!tuple_data_type_0 && !interval_data_type_0) || !interval_data_type_1) + return {}; + + if (interval_data_type_0 && interval_data_type_0->equals(*interval_data_type_1)) + return {}; + + if (tuple_data_type_0) + { + auto & tuple_types = tuple_data_type_0->getElements(); + for (auto & type : tuple_types) + if (!isInterval(type)) + return {}; + } + + std::string function_name; + if (is_plus) + { + function_name = "addInterval"; + } + else + { + function_name = "subtractInterval"; + } + + return FunctionFactory::instance().get(function_name, context); + } + static FunctionOverloadResolverPtr getFunctionForTupleArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { @@ -915,6 +993,30 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(new_arguments, result_type, input_rows_count); } + ColumnPtr executeDateTimeTupleOfIntervalsPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + ColumnsWithTypeAndName new_arguments = arguments; + + /// Tuple argument must be second. + if (isTuple(arguments[0].type)) + std::swap(new_arguments[0], new_arguments[1]); + + auto function = function_builder->build(new_arguments); + + return function->execute(new_arguments, result_type, input_rows_count); + } + + ColumnPtr executeIntervalTupleOfIntervalsPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + ColumnsWithTypeAndName new_arguments = arguments; + + auto function = function_builder->build(new_arguments); + + return function->execute(new_arguments, result_type, input_rows_count); + } + ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const { @@ -1134,6 +1236,34 @@ public: return function->getResultType(); } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + if (auto function_builder = getFunctionForDateTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) + { + ColumnsWithTypeAndName new_arguments(2); + + for (size_t i = 0; i < 2; ++i) + new_arguments[i].type = arguments[i]; + + /// Tuple argument must be second. + if (isTuple(new_arguments[0].type)) + std::swap(new_arguments[0], new_arguments[1]); + + auto function = function_builder->build(new_arguments); + return function->getResultType(); + } + + /// Special case when the function is plus or minus, one of arguments is Interval/Tuple of Intervals and another is Interval. + if (auto function_builder = getFunctionForMergeIntervalsArithmetic(arguments[0], arguments[1], context)) + { + ColumnsWithTypeAndName new_arguments(2); + + for (size_t i = 0; i < 2; ++i) + new_arguments[i].type = arguments[i]; + + auto function = function_builder->build(new_arguments); + return function->getResultType(); + } + /// Special case when the function is multiply or divide, one of arguments is Tuple and another is Number. if (auto function_builder = getFunctionForTupleAndNumberArithmetic(arguments[0], arguments[1], context)) { @@ -1185,6 +1315,21 @@ public: type_res = std::make_shared(); return true; } + else if constexpr (std::is_same_v || std::is_same_v) + { + if constexpr (std::is_same_v && + std::is_same_v) + { + if constexpr (is_plus || is_minus) + { + if (left.getKind() == right.getKind()) + { + type_res = std::make_shared(left.getKind()); + return true; + } + } + } + } else { using ResultDataType = typename BinaryOperationTraits::ResultDataType; @@ -1566,6 +1711,18 @@ public: return executeDateTimeIntervalPlusMinus(arguments, result_type, input_rows_count, function_builder); } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + if (auto function_builder = getFunctionForDateTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + { + return executeDateTimeTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); + } + + /// Special case when the function is plus or minus, one of arguments is Interval/Tuple of Intervals and another is Interval. + if (auto function_builder = getFunctionForMergeIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + { + return executeIntervalTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); + } + /// Special case when the function is plus, minus or multiply, both arguments are tuples. if (auto function_builder = getFunctionForTupleArithmetic(arguments[0].type, arguments[1].type, context)) { diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index e4605e5e214..befab1e0c91 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -145,7 +146,8 @@ class FunctionUnaryArithmetic : public IFunction DataTypeDecimal, DataTypeDecimal, DataTypeDecimal, - DataTypeFixedString + DataTypeFixedString, + DataTypeInterval >(type, std::forward(f)); } @@ -211,6 +213,12 @@ public: return false; result = std::make_shared(type.getN()); } + else if constexpr (std::is_same_v) + { + if constexpr (!IsUnaryOperation::negate) + return false; + result = std::make_shared(type.getKind()); + } else { using T0 = typename DataType::FieldType; diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 283f1ea5a43..2c7883cf471 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index 68425ee496e..d8da525e63a 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -91,23 +91,30 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() < 2 || 3 < arguments.size()) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 2 or 3.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", + getName(), arguments.size()); if (!isStringOrFixedString(arguments[0])) throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0]->getName(), getName()); if (!isString(arguments[1])) throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[1]->getName(), getName()); if (arguments.size() >= 3) { if (!isUnsignedInteger(arguments[2])) throw Exception( - "Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[2]->getName(), getName()); } return std::make_shared>(); @@ -196,9 +203,11 @@ public: vec_res); else throw Exception( - "Illegal columns " + arguments[0].column->getName() + " and " - + arguments[1].column->getName() + " of arguments of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); + ErrorCodes::ILLEGAL_COLUMN, + "Illegal columns {} and {} of arguments of function {}", + arguments[0].column->getName(), + arguments[1].column->getName(), + getName()); return col_res; } diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 6862a097d0e..edb0df3ae34 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -25,7 +25,7 @@ namespace impl /// Is the [I]LIKE expression reduced to finding a substring in a string? inline bool likePatternIsSubstring(std::string_view pattern, String & res) { - if (pattern.size() < 2 || pattern.front() != '%' || pattern.back() != '%') + if (pattern.size() < 2 || !pattern.starts_with('%') || !pattern.ends_with('%')) return false; res.clear(); @@ -101,9 +101,7 @@ struct MatchImpl static constexpr bool case_insensitive = (case_ == MatchTraits::Case::Insensitive); static constexpr bool negate = (result_ == MatchTraits::Result::Negate); - using Searcher = std::conditional_t; + using Searcher = std::conditional_t; static void vectorConstant( const ColumnString::Chars & haystack_data, @@ -115,13 +113,12 @@ struct MatchImpl const size_t haystack_size = haystack_offsets.size(); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack_offsets.empty()) return; - /// A simple case where the [I]LIKE expression reduces to finding a substring in a string + /// Special case that the [I]LIKE expression reduces to finding a substring in a string String strstr_pattern; if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern)) { @@ -158,105 +155,101 @@ struct MatchImpl /// Tail, in which there can be no substring. if (i < res.size()) memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + + return; + } + + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + + String required_substring; + bool is_trivial; + bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + + regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + + if (required_substring.empty()) + { + if (!regexp.getRE2()) /// An empty regexp. Always matches. + memset(res.data(), !negate, haystack_size * sizeof(res[0])); + else + { + size_t prev_offset = 0; + for (size_t i = 0; i < haystack_size; ++i) + { + const bool match = regexp.getRE2()->Match( + {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, + 0, + haystack_offsets[i] - prev_offset - 1, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + + prev_offset = haystack_offsets[i]; + } + } } else { - const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + /// NOTE This almost matches with the case of impl::likePatternIsSubstring. - String required_substring; - bool is_trivial; - bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + const UInt8 * const begin = haystack_data.data(); + const UInt8 * const end = haystack_data.begin() + haystack_data.size(); + const UInt8 * pos = begin; - regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + /// The current index in the array of strings. + size_t i = 0; - if (required_substring.empty()) + Searcher searcher(required_substring.data(), required_substring.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) { - if (!regexp.getRE2()) /// An empty regexp. Always matches. + /// Determine which index it refers to. + while (begin + haystack_offsets[i] <= pos) { - if (haystack_size) - memset(res.data(), !negate, haystack_size * sizeof(res[0])); + res[i] = negate; + ++i; } - else + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + required_substring.size() < begin + haystack_offsets[i]) { - size_t prev_offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + /// And if it does not, if necessary, we check the regexp. + if (is_trivial) + res[i] = !negate; + else { + const char * str_data = reinterpret_cast(&haystack_data[haystack_offsets[i - 1]]); + size_t str_size = haystack_offsets[i] - haystack_offsets[i - 1] - 1; + + /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, + * so that it can match when `required_substring` occurs into the string several times, + * and at the first occurrence, the regexp is not a match. + */ + const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; + const size_t end_pos = str_size; + const bool match = regexp.getRE2()->Match( - {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, - 0, - haystack_offsets[i] - prev_offset - 1, + {str_data, str_size}, + start_pos, + end_pos, re2_st::RE2::UNANCHORED, nullptr, 0); res[i] = negate ^ match; - - prev_offset = haystack_offsets[i]; } } + else + res[i] = negate; + + pos = begin + haystack_offsets[i]; + ++i; } - else - { - /// NOTE This almost matches with the case of impl::likePatternIsSubstring. - const UInt8 * const begin = haystack_data.data(); - const UInt8 * const end = haystack_data.begin() + haystack_data.size(); - const UInt8 * pos = begin; - - /// The current index in the array of strings. - size_t i = 0; - - Searcher searcher(required_substring.data(), required_substring.size(), end - pos); - - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Determine which index it refers to. - while (begin + haystack_offsets[i] <= pos) - { - res[i] = negate; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + required_substring.size() < begin + haystack_offsets[i]) - { - /// And if it does not, if necessary, we check the regexp. - - if (is_trivial) - res[i] = !negate; - else - { - const char * str_data = reinterpret_cast(&haystack_data[haystack_offsets[i - 1]]); - size_t str_size = haystack_offsets[i] - haystack_offsets[i - 1] - 1; - - /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, - * so that it can match when `required_substring` occurs into the string several times, - * and at the first occurrence, the regexp is not a match. - */ - const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; - const size_t end_pos = str_size; - - const bool match = regexp.getRE2()->Match( - {str_data, str_size}, - start_pos, - end_pos, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - } - } - else - res[i] = negate; - - pos = begin + haystack_offsets[i]; - ++i; - } - - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); - } + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); } } @@ -274,7 +267,7 @@ struct MatchImpl if (haystack.empty()) return; - /// A simple case where the LIKE expression reduces to finding a substring in a string + /// Special case that the [I]LIKE expression reduces to finding a substring in a string String strstr_pattern; if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern)) { @@ -316,109 +309,105 @@ struct MatchImpl /// Tail, in which there can be no substring. if (i < res.size()) memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + + return; + } + + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + + String required_substring; + bool is_trivial; + bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + + regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + + if (required_substring.empty()) + { + if (!regexp.getRE2()) /// An empty regexp. Always matches. + memset(res.data(), !negate, haystack_size * sizeof(res[0])); + else + { + size_t offset = 0; + for (size_t i = 0; i < haystack_size; ++i) + { + const bool match = regexp.getRE2()->Match( + {reinterpret_cast(&haystack[offset]), N}, + 0, + N, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + + offset += N; + } + } } else { - const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + /// NOTE This almost matches with the case of likePatternIsSubstring. - String required_substring; - bool is_trivial; - bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + const UInt8 * const begin = haystack.data(); + const UInt8 * const end = haystack.data() + haystack.size(); + const UInt8 * pos = begin; - regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + size_t i = 0; + const UInt8 * next_pos = begin; - if (required_substring.empty()) + /// If required substring is larger than string size - it cannot be found. + if (required_substring.size() <= N) { - if (!regexp.getRE2()) /// An empty regexp. Always matches. + Searcher searcher(required_substring.data(), required_substring.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) { - if (haystack_size) - memset(res.data(), !negate, haystack_size * sizeof(res[0])); - } - else - { - size_t offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + /// Let's determine which index it refers to. + while (next_pos + N <= pos) { - const bool match = regexp.getRE2()->Match( - {reinterpret_cast(&haystack[offset]), N}, - 0, - N, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - - offset += N; - } - } - } - else - { - /// NOTE This almost matches with the case of likePatternIsSubstring. - - const UInt8 * const begin = haystack.data(); - const UInt8 * const end = haystack.data() + haystack.size(); - const UInt8 * pos = begin; - - size_t i = 0; - const UInt8 * next_pos = begin; - - /// If required substring is larger than string size - it cannot be found. - if (required_substring.size() <= N) - { - Searcher searcher(required_substring.data(), required_substring.size(), end - pos); - - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Let's determine which index it refers to. - while (next_pos + N <= pos) - { - res[i] = negate; - next_pos += N; - ++i; - } + res[i] = negate; next_pos += N; - - if (pos + required_substring.size() <= next_pos) - { - /// And if it does not, if necessary, we check the regexp. - - if (is_trivial) - res[i] = !negate; - else - { - const char * str_data = reinterpret_cast(next_pos - N); - - /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, - * so that it can match when `required_substring` occurs into the string several times, - * and at the first occurrence, the regexp is not a match. - */ - const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; - const size_t end_pos = N; - - const bool match = regexp.getRE2()->Match( - {str_data, N}, - start_pos, - end_pos, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - } - } - else - res[i] = negate; - - pos = next_pos; ++i; } - } + next_pos += N; - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + if (pos + required_substring.size() <= next_pos) + { + /// And if it does not, if necessary, we check the regexp. + if (is_trivial) + res[i] = !negate; + else + { + const char * str_data = reinterpret_cast(next_pos - N); + + /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, + * so that it can match when `required_substring` occurs into the string several times, + * and at the first occurrence, the regexp is not a match. + */ + const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; + const size_t end_pos = N; + + const bool match = regexp.getRE2()->Match( + {str_data, N}, + start_pos, + end_pos, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + } + } + else + res[i] = negate; + + pos = next_pos; + ++i; + } } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); } } @@ -434,7 +423,6 @@ struct MatchImpl assert(haystack_size == needle_offset.size()); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack_offsets.empty()) @@ -481,9 +469,7 @@ struct MatchImpl if (required_substr.empty()) { if (!regexp->getRE2()) /// An empty regexp. Always matches. - { res[i] = !negate; - } else { const bool match = regexp->getRE2()->Match( @@ -502,15 +488,11 @@ struct MatchImpl const auto * match = searcher.search(cur_haystack_data, cur_haystack_length); if (match == cur_haystack_data + cur_haystack_length) - { res[i] = negate; // no match - } else { if (is_trivial) - { res[i] = !negate; // no wildcards in pattern - } else { const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; @@ -546,7 +528,6 @@ struct MatchImpl assert(haystack_size == needle_offset.size()); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack.empty()) @@ -593,9 +574,7 @@ struct MatchImpl if (required_substr.empty()) { if (!regexp->getRE2()) /// An empty regexp. Always matches. - { res[i] = !negate; - } else { const bool match = regexp->getRE2()->Match( @@ -614,15 +593,11 @@ struct MatchImpl const auto * match = searcher.search(cur_haystack_data, cur_haystack_length); if (match == cur_haystack_data + cur_haystack_length) - { res[i] = negate; // no match - } else { if (is_trivial) - { res[i] = !negate; // no wildcards in pattern - } else { const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; diff --git a/src/Functions/isFinite.cpp b/src/Functions/isFinite.cpp index 90185b64fff..612e7067bf5 100644 --- a/src/Functions/isFinite.cpp +++ b/src/Functions/isFinite.cpp @@ -1,6 +1,5 @@ #include #include -#include #include @@ -20,11 +19,11 @@ struct IsFiniteImpl static bool execute(const T t) { if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b01111111100000000000000000000000) != 0b01111111100000000000000000000000; else if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b0111111111110000000000000000000000000000000000000000000000000000) != 0b0111111111110000000000000000000000000000000000000000000000000000; else diff --git a/src/Functions/isInfinite.cpp b/src/Functions/isInfinite.cpp index e923e1461bc..ace2c334873 100644 --- a/src/Functions/isInfinite.cpp +++ b/src/Functions/isInfinite.cpp @@ -1,6 +1,5 @@ #include #include -#include #include @@ -16,11 +15,11 @@ struct IsInfiniteImpl static bool execute(const T t) { if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b01111111111111111111111111111111) == 0b01111111100000000000000000000000; else if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b0111111111111111111111111111111111111111111111111111111111111111) == 0b0111111111110000000000000000000000000000000000000000000000000000; else diff --git a/src/Functions/notLike.cpp b/src/Functions/notLike.cpp index a546b511a0b..200890d77d8 100644 --- a/src/Functions/notLike.cpp +++ b/src/Functions/notLike.cpp @@ -12,7 +12,8 @@ struct NameNotLike static constexpr auto name = "notLike"; }; -using FunctionNotLike = FunctionsStringSearch>; +using NotLikeImpl = MatchImpl; +using FunctionNotLike = FunctionsStringSearch; } diff --git a/src/Functions/padString.cpp b/src/Functions/padString.cpp index c8ed920755c..486fa328fa0 100644 --- a/src/Functions/padString.cpp +++ b/src/Functions/padString.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { @@ -59,10 +58,10 @@ namespace { if (num_chars <= step) { - writeSlice(StringSource::Slice{bit_cast(pad_string.data()), numCharsToNumBytes(num_chars)}, res_sink); + writeSlice(StringSource::Slice{std::bit_cast(pad_string.data()), numCharsToNumBytes(num_chars)}, res_sink); break; } - writeSlice(StringSource::Slice{bit_cast(pad_string.data()), numCharsToNumBytes(step)}, res_sink); + writeSlice(StringSource::Slice{std::bit_cast(pad_string.data()), numCharsToNumBytes(step)}, res_sink); num_chars -= step; } } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index ccefd74b85b..96615d0a4c9 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -115,6 +117,13 @@ namespace static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; + class FunctionDisplayName : public FunctionConstantBase + { + public: + static constexpr auto name = "displayName"; + explicit FunctionDisplayName(ContextPtr context) : FunctionConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {} + static FunctionPtr create(ContextPtr context) {return std::make_shared(context); } + }; } #if defined(__ELF__) && !defined(OS_FREEBSD) @@ -173,4 +182,20 @@ REGISTER_FUNCTION(GetOSKernelVersion) } +REGISTER_FUNCTION(DisplayName) +{ + factory.registerFunction( + { + R"( +Returns the value of `display_name` from config or server FQDN if not set. + +[example:displayName] +)", + Documentation::Examples{{"displayName", "SELECT displayName();"}}, + Documentation::Categories{"Constant", "Miscellaneous"} + }, + FunctionFactory::CaseSensitive); +} + + } diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 20571f67eff..20835f59cc1 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -415,6 +416,274 @@ public: } }; +template +class FunctionDateOrDateTimeOperationTupleOfIntervals : public ITupleFunction +{ +public: + static constexpr auto name = Impl::name; + + explicit FunctionDateOrDateTimeOperationTupleOfIntervals(ContextPtr context_) : ITupleFunction(context_) {} + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}. Should be a date or a date with time", + arguments[0].type->getName(), getName()}; + + const auto * cur_tuple = checkAndGetDataType(arguments[1].type.get()); + + if (!cur_tuple) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}. Should be a tuple", + arguments[0].type->getName(), getName()}; + + const auto & cur_types = cur_tuple->getElements(); + + Columns cur_elements; + if (arguments[1].column) + cur_elements = getTupleElements(*arguments[1].column); + + size_t tuple_size = cur_types.size(); + if (tuple_size == 0) + return arguments[0].type; + + auto plus = FunctionFactory::instance().get(Impl::func_name, context); + DataTypePtr res_type = arguments[0].type; + for (size_t i = 0; i < tuple_size; ++i) + { + try + { + ColumnWithTypeAndName left{res_type, {}}; + ColumnWithTypeAndName right{cur_elements.empty() ? nullptr : cur_elements[i], cur_types[i], {}}; + auto plus_elem = plus->build({left, right}); + res_type = plus_elem->getResultType(); + } + catch (DB::Exception & e) + { + e.addMessage("While executing function {} for tuple element {}", getName(), i); + throw; + } + } + + return res_type; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto * cur_tuple = checkAndGetDataType(arguments[1].type.get()); + const auto & cur_types = cur_tuple->getElements(); + auto cur_elements = getTupleElements(*arguments[1].column); + + size_t tuple_size = cur_elements.size(); + if (tuple_size == 0) + return arguments[0].column; + + auto plus = FunctionFactory::instance().get(Impl::func_name, context); + ColumnWithTypeAndName res; + for (size_t i = 0; i < tuple_size; ++i) + { + ColumnWithTypeAndName column{cur_elements[i], cur_types[i], {}}; + auto elem_plus = plus->build(ColumnsWithTypeAndName{i == 0 ? arguments[0] : res, column}); + auto res_type = elem_plus->getResultType(); + res.column = elem_plus->execute({i == 0 ? arguments[0] : res, column}, res_type, input_rows_count); + res.type = res_type; + } + + return res.column; + } +}; + +struct AddTupleOfIntervalsImpl +{ + static constexpr auto name = "addTupleOfIntervals"; + static constexpr auto func_name = "plus"; +}; + +struct SubtractTupleOfIntervalsImpl +{ + static constexpr auto name = "subtractTupleOfIntervals"; + static constexpr auto func_name = "minus"; +}; + +using FunctionAddTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; + +using FunctionSubtractTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; + +template +struct FunctionTupleOperationInterval : public ITupleFunction +{ +public: + static constexpr auto name = is_minus ? "subtractInterval" : "addInterval"; + + explicit FunctionTupleOperationInterval(ContextPtr context_) : ITupleFunction(context_) {} + + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isTuple(arguments[0]) && !isInterval(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, must be Tuple or Interval", + arguments[0]->getName(), getName()); + + if (!isInterval(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, must be Interval", + arguments[1]->getName(), getName()); + + DataTypes types; + + const auto * tuple = checkAndGetDataType(arguments[0].get()); + + if (tuple) + { + const auto & cur_types = tuple->getElements(); + + for (const auto & type : cur_types) + if (!isInterval(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of Tuple element of first argument of function {}, must be Interval", + type->getName(), getName()); + + types = cur_types; + } + else + { + types = {arguments[0]}; + } + + const auto * interval_last = checkAndGetDataType(types.back().get()); + const auto * interval_new = checkAndGetDataType(arguments[1].get()); + + if (!interval_last->equals(*interval_new)) + types.push_back(arguments[1]); + + return std::make_shared(types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!isInterval(arguments[1].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, must be Interval", + arguments[1].type->getName(), getName()); + + Columns tuple_columns; + + const auto * first_tuple = checkAndGetDataType(arguments[0].type.get()); + const auto * first_interval = checkAndGetDataType(arguments[0].type.get()); + const auto * second_interval = checkAndGetDataType(arguments[1].type.get()); + + bool can_be_merged; + + if (first_interval) + { + can_be_merged = first_interval->equals(*second_interval); + + if (can_be_merged) + tuple_columns.resize(1); + else + tuple_columns.resize(2); + + tuple_columns[0] = arguments[0].column->convertToFullColumnIfConst(); + } + else if (first_tuple) + { + const auto & cur_types = first_tuple->getElements(); + + for (const auto & type : cur_types) + if (!isInterval(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of Tuple element of first argument of function {}, must be Interval", + type->getName(), getName()); + + auto cur_elements = getTupleElements(*arguments[0].column); + size_t tuple_size = cur_elements.size(); + + if (tuple_size == 0) + { + can_be_merged = false; + } + else + { + const auto * tuple_last_interval = checkAndGetDataType(cur_types.back().get()); + can_be_merged = tuple_last_interval->equals(*second_interval); + } + + if (can_be_merged) + tuple_columns.resize(tuple_size); + else + tuple_columns.resize(tuple_size + 1); + + for (size_t i = 0; i < tuple_size; ++i) + tuple_columns[i] = cur_elements[i]; + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, must be Tuple or Interval", + arguments[0].type->getName(), getName()); + + + ColumnPtr & last_column = tuple_columns.back(); + + if (can_be_merged) + { + ColumnWithTypeAndName left{last_column, arguments[1].type, {}}; + + if constexpr (is_minus) + { + auto minus = FunctionFactory::instance().get("minus", context); + auto elem_minus = minus->build({left, arguments[1]}); + last_column = elem_minus->execute({left, arguments[1]}, arguments[1].type, input_rows_count) + ->convertToFullColumnIfConst(); + } + else + { + auto plus = FunctionFactory::instance().get("plus", context); + auto elem_plus = plus->build({left, arguments[1]}); + last_column = elem_plus->execute({left, arguments[1]}, arguments[1].type, input_rows_count) + ->convertToFullColumnIfConst(); + } + } + else + { + if constexpr (is_minus) + { + auto negate = FunctionFactory::instance().get("negate", context); + auto elem_negate = negate->build({arguments[1]}); + last_column = elem_negate->execute({arguments[1]}, arguments[1].type, input_rows_count); + } + else + { + last_column = arguments[1].column; + } + } + + return ColumnTuple::create(tuple_columns); + } +}; + +using FunctionTupleAddInterval = FunctionTupleOperationInterval; + +using FunctionTupleSubtractInterval = FunctionTupleOperationInterval; + + /// this is for convenient usage in LNormalize template class FunctionLNorm : public ITupleFunction {}; @@ -1282,6 +1551,65 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction( + { + R"( +Consecutively adds a tuple of intervals to a Date or a DateTime. +[example:tuple] +)", + Documentation::Examples{ + {"tuple", "WITH toDate('2018-01-01') AS date SELECT addTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR))"}, + }, + Documentation::Categories{"Tuple", "Interval", "Date", "DateTime"} + }); + + factory.registerFunction( + { + R"( +Consecutively subtracts a tuple of intervals from a Date or a DateTime. +[example:tuple] +)", + Documentation::Examples{ + {"tuple", "WITH toDate('2018-01-01') AS date SELECT subtractTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR))"}, + }, + Documentation::Categories{"Tuple", "Interval", "Date", "DateTime"} + }); + + factory.registerFunction( + { + R"( +Adds an interval to another interval or tuple of intervals. The returned value is tuple of intervals. +[example:tuple] +[example:interval1] + +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +[example:interval2] +)", + Documentation::Examples{ + {"tuple", "SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH)"}, + {"interval1", "SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH)"}, + {"interval2", "SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 DAY)"}, + }, + Documentation::Categories{"Tuple", "Interval"} + }); + factory.registerFunction( + { + R"( +Adds an negated interval to another interval or tuple of intervals. The returned value is tuple of intervals. +[example:tuple] +[example:interval1] + +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +[example:interval2] +)", + Documentation::Examples{ + {"tuple", "SELECT subtractInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH)"}, + {"interval1", "SELECT subtractInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH)"}, + {"interval2", "SELECT subtractInterval(INTERVAL 2 DAY, INTERVAL 1 DAY)"}, + }, + Documentation::Categories{"Tuple", "Interval"} + }); + factory.registerFunction(); factory.registerFunction(); diff --git a/src/Interpreters/BloomFilterHash.h b/src/Interpreters/BloomFilterHash.h index 31532cc888a..b95abbfd770 100644 --- a/src/Interpreters/BloomFilterHash.h +++ b/src/Interpreters/BloomFilterHash.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp b/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp index 257bbda68eb..5d48391d56d 100644 --- a/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp +++ b/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp @@ -15,7 +15,7 @@ void ConvertFunctionOrLikeData::visit(ASTFunction & function, ASTPtr &) if (function.name != "or") return; - std::unordered_map> identifier_to_literals; + std::unordered_map> identifier_to_literals; for (auto & child : function.children) { if (auto * expr_list_fn = child->as()) @@ -51,10 +51,11 @@ void ConvertFunctionOrLikeData::visit(ASTFunction & function, ASTPtr &) regexp = "(?i)" + regexp; unique_elems.pop_back(); - auto it = identifier_to_literals.find(identifier); + auto it = identifier_to_literals.find(identifier->getAliasOrColumnName()); + if (it == identifier_to_literals.end()) { - it = identifier_to_literals.insert({identifier, std::make_shared(Field{Array{}})}).first; + it = identifier_to_literals.insert({identifier->getAliasOrColumnName(), std::make_shared(Field{Array{}})}).first; auto match = makeASTFunction("multiMatchAny"); match->arguments->children.push_back(arguments[0]); match->arguments->children.push_back(it->second); diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 551d20f835a..84f5570349b 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -263,6 +263,11 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as user_process_list.user_memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator_for_user); user_process_list.user_memory_tracker.setDescription("(for user)"); + if (!total_network_throttler && settings.max_network_bandwidth_for_all_users) + { + total_network_throttler = std::make_shared(settings.max_network_bandwidth_for_all_users); + } + if (!user_process_list.user_throttler) { if (settings.max_network_bandwidth_for_user) @@ -270,11 +275,6 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as else if (settings.max_network_bandwidth_for_all_users) user_process_list.user_throttler = total_network_throttler; } - - if (!total_network_throttler && settings.max_network_bandwidth_for_all_users) - { - total_network_throttler = std::make_shared(settings.max_network_bandwidth_for_all_users); - } } return res; diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 2a8b256c3d1..6a128d37e5d 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -113,12 +113,20 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) if (!is_cycle) { /// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. + /// Check size of the alias before cloning too large alias AST + alias_node->checkSize(data.settings.max_expanded_ast_elements); ast = alias_node->clone(); ast->setAlias(node_alias); } } else - ast = alias_node; + { + /// Check size of the alias before cloning too large alias AST + alias_node->checkSize(data.settings.max_expanded_ast_elements); + auto alias_name = ast->getAliasOrColumnName(); + ast = alias_node->clone(); + ast->setAlias(alias_name); + } } } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 2a41196c15d..d29aa248ec4 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -682,7 +682,7 @@ public: bool parseLambda() { - // 0. If empty - create function tuple with 0 args + // 1. If empty - create function tuple with 0 args if (isCurrentElementEmpty()) { auto function = makeASTFunction("tuple"); @@ -693,16 +693,16 @@ public: if (operands.size() != 1 || !operators.empty() || !mergeElement()) return false; - /// 1. If there is already tuple do nothing + /// 2. If there is already tuple do nothing if (tryGetFunctionName(elements.back()) == "tuple") { pushOperand(elements.back()); elements.pop_back(); } - /// 2. Put all elements in a single tuple + /// 3. Put all elements in a single tuple else { - auto function = makeASTFunction("tuple", elements); + auto function = makeASTFunction("tuple", std::move(elements)); elements.clear(); pushOperand(function); } @@ -1050,12 +1050,20 @@ public: if (!mergeElement()) return false; - // Special case for (('a', 'b')) -> tuple(('a', 'b')) if (!is_tuple && elements.size() == 1) + { + // Special case for (('a', 'b')) = tuple(('a', 'b')) if (auto * literal = elements[0]->as()) if (literal->value.getType() == Field::Types::Tuple) is_tuple = true; + // Special case for f(x, (y) -> z) = f(x, tuple(y) -> z) + auto test_pos = pos; + auto test_expected = expected; + if (parseOperator(test_pos, "->", test_expected)) + is_tuple = true; + } + finished = true; } @@ -1734,49 +1742,54 @@ public: if (state == 0) { + state = 1; + auto begin = pos; auto init_expected = expected; ASTPtr string_literal; + String literal; + //// A String literal followed INTERVAL keyword, /// the literal can be a part of an expression or /// include Number and INTERVAL TYPE at the same time - if (ParserStringLiteral{}.parse(pos, string_literal, expected)) + if (ParserStringLiteral{}.parse(pos, string_literal, expected) + && string_literal->as().value.tryGet(literal)) { - String literal; - if (string_literal->as().value.tryGet(literal)) + Tokens tokens(literal.data(), literal.data() + literal.size()); + IParser::Pos token_pos(tokens, 0); + Expected token_expected; + ASTPtr expr; + + if (!ParserNumber{}.parse(token_pos, expr, token_expected)) + return false; + + /// case: INTERVAL '1' HOUR + /// back to begin + if (!token_pos.isValid()) { - Tokens tokens(literal.data(), literal.data() + literal.size()); - IParser::Pos token_pos(tokens, 0); - Expected token_expected; - ASTPtr expr; - - if (!ParserNumber{}.parse(token_pos, expr, token_expected)) - { - return false; - } - else - { - /// case: INTERVAL '1' HOUR - /// back to begin - if (!token_pos.isValid()) - { - pos = begin; - expected = init_expected; - } - else - { - /// case: INTERVAL '1 HOUR' - if (!parseIntervalKind(token_pos, token_expected, interval_kind)) - return false; - - elements = {makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)}; - finished = true; - return true; - } - } + pos = begin; + expected = init_expected; + return true; } + + /// case: INTERVAL '1 HOUR' + if (!parseIntervalKind(token_pos, token_expected, interval_kind)) + return false; + + pushResult(makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)); + + /// case: INTERVAL '1 HOUR 1 SECOND ...' + while (token_pos.isValid()) + { + if (!ParserNumber{}.parse(token_pos, expr, token_expected) || + !parseIntervalKind(token_pos, token_expected, interval_kind)) + return false; + + pushResult(makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)); + } + + finished = true; } - state = 1; return true; } @@ -1795,6 +1808,17 @@ public: return true; } +protected: + bool getResultImpl(ASTPtr & node) override + { + if (elements.size() == 1) + node = elements[0]; + else + node = makeASTFunction("tuple", std::move(elements)); + + return true; + } + private: IntervalKind interval_kind; }; diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 4ab2c5b3802..cd94ca7ceae 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -109,6 +109,13 @@ bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) { std::lock_guard guard(processors_mutex); + /// Do not add new processors to existing list, since the query was already cancelled. + if (cancelled) + { + for (auto & processor : new_processors) + processor->cancel(); + return false; + } processors->insert(processors->end(), new_processors.begin(), new_processors.end()); } @@ -388,6 +395,7 @@ void ExecutingGraph::cancel() std::lock_guard guard(processors_mutex); for (auto & processor : *processors) processor->cancel(); + cancelled = true; } } diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index b374f968122..834ef5d4d9d 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -157,6 +157,7 @@ private: UpgradableMutex nodes_mutex; const bool profile_processors; + bool cancelled = false; }; } diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 51e5ee47077..e0da8031dd8 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -30,7 +30,7 @@ class ASTStorage; M(String, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ M(UInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) \ M(Bool, kafka_thread_per_consumer, false, "Provide independent thread for each consumer", 0) \ - M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Passible values: default, stream.", 0) \ + M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default, stream.", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ /** TODO: */ diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 99c14ede3e2..8ffd1059a58 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -29,6 +29,7 @@ #include #include +#include #include #include #include @@ -55,10 +56,15 @@ String Range::toString() const } -/// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. -String extractFixedPrefixFromLikePattern(const String & like_pattern) +/// Returns the prefix of like_pattern before the first wildcard, e.g. 'Hello\_World% ...' --> 'Hello\_World' +/// We call a pattern "perfect prefix" if: +/// - (1) the pattern has a wildcard +/// - (2) the first wildcard is '%' and is only followed by nothing or other '%' +/// e.g. 'test%' or 'test%% has perfect prefix 'test', 'test%x', 'test%_' or 'test_' has no perfect prefix. +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool requires_perfect_prefix) { String fixed_prefix; + fixed_prefix.reserve(like_pattern.size()); const char * pos = like_pattern.data(); const char * end = pos + like_pattern.size(); @@ -67,10 +73,13 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern) switch (*pos) { case '%': - [[fallthrough]]; case '_': + if (requires_perfect_prefix) + { + bool is_prefect_prefix = std::all_of(pos, end, [](auto c) { return c == '%'; }); + return is_prefect_prefix ? fixed_prefix : ""; + } return fixed_prefix; - case '\\': ++pos; if (pos == end) @@ -78,12 +87,13 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern) [[fallthrough]]; default: fixed_prefix += *pos; - break; } ++pos; } - + /// If we can reach this code, it means there was no wildcard found in the pattern, so it is not a perfect prefix + if (requires_perfect_prefix) + return ""; return fixed_prefix; } @@ -346,7 +356,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get()); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ false); if (prefix.empty()) return false; @@ -360,6 +370,27 @@ const KeyCondition::AtomMap KeyCondition::atom_map return true; } }, + { + "notLike", + [] (RPNElement & out, const Field & value) + { + if (value.getType() != Field::Types::String) + return false; + + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ true); + if (prefix.empty()) + return false; + + String right_bound = firstStringThatIsGreaterThanAllStringsWithPrefix(prefix); + + out.function = RPNElement::FUNCTION_NOT_IN_RANGE; + out.range = !right_bound.empty() + ? Range(prefix, true, right_bound, false) + : Range::createLeftBounded(prefix, true); + + return true; + } + }, { "startsWith", [] (RPNElement & out, const Field & value) diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index fe1bffa9305..138dce83db9 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -485,6 +485,6 @@ private: bool strict; }; -String extractFixedPrefixFromLikePattern(const String & like_pattern); +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool requires_perfect_prefix); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 977133a8ad8..051cb79b3d7 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1019,13 +1019,14 @@ void MergeTreeData::loadDataPartsFromDisk( if (!part_opt) return; - LOG_TRACE(log, "Loading part {} from disk {}", part_name, part_disk_ptr->getName()); const auto & part_info = *part_opt; auto single_disk_volume = std::make_shared("volume_" + part_name, part_disk_ptr, 0); auto data_part_storage = std::make_shared(single_disk_volume, relative_data_path, part_name); auto part = createPart(part_name, part_info, data_part_storage); bool broken = false; + LOG_TRACE(log, "Loading part {} ({}) from disk {}", part_name, part->getType().toString(), part_disk_ptr->getName()); + String part_path = fs::path(relative_data_path) / part_name; String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; if (part_disk_ptr->exists(marker_path)) @@ -1130,8 +1131,11 @@ void MergeTreeData::loadDataPartsFromDisk( { for (size_t thread = 0; thread < num_threads; ++thread) { - pool.scheduleOrThrowOnError([&, thread] + pool.scheduleOrThrowOnError([&, thread, thread_group = CurrentThread::getGroup()] { + if (thread_group) + CurrentThread::attachToIfDetached(thread_group); + while (true) { std::pair thread_part; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 674e02b16ec..4091d37ec11 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -781,6 +781,11 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( ReadFromMergeTree::IndexStats & index_stats) { const Settings & settings = context->getSettingsRef(); + + /// TODO: Analyzer syntax analyzer result + if (settings.allow_experimental_analyzer) + return; + std::optional partition_pruner; std::optional minmax_idx_condition; DataTypes minmax_columns_types; diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp index b19080b5097..9bc0e4e6dc0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index 33668b96a60..c2ed081ac00 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp index f80d40d2fa8..7efaf0866db 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp new file mode 100644 index 00000000000..e8287a2fd61 --- /dev/null +++ b/src/Storages/StorageDelta.cpp @@ -0,0 +1,300 @@ +#include "config.h" +#if USE_AWS_S3 + +#include +#include + +#include +#include +#include +#include + +#include +#include +#include + +#include + +#include +#include +#include + +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int S3_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INCORRECT_DATA; +} + +void DeltaLakeMetadata::setLastModifiedTime(const String & filename, uint64_t timestamp) +{ + file_update_time[filename] = timestamp; +} + +void DeltaLakeMetadata::remove(const String & filename, uint64_t /*timestamp */) +{ + bool erase = file_update_time.erase(filename); + if (!erase) + throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid table metadata, tried to remove {} before adding it", filename); +} + +std::vector DeltaLakeMetadata::ListCurrentFiles() && +{ + std::vector keys; + keys.reserve(file_update_time.size()); + + for (auto && [k, _] : file_update_time) + keys.push_back(k); + + return keys; +} + +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context) + : base_configuration(configuration_), table_path(table_path_) +{ + Init(context); +} + +void JsonMetadataGetter::Init(ContextPtr context) +{ + auto keys = getJsonLogFiles(); + + // read data from every json log file + for (const String & key : keys) + { + auto buf = createS3ReadBuffer(key, context); + + char c; + while (!buf->eof()) + { + /// May be some invalid characters before json. + while (buf->peek(c) && c != '{') + buf->ignore(); + + if (buf->eof()) + break; + + String json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); + + if (json_str.empty()) + continue; + + const JSON json(json_str); + handleJSON(json); + } + } +} + +std::vector JsonMetadataGetter::getJsonLogFiles() +{ + std::vector keys; + + const auto & client = base_configuration.client; + + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + bool is_finished{false}; + const auto bucket{base_configuration.uri.bucket}; + + request.SetBucket(bucket); + + /// DeltaLake format stores all metadata json files in _delta_log directory + static constexpr auto deltalake_metadata_directory = "_delta_log"; + request.SetPrefix(std::filesystem::path(table_path) / deltalake_metadata_directory); + + while (!is_finished) + { + outcome = client->ListObjectsV2(request); + if (!outcome.IsSuccess()) + throw Exception( + ErrorCodes::S3_ERROR, + "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", + quoteString(bucket), + quoteString(table_path), + backQuote(outcome.GetError().GetExceptionName()), + quoteString(outcome.GetError().GetMessage())); + + const auto & result_batch = outcome.GetResult().GetContents(); + for (const auto & obj : result_batch) + { + const auto & filename = obj.GetKey(); + + // DeltaLake metadata files have json extension + if (std::filesystem::path(filename).extension() == ".json") + keys.push_back(filename); + } + + /// Needed in case any more results are available + /// if so, we will continue reading, and not read keys that were already read + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + + /// Set to false if all of the results were returned. Set to true if more keys + /// are available to return. If the number of results exceeds that specified by + /// MaxKeys, all of the results might not be returned + is_finished = !outcome.GetResult().GetIsTruncated(); + } + + return keys; +} + +std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key, ContextPtr context) +{ + /// TODO: add parallel downloads + return std::make_shared( + base_configuration.client, + base_configuration.uri.bucket, + key, + base_configuration.uri.version_id, + /* max single read retries */10, + context->getReadSettings()); +} + +void JsonMetadataGetter::handleJSON(const JSON & json) +{ + if (json.has("add")) + { + auto path = json["add"]["path"].getString(); + auto timestamp = json["add"]["modificationTime"].getInt(); + + metadata.setLastModifiedTime(path, timestamp); + } + else if (json.has("remove")) + { + auto path = json["remove"]["path"].getString(); + auto timestamp = json["remove"]["deletionTimestamp"].getInt(); + + metadata.remove(path, timestamp); + } +} + +StorageDelta::StorageDelta( + const StorageS3Configuration & configuration_, + const StorageID & table_id_, + ColumnsDescription columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + std::optional format_settings_) + : IStorage(table_id_) + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} + , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) + , table_path(base_configuration.uri.key) +{ + StorageInMemoryMetadata storage_metadata; + StorageS3::updateS3Configuration(context_, base_configuration); + + JsonMetadataGetter getter{base_configuration, table_path, context_}; + + auto keys = getter.getFiles(); + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + + LOG_DEBUG(log, "New uri: {}", new_uri); + LOG_DEBUG(log, "Table path: {}", table_path); + + // set new url in configuration + StorageS3Configuration new_configuration; + new_configuration.url = new_uri; + new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; + new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key; + new_configuration.format = configuration_.format; + + + if (columns_.empty()) + { + columns_ = StorageS3::getTableStructureFromData( + new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); + storage_metadata.setColumns(columns_); + } + else + storage_metadata.setColumns(columns_); + + + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + s3engine = std::make_shared( + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */ false, + nullptr); +} + +Pipe StorageDelta::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) +{ + StorageS3::updateS3Configuration(context, base_configuration); + + return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); +} + +String StorageDelta::generateQueryFromKeys(std::vector && keys) +{ + // DeltaLake store data parts in different files + // keys are filenames of parts + // for StorageS3 to read all parts we need format {key1,key2,key3,...keyn} + std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ",")); + return new_query; +} + +void registerStorageDelta(StorageFactory & factory) +{ + factory.registerStorage( + "DeltaLake", + [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() < 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); + + StorageS3Configuration configuration; + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + if (engine_args.size() == 4) + configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + else + { + /// DeltaLake uses Parquet by default. + configuration.format = "Parquet"; + } + + return std::make_shared( + configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), std::nullopt); + }, + { + .supports_settings = true, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +} + +#endif diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h new file mode 100644 index 00000000000..e3bb4c0b416 --- /dev/null +++ b/src/Storages/StorageDelta.h @@ -0,0 +1,107 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +# include +# include + +# include +# include + +namespace Poco +{ +class Logger; +} + +namespace Aws::S3 +{ +class S3Client; +} + +namespace DB +{ + +// class to parse json deltalake metadata and find files needed for query in table +class DeltaLakeMetadata +{ +public: + DeltaLakeMetadata() = default; + + void setLastModifiedTime(const String & filename, uint64_t timestamp); + void remove(const String & filename, uint64_t timestamp); + + std::vector ListCurrentFiles() &&; + +private: + std::unordered_map file_update_time; +}; + +// class to get deltalake log json files and read json from them +class JsonMetadataGetter +{ +public: + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context); + + std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } + +private: + void Init(ContextPtr context); + + std::vector getJsonLogFiles(); + + std::shared_ptr createS3ReadBuffer(const String & key, ContextPtr context); + + void handleJSON(const JSON & json); + + StorageS3::S3Configuration base_configuration; + String table_path; + DeltaLakeMetadata metadata; +}; + +class StorageDelta : public IStorage +{ +public: + // 1. Parses internal file structure of table + // 2. Finds out parts with latest version + // 3. Creates url for underlying StorageS3 enigne to handle reads + StorageDelta( + const StorageS3Configuration & configuration_, + const StorageID & table_id_, + ColumnsDescription columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + std::optional format_settings_); + + String getName() const override { return "DeltaLake"; } + + // Reads latest version of DeltaLake table + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + +private: + void Init(); + + // DeltaLake stores data in parts in different files + // keys is vector of parts with latest version + // generateQueryFromKeys constructs query from parts filenames for + // underlying StorageS3 engine + static String generateQueryFromKeys(std::vector && keys); + + StorageS3::S3Configuration base_configuration; + std::shared_ptr s3engine; + Poco::Logger * log; + String table_path; +}; + +} + +#endif diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp new file mode 100644 index 00000000000..121856c4a57 --- /dev/null +++ b/src/Storages/StorageHudi.cpp @@ -0,0 +1,231 @@ +#include "config.h" + +#if USE_AWS_S3 + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; + extern const int LOGICAL_ERROR; +} + +StorageHudi::StorageHudi( + const StorageS3Configuration & configuration_, + const StorageID & table_id_, + ColumnsDescription columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + std::optional format_settings_) + : IStorage(table_id_) + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} + , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) + , table_path(base_configuration.uri.key) +{ + StorageInMemoryMetadata storage_metadata; + StorageS3::updateS3Configuration(context_, base_configuration); + + auto keys = getKeysFromS3(); + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(keys, configuration_.format); + + LOG_DEBUG(log, "New uri: {}", new_uri); + LOG_DEBUG(log, "Table path: {}", table_path); + + StorageS3Configuration new_configuration; + new_configuration.url = new_uri; + new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; + new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key; + new_configuration.format = configuration_.format; + + if (columns_.empty()) + { + columns_ = StorageS3::getTableStructureFromData( + new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); + storage_metadata.setColumns(columns_); + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + s3engine = std::make_shared( + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */ false, + nullptr); +} + +Pipe StorageHudi::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) +{ + StorageS3::updateS3Configuration(context, base_configuration); + return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); +} + +std::vector StorageHudi::getKeysFromS3() +{ + std::vector keys; + + const auto & client = base_configuration.client; + + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + bool is_finished{false}; + const auto bucket{base_configuration.uri.bucket}; + + request.SetBucket(bucket); + request.SetPrefix(table_path); + + while (!is_finished) + { + outcome = client->ListObjectsV2(request); + if (!outcome.IsSuccess()) + throw Exception( + ErrorCodes::S3_ERROR, + "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", + quoteString(bucket), + quoteString(table_path), + backQuote(outcome.GetError().GetExceptionName()), + quoteString(outcome.GetError().GetMessage())); + + const auto & result_batch = outcome.GetResult().GetContents(); + for (const auto & obj : result_batch) + { + const auto & filename = obj.GetKey().substr(table_path.size()); /// Object name without tablepath prefix. + keys.push_back(filename); + LOG_DEBUG(log, "Found file: {}", filename); + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + is_finished = !outcome.GetResult().GetIsTruncated(); + } + + return keys; +} + +String StorageHudi::generateQueryFromKeys(const std::vector & keys, const String & format) +{ + /// For each partition path take only latest file. + struct FileInfo + { + String filename; + UInt64 timestamp; + }; + std::unordered_map latest_parts; /// Partition path (directory) -> latest part file info. + + /// Make format lowercase. + const auto expected_extension= "." + Poco::toLower(format); + /// Filter only files with specific format. + auto keys_filter = [&](const String & key) { return std::filesystem::path(key).extension() == expected_extension; }; + + for (const auto & key : keys | std::views::filter(keys_filter)) + { + const auto key_path = fs::path(key); + const String filename = key_path.filename(); + const String partition_path = key_path.parent_path(); + + /// Every filename contains metadata split by "_", timestamp is after last "_". + const auto delim = key.find_last_of('_') + 1; + if (delim == std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format of metadata files"); + const auto timestamp = parse(key.substr(delim + 1)); + + auto it = latest_parts.find(partition_path); + if (it == latest_parts.end()) + { + latest_parts.emplace(partition_path, FileInfo{filename, timestamp}); + } + else if (it->second.timestamp < timestamp) + { + it->second = {filename, timestamp}; + } + } + + std::string list_of_keys; + + for (const auto & [directory, file_info] : latest_parts) + { + if (!list_of_keys.empty()) + list_of_keys += ","; + + list_of_keys += std::filesystem::path(directory) / file_info.filename; + } + + return "{" + list_of_keys + "}"; +} + + +void registerStorageHudi(StorageFactory & factory) +{ + factory.registerStorage( + "Hudi", + [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty() || engine_args.size() < 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Hudi requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); + + StorageS3Configuration configuration; + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + if (engine_args.size() == 4) + configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + else + { + // Apache Hudi uses Parquet by default + configuration.format = "Parquet"; + } + + auto format_settings = getFormatSettings(args.getContext()); + + return std::make_shared( + configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), format_settings); + }, + { + .supports_settings = true, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +} + +#endif diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h new file mode 100644 index 00000000000..bebda4cd4f6 --- /dev/null +++ b/src/Storages/StorageHudi.h @@ -0,0 +1,69 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +# include +# include + +namespace Poco +{ +class Logger; +} + +namespace Aws::S3 +{ +class S3Client; +} + +namespace DB +{ + +class StorageHudi : public IStorage +{ +public: + /// 1. Parses internal file structure of table. + /// 2. Finds out parts with latest version. + /// 3. Creates url for underlying StorageS3 enigne to handle reads. + StorageHudi( + const StorageS3Configuration & configuration_, + const StorageID & table_id_, + ColumnsDescription columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + std::optional format_settings_); + + String getName() const override { return "Hudi"; } + + + /// Reads latest version of Apache Hudi table + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + +private: + std::vector getKeysFromS3(); + + /// Apache Hudi store parts of data in different files. + /// Every part file has timestamp in it. + /// Every partition(directory) in Apache Hudi has different versions of part. + /// To find needed parts we need to find out latest part file for every partition. + /// Part format is usually parquet, but can differ. + static String generateQueryFromKeys(const std::vector & keys, const String & format); + + StorageS3::S3Configuration base_configuration; + std::shared_ptr s3engine; + Poco::Logger * log; + String table_path; +}; + +} + +#endif diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 23947a32092..81bbe2c86ae 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -212,6 +212,8 @@ public: private: friend class StorageS3Cluster; friend class TableFunctionS3Cluster; + friend class StorageHudi; + friend class StorageDelta; S3Configuration s3_configuration; std::vector keys; diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index d094fefddcb..1828c5932ad 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -16,18 +17,47 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription{{ - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"partition_id", std::make_shared(std::make_shared())}, - {"name", std::make_shared()}, - {"disk", std::make_shared()}, - {"reason", std::make_shared(std::make_shared())}, + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"partition_id", std::make_shared(std::make_shared())}, + {"name", std::make_shared()}, + {"bytes_on_disk", std::make_shared()}, + {"disk", std::make_shared()}, + {"path", std::make_shared()}, + {"reason", std::make_shared(std::make_shared())}, {"min_block_number", std::make_shared(std::make_shared())}, {"max_block_number", std::make_shared(std::make_shared())}, - {"level", std::make_shared(std::make_shared())} + {"level", std::make_shared(std::make_shared())} }}); setInMemoryMetadata(storage_metadata); } +static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & from, UInt64 & total_size) +{ + /// Files or directories of detached part may not exist. Only count the size of existing files. + if (disk->isFile(from)) + { + total_size += disk->getFileSize(from); + } + else + { + for (auto it = disk->iterateDirectory(from); it->isValid(); it->next()) + calculateTotalSizeOnDiskImpl(disk, fs::path(from) / it->name(), total_size); + } +} + +static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) +{ + UInt64 total_size = 0; + try + { + calculateTotalSizeOnDiskImpl(disk, from, total_size); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + return total_size; +} Pipe StorageSystemDetachedParts::read( const Names & /* column_names */, @@ -50,11 +80,14 @@ Pipe StorageSystemDetachedParts::read( for (const auto & p : parts) { size_t i = 0; + String detached_part_path = fs::path(MergeTreeData::DETACHED_DIR_NAME) / p.dir_name; new_columns[i++]->insert(info.database); new_columns[i++]->insert(info.table); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); + new_columns[i++]->insert(calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path)); new_columns[i++]->insert(p.disk->getName()); + new_columns[i++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string()); new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); new_columns[i++]->insert(p.valid_name ? p.min_block : Field()); new_columns[i++]->insert(p.valid_name ? p.max_block : Field()); diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index 275365648f3..731cdf94157 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -116,7 +116,7 @@ void StorageSystemMergeTreeMetadataCache::fillData(MutableColumns & res_columns, } else { - String target = extractFixedPrefixFromLikePattern(key); + String target = extractFixedPrefixFromLikePattern(key, /*requires_perfect_prefix*/ false); if (target.empty()) throw Exception( "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Storages/System/StorageSystemTableFunctions.cpp b/src/Storages/System/StorageSystemTableFunctions.cpp index 308cbc5686d..07a504edc5e 100644 --- a/src/Storages/System/StorageSystemTableFunctions.cpp +++ b/src/Storages/System/StorageSystemTableFunctions.cpp @@ -1,16 +1,23 @@ #include - #include +#include + namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_FUNCTION; +} + NamesAndTypesList StorageSystemTableFunctions::getNamesAndTypes() { return - { - {"name", std::make_shared()}, - {"description", std::make_shared()} - }; + { + {"name", std::make_shared()}, + {"description", std::make_shared()}, + {"allow_readonly", std::make_shared()} + }; } void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const @@ -20,7 +27,15 @@ void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, Context for (const auto & function_name : functions_names) { res_columns[0]->insert(function_name); - res_columns[1]->insert(factory.getDocumentation(function_name).description); + + auto properties = factory.tryGetProperties(function_name); + if (properties) + { + res_columns[1]->insert(properties->documentation.description); + res_columns[2]->insert(properties->allow_readonly); + } + else + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", function_name); } } diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index f9c8a735616..fd81b55ef61 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -32,6 +32,8 @@ void registerStorageMeiliSearch(StorageFactory& factory); #if USE_AWS_S3 void registerStorageS3(StorageFactory & factory); void registerStorageCOS(StorageFactory & factory); +void registerStorageHudi(StorageFactory & factory); +void registerStorageDelta(StorageFactory & factory); #endif #if USE_HDFS @@ -118,6 +120,8 @@ void registerStorages() #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); + registerStorageHudi(factory); + registerStorageDelta(factory); #endif #if USE_HDFS diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 82b6230dc30..da0de7e47f6 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace ProfileEvents @@ -25,8 +26,8 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr conte ProfileEvents::increment(ProfileEvents::TableFunctionExecute); AccessFlags required_access = getSourceAccessType(); - String function_name = getName(); - if ((function_name != "null") && (function_name != "view") && (function_name != "viewIfPermitted")) + auto table_function_properties = TableFunctionFactory::instance().tryGetProperties(getName()); + if (!(table_function_properties && table_function_properties->allow_readonly)) required_access |= AccessType::CREATE_TEMPORARY_TABLE; context->checkAccess(required_access); diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 4b9a87b93f1..a05edcd32c8 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -79,6 +80,14 @@ private: virtual const char * getStorageTypeName() const = 0; }; +/// Properties of table function that are independent of argument types and parameters. +struct TableFunctionProperties +{ + Documentation documentation; + bool allow_readonly = false; +}; + + using TableFunctionPtr = std::shared_ptr; diff --git a/src/TableFunctions/TableFunctionExplain.cpp b/src/TableFunctions/TableFunctionExplain.cpp index 02493aa9b0e..02b9308ed22 100644 --- a/src/TableFunctions/TableFunctionExplain.cpp +++ b/src/TableFunctions/TableFunctionExplain.cpp @@ -91,7 +91,7 @@ InterpreterExplainQuery TableFunctionExplain::getInterpreter(ContextPtr context) void registerTableFunctionExplain(TableFunctionFactory & factory) { - factory.registerFunction({R"( + factory.registerFunction({.documentation = {R"( Returns result of EXPLAIN query. The function should not be called directly but can be invoked via `SELECT * FROM (EXPLAIN )`. @@ -103,7 +103,7 @@ Example: )", {{"1", "SELECT explain FROM (EXPLAIN AST SELECT * FROM system.numbers) WHERE explain LIKE '%Asterisk%'"}} -}); +}}); } diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 5ed22e39300..3d2a72ddc9e 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -16,16 +16,15 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - void TableFunctionFactory::registerFunction( - const std::string & name, TableFunctionCreator creator, Documentation doc, CaseSensitiveness case_sensitiveness) + const std::string & name, Value value, CaseSensitiveness case_sensitiveness) { - if (!table_functions.emplace(name, TableFunctionFactoryData{creator, doc}).second) + if (!table_functions.emplace(name, value).second) throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); if (case_sensitiveness == CaseInsensitive - && !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, doc}).second) + && !case_insensitive_table_functions.emplace(Poco::toLower(name), value).second) throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } @@ -59,13 +58,13 @@ TableFunctionPtr TableFunctionFactory::tryGet( auto it = table_functions.find(name); if (table_functions.end() != it) { - res = it->second.first(); + res = it->second.creator(); } else { it = case_insensitive_table_functions.find(Poco::toLower(name)); if (case_insensitive_table_functions.end() != it) - res = it->second.first(); + res = it->second.creator(); } if (!res) @@ -86,13 +85,29 @@ bool TableFunctionFactory::isTableFunctionName(const std::string & name) const return table_functions.contains(name); } -Documentation TableFunctionFactory::getDocumentation(const std::string & name) const +std::optional TableFunctionFactory::tryGetProperties(const String & name) const { - auto it = table_functions.find(name); - if (it == table_functions.end()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", name); + return tryGetPropertiesImpl(name); +} - return it->second.second; +std::optional TableFunctionFactory::tryGetPropertiesImpl(const String & name_param) const +{ + String name = getAliasToOrName(name_param); + Value found; + + /// Find by exact match. + if (auto it = table_functions.find(name); it != table_functions.end()) + { + found = it->second; + } + + if (auto jt = case_insensitive_table_functions.find(Poco::toLower(name)); jt != case_insensitive_table_functions.end()) + found = jt->second; + + if (found.creator) + return found.properties; + + return {}; } TableFunctionFactory & TableFunctionFactory::instance() diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index 8ff352ff9ac..2cc648ba181 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -18,7 +17,24 @@ namespace DB class Context; using TableFunctionCreator = std::function; -using TableFunctionFactoryData = std::pair; + +struct TableFunctionFactoryData +{ + TableFunctionCreator creator; + TableFunctionProperties properties; + + TableFunctionFactoryData() = default; + TableFunctionFactoryData(const TableFunctionFactoryData &) = default; + TableFunctionFactoryData & operator = (const TableFunctionFactoryData &) = default; + + template + requires (!std::is_same_v) + TableFunctionFactoryData(Creator creator_, TableFunctionProperties properties_ = {}) /// NOLINT + : creator(std::forward(creator_)), properties(std::move(properties_)) + { + } +}; + /** Lets you get a table function by its name. */ @@ -31,15 +47,16 @@ public: /// No locking, you must register all functions before usage of get. void registerFunction( const std::string & name, - TableFunctionCreator creator, - Documentation doc = {}, + Value value, CaseSensitiveness case_sensitiveness = CaseSensitive); template - void registerFunction(Documentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) + void registerFunction(TableFunctionProperties properties = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) { auto creator = []() -> TableFunctionPtr { return std::make_shared(); }; - registerFunction(Function::name, std::move(creator), std::move(doc), case_sensitiveness); + registerFunction(Function::name, + TableFunctionFactoryData{std::move(creator), {std::move(properties)}} , + case_sensitiveness); } /// Throws an exception if not found. @@ -48,7 +65,7 @@ public: /// Returns nullptr if not found. TableFunctionPtr tryGet(const std::string & name, ContextPtr context) const; - Documentation getDocumentation(const std::string & name) const; + std::optional tryGetProperties(const String & name) const; bool isTableFunctionName(const std::string & name) const; @@ -61,6 +78,8 @@ private: String getFactoryName() const override { return "TableFunctionFactory"; } + std::optional tryGetPropertiesImpl(const String & name) const; + TableFunctions table_functions; TableFunctions case_insensitive_table_functions; }; diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 8d1c06c7c4d..1ddbb48962d 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -91,7 +91,7 @@ StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_functio void registerTableFunctionGenerate(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index 1a35830b06c..9ff07cc1946 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -52,6 +52,6 @@ StoragePtr TableFunctionNull::executeImpl(const ASTPtr & /*ast_function*/, Conte void registerTableFunctionNull(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 0ef1d3815d9..2056cd838f5 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -51,8 +51,8 @@ StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_f void registerTableFunctionNumbers(TableFunctionFactory & factory) { - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction>({.documentation = {}, .allow_readonly = true}); + factory.registerFunction>({.documentation = {}, .allow_readonly = true}); } template diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 2c62e29810f..14e0774cf06 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -346,8 +346,8 @@ void registerTableFunctionRemote(TableFunctionFactory & factory) { factory.registerFunction("remote", [] () -> TableFunctionPtr { return std::make_shared("remote"); }); factory.registerFunction("remoteSecure", [] () -> TableFunctionPtr { return std::make_shared("remote", /* secure = */ true); }); - factory.registerFunction("cluster", [] () -> TableFunctionPtr { return std::make_shared("cluster"); }); - factory.registerFunction("clusterAllReplicas", [] () -> TableFunctionPtr { return std::make_shared("clusterAllReplicas"); }); + factory.registerFunction("cluster", {[] () -> TableFunctionPtr { return std::make_shared("cluster"); }, {.documentation = {}, .allow_readonly = true}}); + factory.registerFunction("clusterAllReplicas", {[] () -> TableFunctionPtr { return std::make_shared("clusterAllReplicas"); }, {.documentation = {}, .allow_readonly = true}}); } } diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 7b8de69a4f8..05574825275 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -147,7 +147,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, Context void registerTableFunctionValues(TableFunctionFactory & factory) { - factory.registerFunction({}, TableFunctionFactory::CaseInsensitive); + factory.registerFunction({.documentation = {}, .allow_readonly = true}, TableFunctionFactory::CaseInsensitive); } } diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index c53d26a794b..c999cba08e9 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -55,7 +55,7 @@ StoragePtr TableFunctionView::executeImpl( void registerTableFunctionView(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index ba3d2cb9d16..6128fe0a36f 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -107,7 +107,7 @@ bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 126166b547b..5874fca67e6 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -48,7 +48,7 @@ StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_fun void registerTableFunctionZeros(TableFunctionFactory & factory) { - factory.registerFunction>({R"( + factory.registerFunction>({.documentation = {R"( Generates a stream of zeros (a table with one column 'zero' of type 'UInt8') of specified size. This table function is used in performance tests, where you want to spend as little time as possible to data generation while testing some other parts of queries. @@ -62,9 +62,9 @@ This query will test the speed of `randomPrintableASCII` function using single t See also the `system.zeros` table. )", {{"1", "SELECT count() FROM zeros(100000000) WHERE NOT ignore(randomPrintableASCII(10))"}} -}); +}}); - factory.registerFunction>({R"( + factory.registerFunction>({.documentation = {R"( Generates a stream of zeros (a table with one column 'zero' of type 'UInt8') of specified size. This table function is used in performance tests, where you want to spend as little time as possible to data generation while testing some other parts of queries. @@ -78,7 +78,7 @@ This query will test the speed of `randomPrintableASCII` function using multiple See also the `system.zeros` table. )", {{"1", "SELECT count() FROM zeros_mt(1000000000) WHERE NOT ignore(randomPrintableASCII(10))"}} -}); +}}); } template diff --git a/tests/ci/mark_release_ready.py b/tests/ci/mark_release_ready.py new file mode 100644 index 00000000000..be1771e62bd --- /dev/null +++ b/tests/ci/mark_release_ready.py @@ -0,0 +1,25 @@ +#!/usr/bin/env python3 + +from commit_status_helper import get_commit +from env_helper import GITHUB_JOB_URL +from get_robot_token import get_best_robot_token +from github_helper import GitHub +from pr_info import PRInfo + +RELEASE_READY_STATUS = "Ready for release" + + +def main(): + pr_info = PRInfo() + gh = GitHub(get_best_robot_token(), per_page=100) + commit = get_commit(gh, pr_info.sha) + commit.create_status( + context=RELEASE_READY_STATUS, + description="the release can be created from the commit", + state="success", + target_url=GITHUB_JOB_URL(), + ) + + +if __name__ == "__main__": + main() diff --git a/tests/ci/release.py b/tests/ci/release.py index c0c248aa513..fd4bda3eae4 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -118,6 +118,8 @@ class Release: except subprocess.CalledProcessError: logging.fatal("Repo contains uncommitted changes") raise + if self._git.branch != "master": + raise Exception("the script must be launched only from master") self.set_release_branch() diff --git a/tests/config/config.d/display_name.xml b/tests/config/config.d/display_name.xml new file mode 100644 index 00000000000..d386a8efaf2 --- /dev/null +++ b/tests/config/config.d/display_name.xml @@ -0,0 +1,3 @@ + + functional-tests + diff --git a/tests/config/install.sh b/tests/config/install.sh index 9a172f0be15..876d05507fe 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -51,6 +51,7 @@ ln -sf $SRC_PATH/config.d/system_unfreeze.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 4eab305358b..551466cf583 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -353,6 +353,11 @@ class ClickhouseIntegrationTestsRunner: ) def _compress_logs(self, dir, relpaths, result_path): + # We execute sync in advance to have all files written after containers + # are finished or killed + subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + "sync", shell=True + ) subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL "tar czf {} -C {} {}".format(result_path, dir, " ".join(relpaths)), shell=True, diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 320209b5d7e..c53bc5a9d0d 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -385,7 +385,7 @@ def test_system_detached_parts(drop_detached_parts_table): ) res = q( - "select * from system.detached_parts where table like 'sdp_%' order by table, name" + "select system.detached_parts.* except (bytes_on_disk, `path`) from system.detached_parts where table like 'sdp_%' order by table, name" ) assert ( res == "default\tsdp_0\tall\tall_1_1_0\tdefault\t\t1\t1\t0\n" diff --git a/tests/integration/test_storage_delta/__init__.py b/tests/integration/test_storage_delta/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py new file mode 100644 index 00000000000..76dab554a57 --- /dev/null +++ b/tests/integration/test_storage_delta/test.py @@ -0,0 +1,132 @@ +import logging +import os +import json + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def prepare_s3_bucket(started_cluster): + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + + +def upload_test_table(started_cluster): + bucket = started_cluster.minio_bucket + + for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): + address_without_prefix = address[len(SCRIPT_DIR) :] + + for name in files: + started_cluster.minio_client.fput_object( + bucket, + os.path.join(address_without_prefix, name), + os.path.join(address, name), + ) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance("main_server", with_minio=True) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + upload_test_table(cluster) + logging.info("Test table uploaded") + + yield cluster + + finally: + cluster.shutdown() + + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def test_create_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + + create_query = f"""CREATE TABLE deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + + +def test_select_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + columns = [ + "begin_lat", + "begin_lon", + "driver", + "end_lat", + "end_lon", + "fare", + "rider", + "ts", + "uuid", + ] + + # create query in case table doesn't exist + create_query = f"""CREATE TABLE IF NOT EXISTS deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + + select_query = "SELECT {} FROM deltalake FORMAT TSV" + + for column_name in columns: + result = run_query(instance, select_query.format(column_name)).splitlines() + assert len(result) > 0 diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc b/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 00000000000..327e2898336 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc differ diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc b/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 00000000000..fd48c979576 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc differ diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..45fd233fd48 --- /dev/null +++ b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json @@ -0,0 +1,9 @@ +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"6eae6736-e014-439d-8301-070bfa5fc358","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"begin_lat\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"begin_lon\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"driver\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"end_lat\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"end_lon\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"fare\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"partitionpath\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"rider\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"ts\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"uuid\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["partitionpath"],"configuration":{},"createdTime":1661963201495}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795,"modificationTime":1661963202988,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966,"modificationTime":1661963203028,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661963203056,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878,"modificationTime":1661963202988,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2878,"modificationTime":1661963203044,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661963203072,"dataChange":true}} +{"commitInfo":{"timestamp":1661963203129,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"partitionpath\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"6","numOutputRows":"10","numOutputBytes":"17107"},"engineInfo":"Apache-Spark/3.2.2 Delta-Lake/1.1.0"}} diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000..408d5e1ded7 --- /dev/null +++ b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json @@ -0,0 +1,13 @@ +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795,"modificationTime":1661964654518,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966,"modificationTime":1661964654558,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2794,"modificationTime":1661964654586,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878,"modificationTime":1661964654518,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2879,"modificationTime":1661964654558,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661964654582,"dataChange":true}} +{"remove":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795}} +{"remove":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966}} +{"remove":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2878}} +{"remove":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795}} +{"remove":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878}} +{"remove":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795}} +{"commitInfo":{"timestamp":1661964655251,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"partitionpath\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"6","numOutputRows":"10","numOutputBytes":"17107"},"engineInfo":"Apache-Spark/3.2.2 Delta-Lake/1.1.0"}} diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc new file mode 100644 index 00000000000..0d07fe9805f Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc new file mode 100644 index 00000000000..1b17a91ca75 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc new file mode 100644 index 00000000000..f7f1df8479d Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet.crc new file mode 100644 index 00000000000..88414b442d1 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet new file mode 100644 index 00000000000..9be4fc88758 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet new file mode 100644 index 00000000000..a9652efacb0 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet new file mode 100644 index 00000000000..fad00b6c557 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet new file mode 100644 index 00000000000..1e79eb2d238 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc new file mode 100644 index 00000000000..ded74b8da6d Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc new file mode 100644 index 00000000000..e0123ca128f Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc new file mode 100644 index 00000000000..065f09eab7a Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc new file mode 100644 index 00000000000..8688bc7218d Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet new file mode 100644 index 00000000000..83d3695feb6 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet new file mode 100644 index 00000000000..53b9a8b10aa Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet new file mode 100644 index 00000000000..0fe5f31b711 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet new file mode 100644 index 00000000000..b43afdfcd30 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc new file mode 100644 index 00000000000..6756a13f94b Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc new file mode 100644 index 00000000000..c099f0af8e4 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc new file mode 100644 index 00000000000..73d821f134b Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc new file mode 100644 index 00000000000..48b7b139c7a Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet new file mode 100644 index 00000000000..4fde2de24ac Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet new file mode 100644 index 00000000000..d936d431527 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet new file mode 100644 index 00000000000..8f9e97a5287 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet new file mode 100644 index 00000000000..6c82903dc90 Binary files /dev/null and b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet differ diff --git a/tests/integration/test_storage_hudi/__init__.py b/tests/integration/test_storage_hudi/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py new file mode 100644 index 00000000000..549421afd89 --- /dev/null +++ b/tests/integration/test_storage_hudi/test.py @@ -0,0 +1,151 @@ +import logging +import os +import json + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def prepare_s3_bucket(started_cluster): + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + + +def upload_test_table(started_cluster): + bucket = started_cluster.minio_bucket + + for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): + address_without_prefix = address[len(SCRIPT_DIR) :] + + for name in files: + started_cluster.minio_client.fput_object( + bucket, + os.path.join(address_without_prefix, name), + os.path.join(address, name), + ) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance("main_server", with_minio=True) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + upload_test_table(cluster) + logging.info("Test table uploaded") + + yield cluster + + finally: + cluster.shutdown() + + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def test_create_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + + create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + + +def test_select_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + columns = [ + "_hoodie_commit_time", + "_hoodie_commit_seqno", + "_hoodie_record_key", + "_hoodie_partition_path", + "_hoodie_file_name", + "begin_lat", + "begin_lon", + "driver", + "end_lat", + "end_lon", + "fare", + "partitionpath", + "rider", + "ts", + "uuid", + ] + + # create query in case table doesn't exist + create_query = f"""CREATE TABLE IF NOT EXISTS hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + + select_query = "SELECT {} FROM hudi FORMAT TSV" + + for column_name in columns: + result = run_query(instance, select_query.format(column_name)).splitlines() + assert len(result) > 0 + + # test if all partition paths is presented in result + distinct_select_query = ( + "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" + ) + result = run_query(instance, distinct_select_query) + expected = [ + "americas/brazil/sao_paulo", + "americas/united_states/san_francisco", + "asia/india/chennai", + ] + + assert TSV(result) == TSV(expected) diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc new file mode 100644 index 00000000000..4bba97b9515 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc new file mode 100644 index 00000000000..3b7b044936a Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc new file mode 100644 index 00000000000..21984c840bc Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc new file mode 100644 index 00000000000..f67f76b7101 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit new file mode 100644 index 00000000000..f8d6c248f49 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit @@ -0,0 +1,165 @@ +{ + "partitionToWriteStats" : { + "americas/brazil/sao_paulo" : [ { + "fileId" : "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0", + "path" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 437831, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/brazil/sao_paulo", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437831, + "minEventTime" : null, + "maxEventTime" : null + } ], + "americas/united_states/san_francisco" : [ { + "fileId" : "34b1b177-f0af-467b-9214-473ead268e55-0", + "path" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 5, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 438186, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/united_states/san_francisco", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 438186, + "minEventTime" : null, + "maxEventTime" : null + } ], + "asia/india/chennai" : [ { + "fileId" : "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0", + "path" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 437623, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "asia/india/chennai", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437623, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"test_table_record\",\"namespace\":\"hoodie.test_table\",\"fields\":[{\"name\":\"begin_lat\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"begin_lon\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"driver\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"end_lat\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"end_lon\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"fare\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"partitionpath\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"rider\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"uuid\",\"type\":[\"null\",\"string\"],\"default\":null}]}" + }, + "operationType" : "UPSERT", + "writeStats" : [ { + "fileId" : "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0", + "path" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 437831, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/brazil/sao_paulo", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437831, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "34b1b177-f0af-467b-9214-473ead268e55-0", + "path" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 5, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 438186, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/united_states/san_francisco", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 438186, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0", + "path" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 437623, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "asia/india/chennai", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437623, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 563, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", "asia/india/chennai" ], + "fileIdAndRelativePaths" : { + "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "34b1b177-f0af-467b-9214-473ead268e55-0" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight new file mode 100644 index 00000000000..f5ef5c92c2b --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight @@ -0,0 +1,161 @@ +{ + "partitionToWriteStats" : { + "americas/brazil/sao_paulo" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "americas/united_states/san_francisco" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "asia/india/chennai" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", "asia/india/chennai" ], + "fileIdAndRelativePaths" : { + "" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties b/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties new file mode 100644 index 00000000000..9ae364baf33 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties @@ -0,0 +1,21 @@ +#Updated at 2022-08-30T08:36:49.089844Z +#Tue Aug 30 08:36:49 UTC 2022 +hoodie.table.type=COPY_ON_WRITE +hoodie.table.metadata.partitions=files +hoodie.table.precombine.field=ts +hoodie.table.partition.fields=partitionpath +hoodie.archivelog.folder=archived +hoodie.timeline.layout.version=1 +hoodie.table.checksum=2702201862 +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.timeline.timezone=LOCAL +hoodie.table.recordkey.fields=uuid +hoodie.table.name=test_table +hoodie.partition.metafile.use.base.format=false +hoodie.datasource.write.hive_style_partitioning=false +hoodie.populate.meta.fields=true +hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator +hoodie.table.base.file.format=PARQUET +hoodie.database.name= +hoodie.datasource.write.partitionpath.urlencode=false +hoodie.table.version=5 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc new file mode 100644 index 00000000000..352b882ec5a Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc new file mode 100644 index 00000000000..b6b8f7fc1a3 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc new file mode 100644 index 00000000000..3b7b044936a Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc new file mode 100644 index 00000000000..54abc5e9b72 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc new file mode 100644 index 00000000000..ec7cb5faf56 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc new file mode 100644 index 00000000000..3b7b044936a Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc new file mode 100644 index 00000000000..509ae4501ee Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit new file mode 100644 index 00000000000..6d22af6dd2e --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit @@ -0,0 +1,97 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.1_0-52-57", + "prevCommit" : "00000000000000", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 10928, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 10928, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.1_0-52-57", + "prevCommit" : "00000000000000", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 10928, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 10928, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 67, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "files-0000" : "files/.files-0000_00000000000000.log.1_0-52-57" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight new file mode 100644 index 00000000000..bb2542e0186 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight @@ -0,0 +1,116 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "" : null, + "files-0000" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit new file mode 100644 index 00000000000..210201f7135 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit @@ -0,0 +1,97 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.2_0-83-93", + "prevCommit" : "00000000000000", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 11180, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 11180, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57", ".files-0000_00000000000000.log.1_0-0-0", ".files-0000_00000000000000.log.2_0-83-93" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.2_0-83-93", + "prevCommit" : "00000000000000", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 11180, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 11180, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57", ".files-0000_00000000000000.log.1_0-0-0", ".files-0000_00000000000000.log.2_0-83-93" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 39, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "files-0000" : "files/.files-0000_00000000000000.log.2_0-83-93" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight new file mode 100644 index 00000000000..ea1b6a10c13 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight @@ -0,0 +1,116 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "" : null, + "files-0000" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties new file mode 100644 index 00000000000..845df718f6d --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties @@ -0,0 +1,14 @@ +#Properties saved on 2022-08-30T08:36:47.657528Z +#Tue Aug 30 08:36:47 UTC 2022 +hoodie.compaction.payload.class=org.apache.hudi.metadata.HoodieMetadataPayload +hoodie.table.type=MERGE_ON_READ +hoodie.archivelog.folder=archived +hoodie.timeline.layout.version=1 +hoodie.table.checksum=1983687495 +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.recordkey.fields=key +hoodie.table.name=test_table_metadata +hoodie.populate.meta.fields=false +hoodie.table.keygenerator.class=org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator +hoodie.table.base.file.format=HFILE +hoodie.table.version=5 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc new file mode 100644 index 00000000000..e016a7f5262 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc new file mode 100644 index 00000000000..c1136be0c0e Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc new file mode 100644 index 00000000000..0aaf9e50256 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc new file mode 100644 index 00000000000..1b5364eed34 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 new file mode 100644 index 00000000000..9bf687c1a4b Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-52-57 b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-52-57 new file mode 100644 index 00000000000..b7e40917ace Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-52-57 differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 new file mode 100644 index 00000000000..13a41e55cb2 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 differ diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata new file mode 100644 index 00000000000..abeb826f8a4 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:48 UTC 2022 +commitTime=00000000000000 +partitionDepth=1 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc new file mode 100644 index 00000000000..901b972aff0 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc new file mode 100644 index 00000000000..eaff8b6a2b1 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet new file mode 100644 index 00000000000..d929431ccd3 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet differ diff --git a/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/..hoodie_partition_metadata.crc new file mode 100644 index 00000000000..901b972aff0 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/..hoodie_partition_metadata.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet.crc b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet.crc new file mode 100644 index 00000000000..d0e239fd35b Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet new file mode 100644 index 00000000000..053841a0b91 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet differ diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc new file mode 100644 index 00000000000..901b972aff0 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc new file mode 100644 index 00000000000..9fae4a53043 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc differ diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet new file mode 100644 index 00000000000..c1ba250ed09 Binary files /dev/null and b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet differ diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.sql b/tests/queries/0_stateless/00502_custom_partitioning_local.sql index c85a978af68..3d5f71429fe 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.sql @@ -18,7 +18,7 @@ ALTER TABLE not_partitioned DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned; SELECT 'system.detached_parts after DETACH PARTITION:'; -SELECT system.detached_parts.* EXCEPT disk FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; +SELECT system.detached_parts.* EXCEPT (bytes_on_disk, `path`, disk) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; DROP TABLE not_partitioned; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index ad27b86c6f5..4ffa31a5618 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -139,7 +139,9 @@ CREATE TABLE system.detached_parts `table` String, `partition_id` Nullable(String), `name` String, + `bytes_on_disk` UInt64, `disk` String, + `path` String, `reason` Nullable(String), `min_block_number` Nullable(Int64), `max_block_number` Nullable(Int64), @@ -1013,7 +1015,8 @@ COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.table_functions ( `name` String, - `description` String + `description` String, + `allow_readonly` UInt8 ) ENGINE = SystemTableFunctions COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/tests/queries/0_stateless/02313_displayname.reference b/tests/queries/0_stateless/02313_displayname.reference new file mode 100644 index 00000000000..604a4944b33 --- /dev/null +++ b/tests/queries/0_stateless/02313_displayname.reference @@ -0,0 +1 @@ +functional-tests diff --git a/tests/queries/0_stateless/02313_displayname.sql b/tests/queries/0_stateless/02313_displayname.sql new file mode 100644 index 00000000000..6b6411b798b --- /dev/null +++ b/tests/queries/0_stateless/02313_displayname.sql @@ -0,0 +1 @@ +select displayName(); diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference new file mode 100644 index 00000000000..e635aec1163 --- /dev/null +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -0,0 +1,32 @@ +SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) +--- +-1 +2022-10-12 +2022-10-10 +(2) +(0) +2022-11-12 +2022-09-10 +(1,2) +(1,0) +--- +2022-10-12 +2022-10-10 +2022-10-12 +(2) Tuple(IntervalSecond) +(0) Tuple(IntervalSecond) +--- +3 IntervalSecond +(1,2) Tuple(IntervalHour, IntervalSecond) +(1,1,1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) +(2,1) Tuple(IntervalSecond, IntervalHour) +-3 IntervalSecond +(-1,-2) Tuple(IntervalHour, IntervalSecond) +(-1,-1,-1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) +(-2,-1) Tuple(IntervalSecond, IntervalHour) +--- +1 2022-03-01 +1 2022-02-28 +1 2023-07-11 00:01:59 +1 2021-07-31 23:00:00 +1 2021-06-10 23:59:59.000 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql new file mode 100644 index 00000000000..be9ccb50d92 --- /dev/null +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -0,0 +1,71 @@ +EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; + +SELECT '---'; + +SELECT negate(INTERVAL 1 SECOND); +SELECT addTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); +SELECT subtractTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); +SELECT addInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT subtractInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); + +SELECT addTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); +SELECT subtractTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); +SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT subtractInterval(tuple(INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); + +SELECT '---'; + +SELECT '2022-10-11'::Date + tuple(INTERVAL 1 DAY); +SELECT '2022-10-11'::Date - tuple(INTERVAL 1 DAY); +SELECT tuple(INTERVAL 1 DAY) + '2022-10-11'::Date; +SELECT tuple(INTERVAL 1 DAY) - '2022-10-11'::Date; -- { serverError 43 } + +WITH tuple(INTERVAL 1 SECOND) + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH tuple(INTERVAL 1 SECOND) - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + tuple(INTERVAL 1 SECOND) as expr SELECT expr, toTypeName(expr); -- { serverError 43 } +WITH INTERVAL 1 SECOND - tuple(INTERVAL 1 SECOND) as expr SELECT expr, toTypeName(expr); -- { serverError 43 } + +SELECT '---'; + +WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 HOUR + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + INTERVAL 1 HOUR + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); + +WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 HOUR - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 SECOND - INTERVAL 1 HOUR - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); + +SELECT '---'; + +WITH '2022-01-30'::Date + INTERVAL 1 MONTH + INTERVAL 1 DAY AS e1, + '2022-01-30'::Date + (INTERVAL 1 MONTH + INTERVAL 1 DAY) AS e2, + '2022-01-30'::Date + (INTERVAL 1 MONTH, INTERVAL 1 DAY) AS e3, + '2022-01-30'::Date + INTERVAL '1 MONTH 1 DAY' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; + +WITH '2022-01-30'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, + '2022-01-30'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, + '2022-01-30'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, + '2022-01-30'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; + +WITH '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR AS e1, + '2022-10-11'::Date + (INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR) AS e2, + '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR) AS e3, + '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; + +WITH '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR AS e1, + '2022-10-11'::DateTime + (- INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR) AS e2, + '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR) AS e3, + '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; + + +WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND AS e1, + '2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2, + '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3, + '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; \ No newline at end of file diff --git a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference new file mode 100644 index 00000000000..02357561e02 --- /dev/null +++ b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference @@ -0,0 +1,2 @@ +200000 +200000 diff --git a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql new file mode 100644 index 00000000000..e821b16ed5c --- /dev/null +++ b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql @@ -0,0 +1,12 @@ +CREATE TABLE data (str String) ENGINE=MergeTree ORDER BY str; +INSERT INTO data (str) SELECT 'aa' FROM numbers(100000); +INSERT INTO data (str) SELECT 'ba' FROM numbers(100000); +INSERT INTO data (str) SELECT 'ca' FROM numbers(100000); +SELECT count() FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; +SELECT count() FROM data WHERE str NOT LIKE 'a%%' SETTINGS force_primary_key=1; +SELECT count() FROM data WHERE str NOT LIKE 'a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE '%a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a_' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a%_' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE '_a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a%\_' SETTINGS force_primary_key=1; -- { serverError 277 } diff --git a/tests/queries/0_stateless/02462_int_to_date.reference b/tests/queries/0_stateless/02462_int_to_date.reference deleted file mode 100644 index f31441cf3b8..00000000000 --- a/tests/queries/0_stateless/02462_int_to_date.reference +++ /dev/null @@ -1,4 +0,0 @@ -20221011 2022-10-11 1665519765 -20221011 2022-10-11 1665519765 -20221011 2022-10-11 1665519765 Int32 -20221011 2022-10-11 1665519765 UInt32 diff --git a/tests/queries/0_stateless/02462_int_to_date.sql b/tests/queries/0_stateless/02462_int_to_date.sql deleted file mode 100644 index cd470ca12f6..00000000000 --- a/tests/queries/0_stateless/02462_int_to_date.sql +++ /dev/null @@ -1,4 +0,0 @@ -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp; -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp; -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); diff --git a/tests/queries/0_stateless/02462_number_to_datetype.reference b/tests/queries/0_stateless/02462_number_to_datetype.reference new file mode 100644 index 00000000000..22cee1308ff --- /dev/null +++ b/tests/queries/0_stateless/02462_number_to_datetype.reference @@ -0,0 +1,54 @@ +-- { echoOn } + +-- toDate +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int64 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt64 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519700 Float32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Float64 +-- toDate32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int64 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt64 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519700 Float32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Float64 +-- toDateTime +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Int64 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 UInt64 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Int32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 UInt32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:24 1665519700 Float32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Float64 +-- toDateTime64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Int64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 UInt64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Int32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 UInt32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:21:54.304 1665519700 Float32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Float64 diff --git a/tests/queries/0_stateless/02462_number_to_datetype.sql b/tests/queries/0_stateless/02462_number_to_datetype.sql new file mode 100644 index 00000000000..5d361f2b0ba --- /dev/null +++ b/tests/queries/0_stateless/02462_number_to_datetype.sql @@ -0,0 +1,35 @@ +-- { echoOn } + +-- toDate +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +-- toDate32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +-- toDateTime +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +-- toDateTime64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +-- { echoOff } diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference new file mode 100644 index 00000000000..500004a06b2 --- /dev/null +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference @@ -0,0 +1,3 @@ +0 +123 +0 diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql new file mode 100644 index 00000000000..c5c82d2e2bf --- /dev/null +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql @@ -0,0 +1,4 @@ +SELECT * from numbers(1); +SELECT * from format('TSV', '123'); +SELECT * from numbers(1) SETTINGS readonly=1; +SELECT * from format('TSV', '123') SETTINGS readonly=1; -- { serverError READONLY } \ No newline at end of file diff --git a/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference new file mode 100644 index 00000000000..623ba8fde99 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference @@ -0,0 +1,3 @@ +45 +-- +45 diff --git a/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql new file mode 100644 index 00000000000..eda90529166 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql @@ -0,0 +1,7 @@ +SET allow_experimental_analyzer = 1; + +WITH subquery AS (SELECT sum(number) FROM numbers(10)) SELECT * FROM subquery; + +SELECT '--'; + +WITH subquery AS (SELECT sum(number) FROM numbers(10)) SELECT (SELECT * FROM subquery); diff --git a/tests/queries/0_stateless/02475_or_function_alias_and_const_where.reference b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.reference new file mode 100644 index 00000000000..b5d8e605a7d --- /dev/null +++ b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.reference @@ -0,0 +1,2 @@ +0 0 +0 0 diff --git a/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql new file mode 100644 index 00000000000..ddb0f70c6de --- /dev/null +++ b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql @@ -0,0 +1,2 @@ +SELECT (number = 1) AND (number = 2) AS value, sum(value) OVER () FROM numbers(1) WHERE 1; +SELECT (number = 1) AND (number = 2) AS value, sum(value) OVER () FROM numbers(1) WHERE 1 SETTINGS allow_experimental_analyzer=1; \ No newline at end of file diff --git a/tests/queries/0_stateless/02476_fix_lambda_parsing.reference b/tests/queries/0_stateless/02476_fix_lambda_parsing.reference new file mode 100644 index 00000000000..18cb46ce23c --- /dev/null +++ b/tests/queries/0_stateless/02476_fix_lambda_parsing.reference @@ -0,0 +1,8 @@ +SELECT f(x, y -> z) +SELECT f(x, y -> z) +SELECT f((x, y) -> z) +SELECT f((x, y) -> z) +SELECT f((x, y) -> z) +SELECT f(x, (x, y) -> z) +SELECT f(x, (x, y) -> z) +CREATE FUNCTION func AS x -> plus(x, (x -> ('2' + 2)) -> plus(1), 1) diff --git a/tests/queries/0_stateless/02476_fix_lambda_parsing.sh b/tests/queries/0_stateless/02476_fix_lambda_parsing.sh new file mode 100755 index 00000000000..641ef59a170 --- /dev/null +++ b/tests/queries/0_stateless/02476_fix_lambda_parsing.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +format="$CLICKHOUSE_FORMAT" + +echo "SELECT f(x, tuple(y) -> z)" | $format +echo "SELECT f(x, (y) -> z)" | $format + +echo "SELECT f(x, y -> z)" | $format +echo "SELECT f((x, y) -> z)" | $format +echo "SELECT f(tuple(x, y) -> z)" | $format + +echo "SELECT f(x, (x, y) -> z)" | $format +echo "SELECT f(x, tuple(x, y) -> z)" | $format + +echo "CREATE FUNCTION func AS x -> plus(x, (x -> ('2' + 2)) -> plus(1), 1)" | $format | $format