Merge branch 'master' into adapting-parquet-block-size

This commit is contained in:
János Benjamin Antal 2024-06-10 14:43:49 +02:00 committed by GitHub
commit 69bacef5d4
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
123 changed files with 5218 additions and 862 deletions

View File

@ -19,3 +19,7 @@ charset = utf-8
indent_style = space
indent_size = 4
trim_trailing_whitespace = true
# Some SQL results have trailing whitespace which is removed by IDEs
[tests/queries/**.reference]
trim_trailing_whitespace = false

View File

@ -48,19 +48,17 @@ At a minimum, the following information should be added (but add more as needed)
- [ ] <!---ci_include_stateful--> Allow: Stateful tests
- [ ] <!---ci_include_integration--> Allow: Integration Tests
- [ ] <!---ci_include_performance--> Allow: Performance tests
- [ ] <!---ci_set_normal_builds--> Allow: Normal Builds
- [ ] <!---ci_set_special_builds--> Allow: Special Builds
- [ ] <!---ci_set_non_required--> Allow: All NOT Required Checks
- [ ] <!---batch_0_1--> Allow: batch 1, 2 for multi-batch jobs
- [ ] <!---batch_2_3--> Allow: batch 3, 4, 5, 6 for multi-batch jobs
---
- [ ] <!---ci_exclude_style--> Exclude: Style check
- [ ] <!---ci_exclude_fast--> Exclude: Fast test
- [ ] <!---ci_exclude_integration--> Exclude: Integration Tests
- [ ] <!---ci_exclude_stateless--> Exclude: Stateless tests
- [ ] <!---ci_exclude_stateful--> Exclude: Stateful tests
- [ ] <!---ci_exclude_performance--> Exclude: Performance tests
- [ ] <!---ci_exclude_asan--> Exclude: All with ASAN
- [ ] <!---ci_exclude_aarch64--> Exclude: All with Aarch64
- [ ] <!---ci_exclude_tsan|msan|ubsan|coverage--> Exclude: All with TSAN, MSAN, UBSAN, Coverage
- [ ] <!---ci_exclude_aarch64|release|debug--> Exclude: All with aarch64, release, debug
---
- [ ] <!---do_not_test--> Do not test
- [ ] <!---upload_all--> Upload binaries for special builds

2
contrib/cld2 vendored

@ -1 +1 @@
Subproject commit bc6d493a2f64ed1fc1c4c4b4294a542a04e04217
Subproject commit 217ba8b8805b41557faadaa47bb6e99f2242eea3

View File

@ -267,7 +267,7 @@ For introspection use `system.s3queue` stateless table and `system.s3queue_log`
`exception` String
)
ENGINE = SystemS3Queue
COMMENT 'SYSTEM TABLE is built on the fly.' │
COMMENT 'Contains in-memory state of S3Queue metadata and currently processed rows per file.' │
└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
```

View File

@ -6,35 +6,26 @@ sidebar_label: MergeTree
# MergeTree
The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most commonly used and most robust ClickHouse table engines.
The `MergeTree` engine and other engines of the `MergeTree` family (e.g. `ReplacingMergeTree`, `AggregatingMergeTree` ) are the most commonly used and most robust table engines in ClickHouse.
Engines in the `MergeTree` family are designed for inserting a very large amount of data into a table. The data is quickly written to the table part by part, then rules are applied for merging the parts in the background. This method is much more efficient than continually rewriting the data in storage during insert.
`MergeTree`-family table engines are designed for high data ingest rates and huge data volumes.
Insert operations create table parts which are merged by a background process with other table parts.
Main features:
Main features of `MergeTree`-family table engines.
- Stores data sorted by primary key.
- The table's primary key determines the sort order within each table part (clustered index). The primary key also does not reference individual rows but blocks of 8192 rows called granules. This makes primary keys of huge data sets small enough to remain loaded in main memory, while still providing fast access to on-disk data.
This allows you to create a small sparse index that helps find data faster.
- Tables can be partitioned using an arbitrary partition expression. Partition pruning ensures partitions are omitted from reading when the query allows it.
- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified.
- Data can be replicated across multiple cluster nodes for high availability, failover, and zero downtime upgrades. See [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md).
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.
- `MergeTree` table engines support various statistics kinds and sampling methods to help query optimization.
- Data replication support.
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](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family.
:::note
Despite a similar name, the [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine is different from `*MergeTree` engines.
:::
If you need to update rows frequently, we recommend using the [`ReplacingMergeTree`](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md) table engine. Using `ALTER TABLE my_table UPDATE` to update rows triggers a mutation, which causes parts to be re-written and uses IO/resources. With `ReplacingMergeTree`, you can simply insert the updated rows and the old rows will be replaced according to the table sorting key.
## Creating a Table {#table_engine-mergetree-creating-a-table}
## Creating Tables {#table_engine-mergetree-creating-a-table}
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
@ -59,23 +50,24 @@ ORDER BY expr
[SETTINGS name = value, ...]
```
For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md).
For a detailed description of the parameters, see the [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md) statement
### Query Clauses {#mergetree-query-clauses}
#### ENGINE
`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters.
`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine has no parameters.
#### ORDER_BY
`ORDER BY` — The sorting key.
A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`.
A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID + 1, EventDate)`.
ClickHouse uses the sorting key as a primary key if the primary key is not defined explicitly by the `PRIMARY KEY` clause.
If no primary key is defined (i.e. `PRIMARY KEY` was not specified), ClickHouse uses the the sorting key as primary key.
Use the `ORDER BY tuple()` syntax, if you do not need sorting, or set `create_table_empty_primary_key_by_default` to `true` to use the `ORDER BY tuple()` syntax by default. See [Selecting the Primary Key](#selecting-the-primary-key).
If no sorting is required, you can use syntax `ORDER BY tuple()`.
Alternatively, if setting `create_table_empty_primary_key_by_default` is enabled, `ORDER BY tuple()` is implicitly added to `CREATE TABLE` statements. See [Selecting a Primary Key](#selecting-a-primary-key).
#### PARTITION BY
@ -87,100 +79,32 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da
`PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional.
By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause.
Specifying a sorting key (using `ORDER BY` clause) implicitly specifies a primary key.
It is usually not necessary to specify the primary key in addition to the primary key.
#### SAMPLE BY
`SAMPLE BY` — An expression for sampling. Optional.
`SAMPLE BY` — A sampling expression. Optional.
If a sampling expression is used, the primary key must contain it. The result of a sampling expression must be an unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
If specified, it must be contained in the primary key.
The sampling expression must result in an unsigned integer.
Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
#### TTL
`TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional.
`TTL` — A list of rules that specify the storage duration of rows and the logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional.
Expression must have one `Date` or `DateTime` column as a result. Example:
```
TTL date + INTERVAL 1 DAY
```
Expression must result in a `Date` or `DateTime`, e.g. `TTL date + INTERVAL 1 DAY`.
Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an action to be done with the part if the expression is satisfied (reaches current time): removal of expired rows, moving a part (if expression is satisfied for all rows in a part) to specified disk (`TO DISK 'xxx'`) or to volume (`TO VOLUME 'xxx'`), or aggregating values in expired rows. Default type of the rule is removal (`DELETE`). List of multiple rules can be specified, but there should be no more than one `DELETE` rule.
For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl)
### SETTINGS
Additional parameters that control the behavior of the `MergeTree` (optional):
#### SETTINGS
#### index_granularity
`index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Data Storage](#mergetree-data-storage).
#### index_granularity_bytes
`index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Data Storage](#mergetree-data-storage).
#### min_index_granularity_bytes
`min_index_granularity_bytes` — Min allowed size of data granules in bytes. Default value: 1024b. To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. See [Data Storage](#mergetree-data-storage).
#### enable_mixed_granularity_parts
`enable_mixed_granularity_parts` — Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries.
#### 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](/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
`min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes.
#### merge_with_ttl_timeout
`merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours).
#### merge_with_recompression_ttl_timeout
`merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours).
#### try_fetch_recompressed_part_timeout
`try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours).
#### write_final_mark
`write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Dont turn it off.
#### merge_max_block_size
`merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192.
#### storage_policy
`storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes).
#### min_bytes_for_wide_part
`min_bytes_for_wide_part`, `min_rows_for_wide_part` — Minimum number of bytes/rows in a data part that can be stored in `Wide` format. You can set one, both or none of these settings. See [Data Storage](#mergetree-data-storage).
#### max_parts_in_total
`max_parts_in_total` — Maximum number of parts in all partitions.
#### 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](/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](/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](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting.
#### allow_experimental_optimized_row_order
`allow_experimental_optimized_row_order` - Experimental. Enables the optimization of the row order during inserts to improve the compressability of the data for compression codecs (e.g. LZ4). Analyzes and reorders the data, and thus increases the CPU overhead of inserts.
See [MergeTree Settings](../../../operations/settings/merge-tree-settings.md).
**Example of Sections Setting**
@ -270,7 +194,7 @@ ClickHouse does not require a unique primary key. You can insert multiple rows w
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}
### Selecting a Primary Key {#selecting-a-primary-key}
The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may:

View File

@ -0,0 +1,194 @@
---
slug: /en/operations/analyzer
sidebar_label: Analyzer
title: Analyzer
description: Details about ClickHouse's query analyzer
keywords: [analyzer]
---
# Analyzer
<BetaBadge />
## Known incompatibilities
In ClickHouse version `24.3`, the new query analyzer was enabled by default.
Despite fixing a large number of bugs and introducing new optimizations, it also introduces some breaking changes in ClickHouse behaviour. Please read the following changes to determine how to rewrite your queries for the new analyzer.
### Invalid queries are no longer optimized
The previous query planning infrastructure applied AST-level optimizations before the query validation step.
Optimizations could rewrite the initial query so it becomes valid and can be executed.
In the new analyzer, query validation takes place before the optimization step.
This means that invalid queries that were possible to execute before are now unsupported.
In such cases, the query must be fixed manually.
**Example 1:**
```sql
SELECT number
FROM numbers(1)
GROUP BY toString(number)
```
The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation.
In the old analyzer, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid.
**Example 2:**
```sql
SELECT
number % 2 AS n,
sum(number)
FROM numbers(10)
GROUP BY n
HAVING number > 5
```
The same problem occurs in this query: column `number` is used after aggregation with another key.
The previous query analyzer fixed this query by moving the `number > 5` filter from the `HAVING` clause to the `WHERE` clause.
To fix the query, you should move all conditions that apply to non-aggregated columns to the `WHERE` section to conform to standard SQL syntax:
```sql
SELECT
number % 2 AS n,
sum(number)
FROM numbers(10)
WHERE number > 5
GROUP BY n
```
### CREATE VIEW with invalid query
The new analyzer always performs type-checking.
Previously, it was possible to create a `VIEW` with an invalid `SELECT` query. It would then fail during the first `SELECT` or `INSERT` (in the case of `MATERIALIZED VIEW`).
Now, it's not possible to create such `VIEW`s anymore.
**Example:**
```sql
CREATE TABLE source (data String) ENGINE=MergeTree ORDER BY tuple();
CREATE VIEW some_view
AS SELECT JSONExtract(data, 'test', 'DateTime64(3)')
FROM source;
```
### Known incompatibilities of the `JOIN` clause
#### Join using column from projection
Alias from the `SELECT` list can not be used as a `JOIN USING` key by default.
A new setting, `analyzer_compatibility_join_using_top_level_identifier`, when enabled, alters the behavior of `JOIN USING` to prefer to resolve identifiers based on expressions from the projection list of the `SELECT` query, rather than using the columns from left table directly.
**Example:**
```sql
SELECT a + 1 AS b, t2.s
FROM Values('a UInt64, b UInt64', (1, 1)) AS t1
JOIN Values('b UInt64, s String', (1, 'one'), (2, 'two')) t2
USING (b);
```
With `analyzer_compatibility_join_using_top_level_identifier` set to `true`, the join condition is interpreted as `t1.a + 1 = t2.b`, matching the behavior of earlier versions. So, the result will be `2, 'two'`.
When the setting is `false`, the join condition defaults to `t1.b = t2.b`, and the query will return `2, 'one'`.
If `b` is not present in `t1`, the query will fail with an error.
#### Changes in behavior with `JOIN USING` and `ALIAS`/`MATERIALIZED` columns
In the new analyzer, using `*` in a `JOIN USING` query that involves `ALIAS` or `MATERIALIZED` columns will include those columns in the result set by default.
**Example:**
```sql
CREATE TABLE t1 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id;
INSERT INTO t1 VALUES (1), (2);
CREATE TABLE t2 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id;
INSERT INTO t2 VALUES (2), (3);
SELECT * FROM t1
FULL JOIN t2 USING (payload);
```
In the new analyzer, the result of this query will include the `payload` column along with `id` from both tables. In contrast, the previous analyzer would only include these `ALIAS` columns if specific settings (`asterisk_include_alias_columns` or `asterisk_include_materialized_columns`) were enabled, and the columns might appear in a different order.
To ensure consistent and expected results, especially when migrating old queries to the new analyzer, it is advisable to specify columns explicitly in the `SELECT` clause rather than using `*`.
#### Handling of Type Modifiers for columns in `USING` Clause
In the new version of the analyzer, the rules for determining the common supertype for columns specified in the `USING` clause have been standardized to produce more predictable outcomes, especially when dealing with type modifiers like `LowCardinality` and `Nullable`.
- `LowCardinality(T)` and `T`: When a column of type `LowCardinality(T)` is joined with a column of type `T`, the resulting common supertype will be `T`, effectively discarding the `LowCardinality` modifier.
- `Nullable(T)` and `T`: When a column of type `Nullable(T)` is joined with a column of type `T`, the resulting common supertype will be `Nullable(T)`, ensuring that the nullable property is preserved.
**Example:**
```sql
SELECT id, toTypeName(id) FROM Values('id LowCardinality(String)', ('a')) AS t1
FULL OUTER JOIN Values('id String', ('b')) AS t2
USING (id);
```
In this query, the common supertype for `id` is determined as `String`, discarding the `LowCardinality` modifier from `t1`.
### Projection column names changes
During projection names computation, aliases are not substituted.
```sql
SELECT
1 + 1 AS x,
x + 1
SETTINGS allow_experimental_analyzer = 0
FORMAT PrettyCompact
┌─x─┬─plus(plus(1, 1), 1)─┐
1. │ 2 │ 3 │
└───┴─────────────────────┘
SELECT
1 + 1 AS x,
x + 1
SETTINGS allow_experimental_analyzer = 1
FORMAT PrettyCompact
┌─x─┬─plus(x, 1)─┐
1. │ 2 │ 3 │
└───┴────────────┘
```
### Incompatible function arguments types
In the new analyzer, type inference happens during initial query analysis.
This change means that type checks are done before short-circuit evaluation; thus, `if` function arguments must always have a common supertype.
**Example:**
The following query fails with `There is no supertype for types Array(UInt8), String because some of them are Array and some of them are not`:
```sql
SELECT toTypeName(if(0, [2, 3, 4], 'String'))
```
### Heterogeneous clusters
The new analyzer significantly changed the communication protocol between servers in the cluster. Thus, it's impossible to run distributed queries on servers with different `allow_experimental_analyzer` setting values.
### Mutations are interpreted by previous analyzer
Mutations are still using the old analyzer.
This means some new ClickHouse SQL features can't be used in mutations. For example, the `QUALIFY` clause.
Status can be checked [here](https://github.com/ClickHouse/ClickHouse/issues/61563).
### Unsupported features
The list of features new analyzer currently doesn't support:
- Annoy index.
- Hypothesis index. Work in progress [here](https://github.com/ClickHouse/ClickHouse/pull/48381).
- Window view is not supported. There are no plans to support it in the future.

View File

@ -443,3 +443,59 @@ SELECT dictGet('dict', 'b', 1);
│ a │
└─────────────────────────┘
```
## Named collections for accessing Kafka
The description of parameters see [Kafka](../engines/table-engines/integrations/kafka.md).
### DDL example
```sql
CREATE NAMED COLLECTION my_kafka_cluster AS
kafka_broker_list = 'localhost:9092',
kafka_topic_list = 'kafka_topic',
kafka_group_name = 'consumer_group',
kafka_format = 'JSONEachRow',
kafka_max_block_size = '1048576';
```
### XML example
```xml
<clickhouse>
<named_collections>
<my_kafka_cluster>
<kafka_broker_list>localhost:9092</kafka_broker_list>
<kafka_topic_list>kafka_topic</kafka_topic_list>
<kafka_group_name>consumer_group</kafka_group_name>
<kafka_format>JSONEachRow</kafka_format>
<kafka_max_block_size>1048576</kafka_max_block_size>
</my_kafka_cluster>
</named_collections>
</clickhouse>
```
### Example of using named collections with a Kafka table
Both of the following examples use the same named collection `my_kafka_cluster`:
```sql
CREATE TABLE queue
(
timestamp UInt64,
level String,
message String
)
ENGINE = Kafka(my_kafka_cluster)
CREATE TABLE queue
(
timestamp UInt64,
level String,
message String
)
ENGINE = Kafka(my_kafka_cluster)
SETTINGS kafka_num_consumers = 4,
kafka_thread_per_consumer = 1;
```

View File

@ -3,9 +3,126 @@ slug: /en/operations/settings/merge-tree-settings
title: "MergeTree tables settings"
---
The values of `merge_tree` settings (for all MergeTree tables) can be viewed in the table `system.merge_tree_settings`, they can be overridden in `config.xml` in the `merge_tree` section, or set in the `SETTINGS` section of each table.
System table `system.merge_tree_settings` shows the globally set MergeTree settings.
These are example overrides for `max_suspicious_broken_parts`:
MergeTree settings can be set in the `merge_tree` section of the server config file, or specified for each `MergeTree` table individually in
the `SETTINGS` clause of the `CREATE TABLE` statement.
Example for customizing setting `max_suspicious_broken_parts`:
Configure the default for all `MergeTree` tables in the server configuration file:
``` text
<merge_tree>
<max_suspicious_broken_parts>5</max_suspicious_broken_parts>
</merge_tree>
```
Set for a particular table:
``` sql
CREATE TABLE tab
(
`A` Int64
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS max_suspicious_broken_parts = 500;
```
Change the settings for a particular table using `ALTER TABLE ... MODIFY SETTING`:
```sql
ALTER TABLE tab MODIFY SETTING max_suspicious_broken_parts = 100;
-- reset to global default (value from system.merge_tree_settings)
ALTER TABLE tab RESET SETTING max_suspicious_broken_parts;
```
## index_granularity
Maximum number of data rows between the marks of an index.
Default value: 8192.
## index_granularity_bytes
Maximum size of data granules in bytes.
Default value: 10Mb.
To restrict the granule size only by number of rows, set to 0 (not recommended).
## min_index_granularity_bytes
Min allowed size of data granules in bytes.
Default value: 1024b.
To provide a safeguard against accidentally creating tables with very low index_granularity_bytes.
## enable_mixed_granularity_parts
Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries.
## use_minimalistic_part_header_in_zookeeper
Storage method of the data parts headers in ZooKeeper. If enabled, ZooKeeper stores less data. For details, see [here](../server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper).
## min_merge_bytes_to_use_direct_io
The minimum data volume for merge operation that is required for using direct I/O access to the storage disk.
When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged.
If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option).
If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled.
Default value: `10 * 1024 * 1024 * 1024` bytes.
## merge_with_ttl_timeout
Minimum delay in seconds before repeating a merge with delete TTL.
Default value: `14400` seconds (4 hours).
## merge_with_recompression_ttl_timeout
Minimum delay in seconds before repeating a merge with recompression TTL.
Default value: `14400` seconds (4 hours).
## write_final_mark
Enables or disables writing the final index mark at the end of data part (after the last byte).
Default value: 1.
Dont change or bad things will happen.
## storage_policy
Storage policy.
## min_bytes_for_wide_part
Minimum number of bytes/rows in a data part that can be stored in `Wide` format.
You can set one, both or none of these settings.
## 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
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
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.
## max_suspicious_broken_parts
@ -17,37 +134,6 @@ Possible values:
Default value: 100.
Override example in `config.xml`:
``` text
<merge_tree>
<max_suspicious_broken_parts>5</max_suspicious_broken_parts>
</merge_tree>
```
An example to set in `SETTINGS` for a particular table:
``` sql
CREATE TABLE foo
(
`A` Int64
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS max_suspicious_broken_parts = 500;
```
An example of changing the settings for a specific table with the `ALTER TABLE ... MODIFY SETTING` command:
``` sql
ALTER TABLE foo
MODIFY SETTING max_suspicious_broken_parts = 100;
-- reset to default (use value from system.merge_tree_settings)
ALTER TABLE foo
RESET SETTING max_suspicious_broken_parts;
```
## parts_to_throw_insert {#parts-to-throw-insert}
If the number of active parts in a single partition exceeds the `parts_to_throw_insert` value, `INSERT` is interrupted with the `Too many parts (N). Merges are processing significantly slower than inserts` exception.
@ -301,6 +387,8 @@ Default value: 10800
## try_fetch_recompressed_part_timeout
Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression.
Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression.
Possible values:

View File

@ -1590,6 +1590,22 @@ Possible values:
Default value: `default`.
## parallel_replicas_custom_key_range_lower {#parallel_replicas_custom_key_range_lower}
Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`.
When used in conjuction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`.
Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing.
## parallel_replicas_custom_key_range_upper {#parallel_replicas_custom_key_range_upper}
Allows the filter type `range` to split the work evenly between replicas based on the custom range `[0, parallel_replicas_custom_key_range_upper]`. A value of 0 disables the upper bound, setting it the max value of the custom key expression.
When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`.
Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing.
## allow_experimental_parallel_reading_from_replicas
Enables or disables sending SELECT queries to all replicas of a table (up to `max_parallel_replicas`). Reading is parallelized and coordinated dynamically. It will work for any kind of MergeTree table.
@ -3172,7 +3188,7 @@ Default value: `0`.
## lightweight_deletes_sync {#lightweight_deletes_sync}
The same as 'mutation_sync', but controls only execution of lightweight deletes.
The same as 'mutation_sync', but controls only execution of lightweight deletes.
Possible values:
@ -3862,6 +3878,10 @@ Possible values:
Default value: 30.
:::note
It's applicable only to the default profile. A server reboot is required for the changes to take effect.
:::
## http_receive_timeout {#http_receive_timeout}
HTTP receive timeout (in seconds).
@ -4616,16 +4636,6 @@ Read more about [memory overcommit](memory-overcommit.md).
Default value: `1GiB`.
## max_untracked_memory {#max_untracked_memory}
Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.
Default value: `4MiB`.
## min_untracked_memory {#min_untracked_memory}
Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread memory usage divided by 16 and clamped between `min_untracked_memory` and `max_untracked_memory` for every thread. It guarantees that total untracked memory does not exceed 10% of current memory footprint even with a lot of small threads. To disable dynamic limit for untracked memory set value `4MiB`.
Default value: `4KiB`.
## Schema Inference settings
See [schema inference](../../interfaces/schema-inference.md#schema-inference-modes) documentation for more details.

View File

@ -142,6 +142,34 @@ SELECT readWKTPoint('POINT (1.2 3.4)');
(1.2,3.4)
```
## readWKTLineString
Parses a Well-Known Text (WKT) representation of a LineString geometry and returns it in the internal ClickHouse format.
### Syntax
```sql
readWKTLineString(wkt_string)
```
### Arguments
- `wkt_string`: The input WKT string representing a LineString geometry.
### Returned value
The function returns a ClickHouse internal representation of the linestring geometry.
### Example
```sql
SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)');
```
```response
[(1,1),(2,2),(3,3),(1,1)]
```
## readWKTRing
Parses a Well-Known Text (WKT) representation of a Polygon geometry and returns a ring (closed linestring) in the internal ClickHouse format.
@ -163,7 +191,7 @@ The function returns a ClickHouse internal representation of the ring (closed li
### Example
```sql
SELECT readWKTRing('LINESTRING (1 1, 2 2, 3 3, 1 1)');
SELECT readWKTRing('POLYGON ((1 1, 2 2, 3 3, 1 1))');
```
```response

View File

@ -212,7 +212,7 @@ toTypeName(x)
## blockSize {#blockSize}
In ClickHouse, queries are processed in blocks (chunks).
In ClickHouse, queries are processed in [blocks](../../development/architecture.md/#block-block) (chunks).
This function returns the size (row count) of the block the function is called on.
**Syntax**
@ -221,6 +221,33 @@ This function returns the size (row count) of the block the function is called o
blockSize()
```
**Example**
Query:
```sql
DROP TABLE IF EXISTS test;
CREATE TABLE test (n UInt8) ENGINE = Memory;
INSERT INTO test
SELECT * FROM system.numbers LIMIT 5;
SELECT blockSize()
FROM test;
```
Result:
```response
┌─blockSize()─┐
1. │ 5 │
2. │ 5 │
3. │ 5 │
4. │ 5 │
5. │ 5 │
└─────────────┘
```
## byteSize
Returns an estimation of uncompressed byte size of its arguments in memory.
@ -3688,3 +3715,108 @@ Result:
```response
{'version':'1','serial_number':'2D9071D64530052D48308473922C7ADAFA85D6C5','signature_algo':'sha256WithRSAEncryption','issuer':'/CN=marsnet.local CA','not_before':'May 7 17:01:21 2024 GMT','not_after':'May 7 17:01:21 2025 GMT','subject':'/CN=chnode1','pkey_algo':'rsaEncryption'}
```
## lowCardinalityIndices
Returns the position of a value in the dictionary of a [LowCardinality](../data-types/lowcardinality.md) column. Positions start at 1. Since LowCardinality have per-part dictionaries, this function may return different positions for the same value in different parts.
**Syntax**
```sql
lowCardinalityIndices(col)
```
**Arguments**
- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md).
**Returned value**
- The position of the value in the dictionary of the current part. [UInt64](../data-types/int-uint.md).
**Example**
Query:
```sql
DROP TABLE IF EXISTS test;
CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory;
-- create two parts:
INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df');
INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef');
SELECT s, lowCardinalityIndices(s) FROM test;
```
Result:
```response
┌─s──┬─lowCardinalityIndices(s)─┐
1. │ ab │ 1 │
2. │ cd │ 2 │
3. │ ab │ 1 │
4. │ ab │ 1 │
5. │ df │ 3 │
└────┴──────────────────────────┘
┌─s──┬─lowCardinalityIndices(s)─┐
6. │ ef │ 1 │
7. │ cd │ 2 │
8. │ ab │ 3 │
9. │ cd │ 2 │
10. │ ef │ 1 │
└────┴──────────────────────────┘
```
## lowCardinalityKeys
Returns the dictionary values of a [LowCardinality](../data-types/lowcardinality.md) column. If the block is smaller or larger than the dictionary size, the result will be truncated or extended with default values. Since LowCardinality have per-part dictionaries, this function may return different dictionary values in different parts.
**Syntax**
```sql
lowCardinalityIndices(col)
```
**Arguments**
- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md).
**Returned value**
- The dictionary keys. [UInt64](../data-types/int-uint.md).
**Example**
Query:
```sql
DROP TABLE IF EXISTS test;
CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory;
-- create two parts:
INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df');
INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef');
SELECT s, lowCardinalityKeys(s) FROM test;
```
Result:
```response
┌─s──┬─lowCardinalityKeys(s)─┐
1. │ ef │ │
2. │ cd │ ef │
3. │ ab │ cd │
4. │ cd │ ab │
5. │ ef │ │
└────┴───────────────────────┘
┌─s──┬─lowCardinalityKeys(s)─┐
6. │ ab │ │
7. │ cd │ ab │
8. │ ab │ cd │
9. │ ab │ df │
10. │ df │ │
└────┴───────────────────────┘
```

View File

@ -6,49 +6,90 @@ sidebar_label: Rounding
# Rounding Functions
## floor(x\[, N\])
## floor
Returns the largest round number that is less than or equal to `x`. A round number is a multiple of 1/10N, or the nearest number of the appropriate data type if 1 / 10N isnt exact.
N is an integer constant, optional parameter. By default it is zero, which means to round to an integer.
N may be negative.
Returns the largest rounded number less than or equal `x`.
A rounded number is a multiple of 1 / 10 * N, or the nearest number of the appropriate data type if 1 / 10 * N isnt exact.
Examples: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.`
Integer arguments may be rounded with negative `N` argument, with non-negative `N` the function returns `x`, i.e. does nothing.
`x` is any numeric type. The result is a number of the same type.
For integer arguments, it makes sense to round with a negative `N` value (for non-negative `N`, the function does not do anything).
If rounding causes overflow (for example, floor(-128, -1)), an implementation-specific result is returned.
If rounding causes an overflow (for example, `floor(-128, -1)`), the result is undefined.
## ceil(x\[, N\]), ceiling(x\[, N\])
**Syntax**
Returns the smallest round number that is greater than or equal to `x`. In every other way, it is the same as the `floor` function (see above).
``` sql
floor(x[, N])
```
## trunc(x\[, N\]), truncate(x\[, N\])
**Parameters**
Returns the round number with largest absolute value that has an absolute value less than or equal to `x`s. In every other way, it is the same as the floor function (see above).
- `x` - The value to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md).
- `N` . [(U)Int*](../data-types/int-uint.md). The default is zero, which means rounding to an integer. Can be negative.
**Returned value**
A rounded number of the same type as `x`.
**Examples**
Query:
```sql
SELECT floor(123.45, 1) AS rounded
```
Result:
```
┌─rounded─┐
│ 123.4 │
└─────────┘
```
Query:
```sql
SELECT floor(123.45, -1)
```
Result:
```
┌─rounded─┐
│ 120 │
└─────────┘
```
## ceiling
Like `floor` but returns the smallest rounded number greater than or equal `x`.
**Syntax**
``` sql
ceiling(x[, N])
```
Alias: `ceil`
## truncate
Like `floor` but returns the rounded number with largest absolute value that has an absolute value less than or equal to `x`s.
**Syntax**
```sql
trunc(input, precision)
truncate(x[, N])
```
Alias: `truncate`.
**Parameters**
- `input`: A numeric type ([Float](../data-types/float.md), [Decimal](../data-types/decimal.md) or [Integer](../data-types/int-uint.md)).
- `precision`: An [Integer](../data-types/int-uint.md) type.
**Returned value**
- A data type of `input`.
Alias: `trunc`.
**Example**
Query:
```sql
SELECT trunc(123.499, 1) as res;
SELECT truncate(123.499, 1) as res;
```
```response
@ -57,37 +98,40 @@ SELECT trunc(123.499, 1) as res;
└───────┘
```
## round(x\[, N\])
## round
Rounds a value to a specified number of decimal places.
The function returns the nearest number of the specified order. In case when given number has equal distance to surrounding numbers, the function uses bankers rounding for float number types and rounds away from zero for the other number types (Decimal).
The function returns the nearest number of the specified order.
If the input value has equal distance to two neighboring numbers, the function uses bankers rounding for [Float*](../data-types/float.md) inputs and rounds away from zero for the other number types ([Decimal*](../data-types/decimal.md).
**Syntax**
``` sql
round(expression [, decimal_places])
round(x[, N])
```
**Arguments**
- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types).
- `decimal-places` — An integer value.
- If `decimal-places > 0` then the function rounds the value to the right of the decimal point.
- If `decimal-places < 0` then the function rounds the value to the left of the decimal point.
- If `decimal-places = 0` then the function rounds the value to integer. In this case the argument can be omitted.
- `x` — A number to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md).
- `N` — The number of decimal places to round to. Integer. Defaults to `0`.
- If `N > 0`, the function rounds to the right of the decimal point.
- If `N < 0`, the function rounds to the left of the decimal point.
- If `N = 0`, the function rounds to the next integer.
**Returned value:**
The rounded number of the same type as the input number.
A rounded number of the same type as `x`.
**Examples**
Example of usage with Float:
Example with `Float` inputs:
``` sql
```sql
SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3;
```
``` text
```
┌───x─┬─round(divide(number, 2))─┐
│ 0 │ 0 │
│ 0.5 │ 0 │
@ -95,13 +139,13 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3;
└─────┴──────────────────────────┘
```
Example of usage with Decimal:
Example with `Decimal` inputs:
``` sql
```sql
SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3;
```
``` text
```
┌───x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐
│ 0 │ 0 │
│ 0.5 │ 1 │
@ -109,14 +153,14 @@ SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIM
└─────┴──────────────────────────────────────────────────┘
```
If you want to keep the trailing zeros, you need to enable `output_format_decimal_trailing_zeros`
To retain trailing zeros, enable setting `output_format_decimal_trailing_zeros`:
``` sql
```sql
SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3 settings output_format_decimal_trailing_zeros=1;
```
``` text
```
┌──────x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐
│ 0.0000 │ 0.0000 │
│ 0.5000 │ 1.0000 │
@ -151,9 +195,15 @@ round(3.65, 1) = 3.6
Rounds a number to a specified decimal position.
- If the rounding number is halfway between two numbers, the function uses bankers rounding. Banker's rounding is a method of rounding fractional numbers. When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. For example: 3.5 rounds up to 4, 2.5 rounds down to 2. It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`.
If the rounding number is halfway between two numbers, the function uses bankers rounding.
Banker's rounding is a method of rounding fractional numbers
When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position.
For example: 3.5 rounds up to 4, 2.5 rounds down to 2.
It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest).
The [round](#rounding_functions-round) function performs the same rounding for floating point numbers.
The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`.
- In other cases, the function rounds numbers to the nearest integer.
In other cases, the function rounds numbers to the nearest integer.
Using bankers rounding, you can reduce the effect that rounding numbers has on the results of summing or subtracting these numbers.
@ -166,16 +216,20 @@ For example, sum numbers 1.5, 2.5, 3.5, 4.5 with different rounding:
**Syntax**
``` sql
roundBankers(expression [, decimal_places])
roundBankers(x [, N])
```
**Arguments**
- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types).
- `decimal-places` — Decimal places. An integer number.
- `decimal-places > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`.
- `decimal-places < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`.
- `decimal-places = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`.
- `N > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`.
- `N < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`.
- `N = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`.
- `x` — A number to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md).
- `N` — The number of decimal places to round to. Integer. Defaults to `0`.
- If `N > 0`, the function rounds to the right of the decimal point.
- If `N < 0`, the function rounds to the left of the decimal point.
- If `N = 0`, the function rounds to the next integer.
**Returned value**
@ -185,13 +239,13 @@ A value rounded by the bankers rounding method.
Query:
``` sql
```sql
SELECT number / 2 AS x, roundBankers(x, 0) AS b fROM system.numbers limit 10
```
Result:
``` text
```
┌───x─┬─b─┐
│ 0 │ 0 │
│ 0.5 │ 0 │
@ -208,7 +262,7 @@ Result:
Examples of Bankers rounding:
``` text
```
roundBankers(0.4) = 0
roundBankers(-3.5) = -4
roundBankers(4.5) = 4
@ -264,7 +318,7 @@ Result:
## roundDuration
Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`.
Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`.
**Syntax**

View File

@ -10,7 +10,7 @@ sidebar_label: "Функции округления"
Возвращает наибольшее круглое число, которое меньше или равно, чем x.
Круглым называется число, кратное 1 / 10N или ближайшее к нему число соответствующего типа данных, если 1 / 10N не представимо точно.
N - целочисленная константа, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа.
N - целочисленный аргумент, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа.
N может быть отрицательным.
Примеры: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.`

View File

@ -773,7 +773,27 @@ try
LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info);
#endif
bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log");
bool has_trace_collector = false;
/// Disable it if we collect test coverage information, because it will work extremely slow.
#if !WITH_COVERAGE
/// Profilers cannot work reliably with any other libunwind or without PHDR cache.
has_trace_collector = hasPHDRCache() && config().has("trace_log");
#endif
/// Describe multiple reasons when query profiler cannot work.
#if WITH_COVERAGE
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage.");
#endif
#if defined(SANITIZER)
LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers"
" when two different stack unwinding methods will interfere with each other.");
#endif
if (!hasPHDRCache())
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
" (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
// Initialize global thread pool. Do it before we fetch configs from zookeeper
// nodes (`from_zk`), because ZooKeeper interface uses the pool. We will
@ -782,8 +802,27 @@ try
server_settings.max_thread_pool_size,
server_settings.max_thread_pool_free_size,
server_settings.thread_pool_queue_size,
will_have_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0,
will_have_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0);
has_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0,
has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0);
if (has_trace_collector)
{
global_context->createTraceCollector();
/// Set up server-wide memory profiler (for total memory tracker).
if (server_settings.total_memory_profiler_step)
total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step);
if (server_settings.total_memory_tracker_sample_probability > 0.0)
total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability);
if (server_settings.total_memory_profiler_sample_min_allocation_size)
total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size);
if (server_settings.total_memory_profiler_sample_max_allocation_size)
total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size);
}
/// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed).
SCOPE_EXIT({
Stopwatch watch;
@ -1950,52 +1989,9 @@ try
LOG_DEBUG(log, "Loaded metadata.");
/// Init trace collector only after trace_log system table was created
/// Disable it if we collect test coverage information, because it will work extremely slow.
#if !WITH_COVERAGE
/// Profilers cannot work reliably with any other libunwind or without PHDR cache.
if (hasPHDRCache())
{
if (has_trace_collector)
global_context->initializeTraceCollector();
/// Set up server-wide memory profiler (for total memory tracker).
if (server_settings.total_memory_profiler_step)
{
total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step);
}
if (server_settings.total_memory_tracker_sample_probability > 0.0)
{
total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability);
}
if (server_settings.total_memory_profiler_sample_min_allocation_size)
{
total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size);
}
if (server_settings.total_memory_profiler_sample_max_allocation_size)
{
total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size);
}
}
#endif
/// Describe multiple reasons when query profiler cannot work.
#if WITH_COVERAGE
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage.");
#endif
#if defined(SANITIZER)
LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers"
" when two different stack unwinding methods will interfere with each other.");
#endif
if (!hasPHDRCache())
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
" (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
#if defined(OS_LINUX)
auto tasks_stats_provider = TasksStatsCounters::findBestAvailableProvider();
if (tasks_stats_provider == TasksStatsCounters::MetricsProvider::None)

View File

@ -1561,6 +1561,7 @@
<rocksdb>
<options>
<max_background_jobs>8</max_background_jobs>
<info_log_level>DEBUG_LEVEL</info_log_level>
</options>
<column_family_options>
<num_levels>2</num_levels>

View File

@ -289,10 +289,14 @@ void executeColumnIfNeeded(ColumnWithTypeAndName & column, bool empty)
if (!column_function)
return;
size_t original_size = column.column->size();
if (!empty)
column = column_function->reduce();
else
column.column = column_function->getResultType()->createColumn();
column.column = column_function->getResultType()->createColumnConstWithDefaultValue(original_size)->convertToFullColumnIfConst();
chassert(column.column->size() == original_size);
}
int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments)

View File

@ -57,7 +57,6 @@ AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory
{
auto res = memory_tracker->allocImpl(will_be, throw_if_memory_exceeded);
current_thread->untracked_memory = 0;
current_thread->updateUntrackedMemoryLimit(memory_tracker->get());
return res;
}
else
@ -85,13 +84,6 @@ void CurrentMemoryTracker::check()
std::ignore = memory_tracker->allocImpl(0, true);
}
Int64 CurrentMemoryTracker::get()
{
if (auto * memory_tracker = getMemoryTracker())
return memory_tracker->get();
return 0;
}
AllocationTrace CurrentMemoryTracker::alloc(Int64 size)
{
bool throw_if_memory_exceeded = true;
@ -111,12 +103,10 @@ AllocationTrace CurrentMemoryTracker::free(Int64 size)
if (current_thread)
{
current_thread->untracked_memory -= size;
// Note that we use `max_untracked_memory` and not `untracked_memory_limit` to create hysteresis to avoid track/untrack cycles
if (current_thread->untracked_memory < -current_thread->max_untracked_memory)
if (current_thread->untracked_memory < -current_thread->untracked_memory_limit)
{
Int64 untracked_memory = current_thread->untracked_memory;
current_thread->untracked_memory = 0;
current_thread->updateUntrackedMemoryLimit(memory_tracker->get() + untracked_memory);
return memory_tracker->free(-untracked_memory);
}
}

View File

@ -12,9 +12,7 @@ struct CurrentMemoryTracker
/// This function should be called after memory deallocation.
[[nodiscard]] static AllocationTrace free(Int64 size);
static void check();
[[nodiscard]] static Int64 get();
/// Throws MEMORY_LIMIT_EXCEEDED (if it's allowed to throw exceptions)
static void injectFault();

View File

@ -228,9 +228,9 @@ void Timer::cleanup()
#endif
template <typename ProfilerImpl>
QueryProfilerBase<ProfilerImpl>::QueryProfilerBase([[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_)
: log(getLogger("QueryProfiler"))
, pause_signal(pause_signal_)
QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(
[[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_)
: log(getLogger("QueryProfiler")), pause_signal(pause_signal_)
{
#if defined(SANITIZER)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers");

View File

@ -183,12 +183,6 @@ public:
Int64 untracked_memory = 0;
/// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters.
Int64 untracked_memory_limit = 4 * 1024 * 1024;
/// To keep total untracked memory limited to `untracked_memory_ratio * RSS` we have to account threads with small and large memory footprint differently.
/// For this purpose we dynamically change `untracked_memory_limit` after every tracking event using a simple formula:
/// untracked_memory_limit = clamp(untracked_memory_ratio * cur_memory_bytes, min_untracked_memory, max_untracked_memory)
/// Note that this values are updated when thread is attached to a group
Int64 min_untracked_memory = 4 * 1024 * 1024; // Default value is kept 4MB mostly for tests and client (should be changed to 4KB as default value a setting)
Int64 max_untracked_memory = 4 * 1024 * 1024;
/// Statistics of read and write rows/bytes
Progress progress_in;
@ -315,12 +309,6 @@ public:
void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period);
void updateUntrackedMemoryLimit(Int64 current)
{
constexpr Int64 untracked_memory_ratio_bits = 4; // untracked_memory_ratio = 1.0 / (1 << untracked_memory_ratio_bits) = 1.0 / 16 = 6.25%
untracked_memory_limit = std::clamp<Int64>(current >> untracked_memory_ratio_bits, min_untracked_memory, max_untracked_memory);
}
private:
void applyGlobalSettings();
void applyQuerySettings();

View File

@ -202,6 +202,8 @@ class IColumn;
M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \
M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \
M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \
M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \
M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over. A value of 0 disables the upper bound, setting it to the max value of the custom key expression", 0) \
\
M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \
M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \
@ -491,7 +493,6 @@ class IColumn;
M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
M(UInt64, min_untracked_memory, (4 * 1024), "Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread_memory_usage/16 and clamped between min_untracked_memory and max_untracked_memory for every thread.", 0) \
M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \

View File

@ -98,12 +98,13 @@ static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges
{"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"},
{"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."},
{"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."},
{"min_untracked_memory", 4_MiB, 4_KiB, "A new setting to enable more accurate memory tracking."},
{"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"},
{"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."},
{"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."},
{"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."},
{"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}
{"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."},
{"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"},
{"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"},
}},
{"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"},
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},

View File

@ -17,6 +17,13 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory)
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypePointName>()));
});
// Custom type for simple line which consists from several segments.
factory.registerSimpleDataTypeCustom("LineString", []
{
return std::make_pair(DataTypeFactory::instance().get("Array(Point)"),
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeLineStringName>()));
});
// Custom type for simple polygon without holes stored as Array(Point)
factory.registerSimpleDataTypeCustom("Ring", []
{

View File

@ -11,6 +11,12 @@ public:
DataTypePointName() : DataTypeCustomFixedName("Point") {}
};
class DataTypeLineStringName : public DataTypeCustomFixedName
{
public:
DataTypeLineStringName() : DataTypeCustomFixedName("LineString") {}
};
class DataTypeRingName : public DataTypeCustomFixedName
{
public:

View File

@ -794,7 +794,7 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromStorage(const String & table_name, cons
throw_on_error);
create_table_query->set(create_table_query->as<ASTCreateQuery>()->comment,
std::make_shared<ASTLiteral>("SYSTEM TABLE is built on the fly."));
std::make_shared<ASTLiteral>(storage->getInMemoryMetadata().comment));
return create_table_query;
}

View File

@ -19,11 +19,15 @@ namespace ProfileEvents
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
struct WriteBufferFromAzureBlobStorage::PartData
{
Memory<> memory;
size_t data_size = 0;
std::string block_id;
};
BufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureObjectStorageSettings & settings)
@ -119,22 +123,30 @@ void WriteBufferFromAzureBlobStorage::preFinalize()
// This function should not be run again
is_prefinalized = true;
hidePartialData();
if (hidden_size > 0)
detachBuffer();
setFakeBufferWhenPreFinalized();
/// If there is only one block and size is less than or equal to max_single_part_upload_size
/// then we use single part upload instead of multi part upload
if (buffer_allocation_policy->getBufferNumber() == 1)
if (block_ids.empty() && detached_part_data.size() == 1 && detached_part_data.front().data_size <= max_single_part_upload_size)
{
size_t data_size = size_t(position() - memory.data());
if (data_size <= max_single_part_upload_size)
{
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast<const uint8_t *>(memory.data()), data_size);
execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, data_size);
LOG_TRACE(log, "Committed single block for blob `{}`", blob_path);
return;
}
}
auto part_data = std::move(detached_part_data.front());
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast<const uint8_t *>(part_data.memory.data()), part_data.data_size);
execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size);
LOG_TRACE(log, "Committed single block for blob `{}`", blob_path);
writePart();
detached_part_data.pop_front();
return;
}
else
{
writeMultipartUpload();
}
}
void WriteBufferFromAzureBlobStorage::finalizeImpl()
@ -144,9 +156,13 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl()
if (!is_prefinalized)
preFinalize();
chassert(offset() == 0);
chassert(hidden_size == 0);
task_tracker->waitAll();
if (!block_ids.empty())
{
task_tracker->waitAll();
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries);
LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path);
@ -155,14 +171,66 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl()
void WriteBufferFromAzureBlobStorage::nextImpl()
{
if (is_prefinalized)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot write to prefinalized buffer for Azure Blob Storage, the file could have been created");
task_tracker->waitIfAny();
writePart();
hidePartialData();
reallocateFirstBuffer();
if (available() > 0)
return;
detachBuffer();
if (detached_part_data.size() > 1)
writeMultipartUpload();
allocateBuffer();
}
void WriteBufferFromAzureBlobStorage::hidePartialData()
{
if (write_settings.remote_throttler)
write_settings.remote_throttler->add(offset(), ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds);
chassert(memory.size() >= hidden_size + offset());
hidden_size += offset();
chassert(memory.data() + hidden_size == working_buffer.begin() + offset());
chassert(memory.data() + hidden_size == position());
WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size);
chassert(offset() == 0);
}
void WriteBufferFromAzureBlobStorage::reallocateFirstBuffer()
{
chassert(offset() == 0);
if (buffer_allocation_policy->getBufferNumber() > 1 || available() > 0)
return;
const size_t max_first_buffer = buffer_allocation_policy->getBufferSize();
if (memory.size() == max_first_buffer)
return;
size_t size = std::min(memory.size() * 2, max_first_buffer);
memory.resize(size);
WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size);
chassert(offset() == 0);
}
void WriteBufferFromAzureBlobStorage::allocateBuffer()
{
buffer_allocation_policy->nextBuffer();
chassert(0 == hidden_size);
auto size = buffer_allocation_policy->getBufferSize();
if (buffer_allocation_policy->getBufferNumber() == 1)
@ -172,30 +240,56 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer()
WriteBuffer::set(memory.data(), memory.size());
}
void WriteBufferFromAzureBlobStorage::writePart()
void WriteBufferFromAzureBlobStorage::detachBuffer()
{
auto data_size = size_t(position() - memory.data());
size_t data_size = size_t(position() - memory.data());
if (data_size == 0)
return;
const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64));
std::shared_ptr<PartData> part_data = std::make_shared<PartData>(std::move(memory), data_size, block_id);
WriteBuffer::set(nullptr, 0);
chassert(data_size == hidden_size);
auto upload_worker = [this, part_data] ()
auto buf = std::move(memory);
WriteBuffer::set(nullptr, 0);
total_size += hidden_size;
hidden_size = 0;
detached_part_data.push_back({std::move(buf), data_size});
WriteBuffer::set(nullptr, 0);
}
void WriteBufferFromAzureBlobStorage::writePart(WriteBufferFromAzureBlobStorage::PartData && part_data)
{
const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64));
auto worker_data = std::make_shared<std::tuple<std::string, WriteBufferFromAzureBlobStorage::PartData>>(block_id, std::move(part_data));
auto upload_worker = [this, worker_data] ()
{
auto & data_size = std::get<1>(*worker_data).data_size;
auto & data_block_id = std::get<0>(*worker_data);
auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path);
Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast<const uint8_t *>(part_data->memory.data()), part_data->data_size);
execWithRetry([&](){ block_blob_client.StageBlock(part_data->block_id, memory_stream); }, max_unexpected_write_error_retries, part_data->data_size);
if (write_settings.remote_throttler)
write_settings.remote_throttler->add(part_data->data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds);
Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast<const uint8_t *>(std::get<1>(*worker_data).memory.data()), data_size);
execWithRetry([&](){ block_blob_client.StageBlock(data_block_id, memory_stream); }, max_unexpected_write_error_retries, data_size);
};
task_tracker->add(std::move(upload_worker));
}
void WriteBufferFromAzureBlobStorage::setFakeBufferWhenPreFinalized()
{
WriteBuffer::set(fake_buffer_when_prefinalized, sizeof(fake_buffer_when_prefinalized));
}
void WriteBufferFromAzureBlobStorage::writeMultipartUpload()
{
while (!detached_part_data.empty())
{
writePart(std::move(detached_part_data.front()));
detached_part_data.pop_front();
}
}
}
#endif

View File

@ -48,8 +48,13 @@ public:
private:
struct PartData;
void writePart();
void writeMultipartUpload();
void writePart(PartData && part_data);
void detachBuffer();
void reallocateFirstBuffer();
void allocateBuffer();
void hidePartialData();
void setFakeBufferWhenPreFinalized();
void finalizeImpl() override;
void execWithRetry(std::function<void()> func, size_t num_tries, size_t cost = 0);
@ -77,9 +82,16 @@ private:
MemoryBufferPtr allocateBuffer() const;
char fake_buffer_when_prefinalized[1] = {};
bool first_buffer=true;
size_t total_size = 0;
size_t hidden_size = 0;
std::unique_ptr<TaskTracker> task_tracker;
std::deque<PartData> detached_part_data;
};
}

View File

@ -166,6 +166,8 @@ public:
return client.get();
}
bool supportParallelWrite() const override { return true; }
private:
using SharedAzureClientPtr = std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient>;
void removeObjectImpl(const StoredObject & object, const SharedAzureClientPtr & client_ptr, bool if_exists);

View File

@ -314,7 +314,7 @@ void checkFunctionArgumentSizes(const ColumnsWithTypeAndName & arguments, size_t
if (current_size != input_rows_count)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Expected the argument nº#{} ('{}' of type {}) to have {} rows, but it has {}",
"Expected the argument {} ('{}' of type {}) to have {} rows, but it has {}",
i + 1,
arguments[i].name,
arguments[i].type->getName(),

View File

@ -31,7 +31,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_COLUMN;
@ -40,26 +39,22 @@ namespace ErrorCodes
}
/** Rounding Functions:
* round(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for floating point numbers.
* roundBankers(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for all numbers.
* floor(x, N) is the largest number <= x (N = 0 by default).
* ceil(x, N) is the smallest number >= x (N = 0 by default).
* trunc(x, N) - is the largest by absolute value number that is not greater than x by absolute value (N = 0 by default).
*
* The value of the parameter N (scale):
* - N > 0: round to the number with N decimal places after the decimal point
* - N < 0: round to an integer with N zero characters
* - N = 0: round to an integer
*
* Type of the result is the type of argument.
* For integer arguments, when passing negative scale, overflow can occur.
* In that case, the behavior is implementation specific.
*/
/// Rounding Functions:
/// - round(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for floating point numbers.
/// - roundBankers(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for all numbers.
/// - floor(x, N) is the largest number <= x (N = 0 by default).
/// - ceil(x, N) is the smallest number >= x (N = 0 by default).
/// - trunc(x, N) - is the largest by absolute value number that is not greater than x by absolute value (N = 0 by default).
/// The value of the parameter N (scale):
/// - N > 0: round to the number with N decimal places after the decimal point
/// - N < 0: round to an integer with N zero characters
/// - N = 0: round to an integer
/** This parameter controls the behavior of the rounding functions.
*/
/// Type of the result is the type of argument.
/// For integer arguments, when passing negative scale, overflow can occur. In that case, the behavior is undefined.
/// Controls the behavior of the rounding functions.
enum class ScaleMode : uint8_t
{
Positive, // round to a number with N decimal places after the decimal point
@ -75,7 +70,7 @@ enum class RoundingMode : uint8_t
Ceil = _MM_FROUND_TO_POS_INF | _MM_FROUND_NO_EXC,
Trunc = _MM_FROUND_TO_ZERO | _MM_FROUND_NO_EXC,
#else
Round = 8, /// Values are correspond to above just in case.
Round = 8, /// Values correspond to above values, just in case.
Floor = 9,
Ceil = 10,
Trunc = 11,
@ -84,16 +79,21 @@ enum class RoundingMode : uint8_t
enum class TieBreakingMode : uint8_t
{
Auto, // use banker's rounding for floating point numbers, round up otherwise
Bankers, // use banker's rounding
Auto, /// banker's rounding for floating point numbers, round up otherwise
Bankers, /// banker's rounding
};
enum class Vectorize : uint8_t
{
No,
Yes
};
/// For N, no more than the number of digits in the largest type.
using Scale = Int16;
/** Rounding functions for integer values.
*/
/// Rounding functions for integer values.
template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode, TieBreakingMode tie_breaking_mode>
struct IntegerRoundingComputation
{
@ -149,6 +149,8 @@ struct IntegerRoundingComputation
return x;
}
}
std::unreachable();
}
static ALWAYS_INLINE T compute(T x, T scale)
@ -161,9 +163,12 @@ struct IntegerRoundingComputation
case ScaleMode::Negative:
return computeImpl(x, scale);
}
std::unreachable();
}
static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) requires std::integral<T>
static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out)
requires std::integral<T>
{
if constexpr (sizeof(T) <= sizeof(scale) && scale_mode == ScaleMode::Negative)
{
@ -176,20 +181,23 @@ struct IntegerRoundingComputation
*out = compute(*in, static_cast<T>(scale));
}
static ALWAYS_INLINE void compute(const T * __restrict in, T scale, T * __restrict out) requires(!std::integral<T>)
static ALWAYS_INLINE void compute(const T * __restrict in, T scale, T * __restrict out)
requires(!std::integral<T>)
{
*out = compute(*in, scale);
}
};
template <typename T, Vectorize vectorize>
class FloatRoundingComputationBase;
#ifdef __SSE4_1__
template <typename T>
class BaseFloatRoundingComputation;
/// Vectorized implementation for x86.
template <>
class BaseFloatRoundingComputation<Float32>
class FloatRoundingComputationBase<Float32, Vectorize::Yes>
{
public:
using ScalarType = Float32;
@ -210,7 +218,7 @@ public:
};
template <>
class BaseFloatRoundingComputation<Float64>
class FloatRoundingComputationBase<Float64, Vectorize::Yes>
{
public:
using ScalarType = Float64;
@ -230,9 +238,9 @@ public:
}
};
#else
#endif
/// Implementation for ARM. Not vectorized.
/// Sequential implementation for ARM. Also used for scalar arguments.
inline float roundWithMode(float x, RoundingMode mode)
{
@ -243,6 +251,8 @@ inline float roundWithMode(float x, RoundingMode mode)
case RoundingMode::Ceil: return ceilf(x);
case RoundingMode::Trunc: return truncf(x);
}
std::unreachable();
}
inline double roundWithMode(double x, RoundingMode mode)
@ -254,10 +264,12 @@ inline double roundWithMode(double x, RoundingMode mode)
case RoundingMode::Ceil: return ceil(x);
case RoundingMode::Trunc: return trunc(x);
}
std::unreachable();
}
template <typename T>
class BaseFloatRoundingComputation
class FloatRoundingComputationBase<T, Vectorize::No>
{
public:
using ScalarType = T;
@ -277,15 +289,13 @@ public:
}
};
#endif
/** Implementation of low-level round-off functions for floating-point values.
*/
template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode>
class FloatRoundingComputation : public BaseFloatRoundingComputation<T>
template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode, Vectorize vectorize>
class FloatRoundingComputation : public FloatRoundingComputationBase<T, vectorize>
{
using Base = BaseFloatRoundingComputation<T>;
using Base = FloatRoundingComputationBase<T, vectorize>;
public:
static void compute(const T * __restrict in, const typename Base::VectorType & scale, T * __restrict out)
@ -317,15 +327,22 @@ struct FloatRoundingImpl
private:
static_assert(!is_decimal<T>);
using Op = FloatRoundingComputation<T, rounding_mode, scale_mode>;
using Data = std::array<T, Op::data_count>;
template <Vectorize vectorize =
#ifdef __SSE4_1__
Vectorize::Yes
#else
Vectorize::No
#endif
>
using Op = FloatRoundingComputation<T, rounding_mode, scale_mode, vectorize>;
using Data = std::array<T, Op<>::data_count>;
using ColumnType = ColumnVector<T>;
using Container = typename ColumnType::Container;
public:
static NO_INLINE void apply(const Container & in, size_t scale, Container & out)
{
auto mm_scale = Op::prepare(scale);
auto mm_scale = Op<>::prepare(scale);
const size_t data_count = std::tuple_size<Data>();
@ -337,7 +354,7 @@ public:
while (p_in < limit)
{
Op::compute(p_in, mm_scale, p_out);
Op<>::compute(p_in, mm_scale, p_out);
p_in += data_count;
p_out += data_count;
}
@ -350,10 +367,17 @@ public:
size_t tail_size_bytes = (end_in - p_in) * sizeof(*p_in);
memcpy(&tmp_src, p_in, tail_size_bytes);
Op::compute(reinterpret_cast<T *>(&tmp_src), mm_scale, reinterpret_cast<T *>(&tmp_dst));
Op<>::compute(reinterpret_cast<T *>(&tmp_src), mm_scale, reinterpret_cast<T *>(&tmp_dst));
memcpy(p_out, &tmp_dst, tail_size_bytes);
}
}
static void applyOne(T in, size_t scale, T& out)
{
using ScalarOp = Op<Vectorize::No>;
auto s = ScalarOp::prepare(scale);
ScalarOp::compute(&in, s, &out);
}
};
template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode, TieBreakingMode tie_breaking_mode>
@ -409,6 +433,11 @@ public:
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected 'scale' parameter passed to function");
}
}
static void applyOne(T in, size_t scale, T& out)
{
Op::compute(&in, scale, &out);
}
};
@ -444,11 +473,40 @@ public:
memcpy(out.data(), in.data(), in.size() * sizeof(T));
}
}
static void applyOne(NativeType in, UInt32 in_scale, NativeType& out, Scale scale_arg)
{
scale_arg = in_scale - scale_arg;
if (scale_arg > 0)
{
auto scale = intExp10OfSize<NativeType>(scale_arg);
Op::compute(&in, scale, &out);
}
else
{
memcpy(&out, &in, sizeof(T));
}
}
};
/// Select the appropriate processing algorithm depending on the scale.
inline void validateScale(Int64 scale64)
{
if (scale64 > std::numeric_limits<Scale>::max() || scale64 < std::numeric_limits<Scale>::min())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large");
}
/** Select the appropriate processing algorithm depending on the scale.
*/
inline Scale getScaleArg(const ColumnConst* scale_col)
{
const auto & scale_field = scale_col->getField();
Int64 scale64 = scale_field.get<Int64>();
validateScale(scale64);
return scale64;
}
/// Generic dispatcher
template <typename T, RoundingMode rounding_mode, TieBreakingMode tie_breaking_mode>
struct Dispatcher
{
@ -457,30 +515,65 @@ struct Dispatcher
FloatRoundingImpl<T, rounding_mode, scale_mode>,
IntegerRoundingImpl<T, rounding_mode, scale_mode, tie_breaking_mode>>;
static ColumnPtr apply(const IColumn * col_general, Scale scale_arg)
template <typename ScaleType>
static ColumnPtr apply(const IColumn * value_col, const IColumn * scale_col = nullptr)
{
const auto & col = checkAndGetColumn<ColumnVector<T>>(*col_general);
const auto & value_col_typed = checkAndGetColumn<ColumnVector<T>>(*value_col);
auto col_res = ColumnVector<T>::create();
typename ColumnVector<T>::Container & vec_res = col_res->getData();
vec_res.resize(col.getData().size());
vec_res.resize(value_col_typed.getData().size());
if (!vec_res.empty())
{
if (scale_arg == 0)
if (scale_col == nullptr || isColumnConst(*scale_col))
{
size_t scale = 1;
FunctionRoundingImpl<ScaleMode::Zero>::apply(col.getData(), scale, vec_res);
auto scale_arg = (scale_col == nullptr) ? 0 : getScaleArg(checkAndGetColumnConst<ColumnVector<ScaleType>>(scale_col));
if (scale_arg == 0)
{
size_t scale = 1;
FunctionRoundingImpl<ScaleMode::Zero>::apply(value_col_typed.getData(), scale, vec_res);
}
else if (scale_arg > 0)
{
size_t scale = intExp10(scale_arg);
FunctionRoundingImpl<ScaleMode::Positive>::apply(value_col_typed.getData(), scale, vec_res);
}
else
{
size_t scale = intExp10(-scale_arg);
FunctionRoundingImpl<ScaleMode::Negative>::apply(value_col_typed.getData(), scale, vec_res);
}
}
else if (scale_arg > 0)
/// Non-const scale argument:
else if (const auto * scale_col_typed = checkAndGetColumn<ColumnVector<ScaleType>>(scale_col))
{
size_t scale = intExp10(scale_arg);
FunctionRoundingImpl<ScaleMode::Positive>::apply(col.getData(), scale, vec_res);
}
else
{
size_t scale = intExp10(-scale_arg);
FunctionRoundingImpl<ScaleMode::Negative>::apply(col.getData(), scale, vec_res);
const auto & value_data = value_col_typed.getData();
const auto & scale_data = scale_col_typed->getData();
const size_t rows = value_data.size();
for (size_t i = 0; i < rows; ++i)
{
Int64 scale64 = scale_data[i];
validateScale(scale64);
Scale raw_scale = scale64;
if (raw_scale == 0)
{
size_t scale = 1;
FunctionRoundingImpl<ScaleMode::Zero>::applyOne(value_data[i], scale, vec_res[i]);
}
else if (raw_scale > 0)
{
size_t scale = intExp10(raw_scale);
FunctionRoundingImpl<ScaleMode::Positive>::applyOne(value_data[i], scale, vec_res[i]);
}
else
{
size_t scale = intExp10(-raw_scale);
FunctionRoundingImpl<ScaleMode::Negative>::applyOne(value_data[i], scale, vec_res[i]);
}
}
}
}
@ -488,28 +581,51 @@ struct Dispatcher
}
};
/// Dispatcher for Decimal inputs
template <is_decimal T, RoundingMode rounding_mode, TieBreakingMode tie_breaking_mode>
struct Dispatcher<T, rounding_mode, tie_breaking_mode>
{
public:
static ColumnPtr apply(const IColumn * col_general, Scale scale_arg)
template <typename ScaleType>
static ColumnPtr apply(const IColumn * value_col, const IColumn * scale_col = nullptr)
{
const auto & col = checkAndGetColumn<ColumnDecimal<T>>(*col_general);
const typename ColumnDecimal<T>::Container & vec_src = col.getData();
const auto & value_col_typed = checkAndGetColumn<ColumnDecimal<T>>(*value_col);
const typename ColumnDecimal<T>::Container & vec_src = value_col_typed.getData();
auto col_res = ColumnDecimal<T>::create(vec_src.size(), col.getScale());
auto col_res = ColumnDecimal<T>::create(vec_src.size(), value_col_typed.getScale());
auto & vec_res = col_res->getData();
if (!vec_res.empty())
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col.getData(), col.getScale(), vec_res, scale_arg);
{
if (scale_col == nullptr || isColumnConst(*scale_col))
{
auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst<ColumnVector<ScaleType>>(scale_col));
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(value_col_typed.getData(), value_col_typed.getScale(), vec_res, scale_arg);
}
/// Non-const scale argument
else if (const auto * scale_col_typed = checkAndGetColumn<ColumnVector<ScaleType>>(scale_col))
{
const auto & scale = scale_col_typed->getData();
const size_t rows = vec_src.size();
for (size_t i = 0; i < rows; ++i)
{
Int64 scale64 = scale[i];
validateScale(scale64);
Scale raw_scale = scale64;
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::applyOne(value_col_typed.getElement(i), value_col_typed.getScale(),
reinterpret_cast<ColumnDecimal<T>::NativeT&>(col_res->getElement(i)), raw_scale);
}
}
}
return col_res;
}
};
/** A template for functions that round the value of an input parameter of type
* (U)Int8/16/32/64, Float32/64 or Decimal32/64/128, and accept an additional optional parameter (default is 0).
*/
/// Functions that round the value of an input parameter of type (U)Int8/16/32/64, Float32/64 or Decimal32/64/128.
/// Accept an additional optional parameter of type (U)Int8/16/32/64 (0 by default).
template <typename Name, RoundingMode rounding_mode, TieBreakingMode tie_breaking_mode>
class FunctionRounding : public IFunction
{
@ -517,75 +633,58 @@ public:
static constexpr auto name = Name::name;
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionRounding>(); }
String getName() const override
{
return name;
}
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if ((arguments.empty()) || (arguments.size() > 2))
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 1 or 2.",
getName(), arguments.size());
for (const auto & type : arguments)
if (!isNumber(type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}",
arguments[0]->getName(), getName());
return arguments[0];
}
static Scale getScaleArg(const ColumnsWithTypeAndName & arguments)
{
if (arguments.size() == 2)
{
const IColumn & scale_column = *arguments[1].column;
if (!isColumnConst(scale_column))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must be constant");
Field scale_field = assert_cast<const ColumnConst &>(scale_column).getField();
if (scale_field.getType() != Field::Types::UInt64
&& scale_field.getType() != Field::Types::Int64)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type");
Int64 scale64 = scale_field.get<Int64>();
if (scale64 > std::numeric_limits<Scale>::max()
|| scale64 < std::numeric_limits<Scale>::min())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large");
return scale64;
}
return 0;
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors mandatory_args{
{"x", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNumber), nullptr, "A number to round"},
};
FunctionArgumentDescriptors optional_args{
{"N", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeInteger), nullptr, "The number of decimal places to round to"},
};
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
return arguments[0].type;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
{
const ColumnWithTypeAndName & column = arguments[0];
Scale scale_arg = getScaleArg(arguments);
const ColumnWithTypeAndName & value_arg = arguments[0];
ColumnPtr res;
auto call = [&](const auto & types) -> bool
auto call_data = [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using DataType = typename Types::LeftType;
using DataType = typename Types::RightType;
if constexpr (IsDataTypeNumber<DataType> || IsDataTypeDecimal<DataType>)
if (arguments.size() > 1)
{
using FieldType = typename DataType::FieldType;
res = Dispatcher<FieldType, rounding_mode, tie_breaking_mode>::apply(column.column.get(), scale_arg);
const ColumnWithTypeAndName & scale_column = arguments[1];
auto call_scale = [&](const auto & scaleTypes) -> bool
{
using ScaleTypes = std::decay_t<decltype(scaleTypes)>;
using ScaleType = typename ScaleTypes::RightType;
if (isColumnConst(*value_arg.column) && !isColumnConst(*scale_column.column))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale column must be const for const data column");
res = Dispatcher<DataType, rounding_mode, tie_breaking_mode>::template apply<ScaleType>(value_arg.column.get(), scale_column.column.get());
return true;
};
TypeIndex right_index = scale_column.type->getTypeId();
if (!callOnBasicType<void, true, false, false, false>(right_index, call_scale))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type");
return true;
}
return false;
res = Dispatcher<DataType, rounding_mode, tie_breaking_mode>::template apply<int>(value_arg.column.get());
return true;
};
#if !defined(__SSE4_1__)
@ -597,10 +696,9 @@ public:
throw Exception(ErrorCodes::CANNOT_SET_ROUNDING_MODE, "Cannot set floating point rounding mode");
#endif
if (!callOnIndexAndDataType<void>(column.type->getTypeId(), call))
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", column.name, getName());
}
TypeIndex left_index = value_arg.type->getTypeId();
if (!callOnBasicType<void, true, true, true, false>(left_index, call_data))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", value_arg.name, getName());
return res;
}
@ -617,9 +715,8 @@ public:
};
/** Rounds down to a number within explicitly specified array.
* If the value is less than the minimal bound - returns the minimal bound.
*/
/// Rounds down to a number within explicitly specified array.
/// If the value is less than the minimal bound - returns the minimal bound.
class FunctionRoundDown : public IFunction
{
public:
@ -627,7 +724,6 @@ public:
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionRoundDown>(); }
String getName() const override { return name; }
bool isVariadic() const override { return false; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -28,6 +28,9 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
template <typename Point>
using LineString = boost::geometry::model::linestring<Point>;
template <typename Point>
using Ring = boost::geometry::model::ring<Point>;
@ -38,11 +41,13 @@ template <typename Point>
using MultiPolygon = boost::geometry::model::multi_polygon<Polygon<Point>>;
using CartesianPoint = boost::geometry::model::d2::point_xy<Float64>;
using CartesianLineString = LineString<CartesianPoint>;
using CartesianRing = Ring<CartesianPoint>;
using CartesianPolygon = Polygon<CartesianPoint>;
using CartesianMultiPolygon = MultiPolygon<CartesianPoint>;
using SphericalPoint = boost::geometry::model::point<Float64, 2, boost::geometry::cs::spherical_equatorial<boost::geometry::degree>>;
using SphericalLineString = LineString<SphericalPoint>;
using SphericalRing = Ring<SphericalPoint>;
using SphericalPolygon = Polygon<SphericalPoint>;
using SphericalMultiPolygon = MultiPolygon<SphericalPoint>;
@ -85,6 +90,29 @@ struct ColumnToPointsConverter
}
};
/**
* Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost linestring type.
*/
template <typename Point>
struct ColumnToLineStringsConverter
{
static std::vector<LineString<Point>> convert(ColumnPtr col)
{
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
size_t prev_offset = 0;
std::vector<LineString<Point>> answer;
answer.reserve(offsets.size());
auto tmp = ColumnToPointsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
for (size_t offset : offsets)
{
answer.emplace_back(tmp.begin() + prev_offset, tmp.begin() + offset);
prev_offset = offset;
}
return answer;
}
};
/**
* Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost ring type.
*/
@ -208,6 +236,39 @@ private:
ColumnFloat64::Container & second_container;
};
/// Serialize Point, LineString as LineString
template <typename Point>
class LineStringSerializer
{
public:
LineStringSerializer()
: offsets(ColumnUInt64::create())
{}
explicit LineStringSerializer(size_t n)
: offsets(ColumnUInt64::create(n))
{}
void add(const LineString<Point> & ring)
{
size += ring.size();
offsets->insertValue(size);
for (const auto & point : ring)
point_serializer.add(point);
}
ColumnPtr finalize()
{
return ColumnArray::create(point_serializer.finalize(), std::move(offsets));
}
private:
size_t size = 0;
PointSerializer<Point> point_serializer;
ColumnUInt64::MutablePtr offsets;
};
/// Almost the same as LineStringSerializer
/// Serialize Point, Ring as Ring
template <typename Point>
class RingSerializer
@ -344,8 +405,16 @@ static void callOnGeometryDataType(DataTypePtr type, F && f)
/// There is no Point type, because for most of geometry functions it is useless.
if (factory.get("Point")->equals(*type))
return f(ConverterType<ColumnToPointsConverter<Point>>());
/// We should take the name into consideration to avoid ambiguity.
/// Because for example both Ring and LineString are resolved to Array(Tuple(Point)).
else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "LineString")
return f(ConverterType<ColumnToLineStringsConverter<Point>>());
/// For backward compatibility if we call this function not on a custom type, we will consider Array(Tuple(Point)) as type Ring.
else if (factory.get("Ring")->equals(*type))
return f(ConverterType<ColumnToRingsConverter<Point>>());
else if (factory.get("Polygon")->equals(*type))
return f(ConverterType<ColumnToPolygonsConverter<Point>>());
else if (factory.get("MultiPolygon")->equals(*type))

View File

@ -73,6 +73,8 @@ public:
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName());
else if constexpr (std::is_same_v<ColumnToLineStringsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToLineStringsConverter<Point>, RightConverter>)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName());
else
{
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());

View File

@ -71,6 +71,8 @@ public:
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName());
else if constexpr (std::is_same_v<ColumnToLineStringsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToLineStringsConverter<Point>, RightConverter>)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName());
else
{
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());

View File

@ -71,6 +71,8 @@ public:
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName());
else if constexpr (std::is_same_v<ColumnToLineStringsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToLineStringsConverter<Point>, RightConverter>)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName());
else
{
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());

View File

@ -75,6 +75,8 @@ public:
if constexpr (std::is_same_v<ColumnToPointsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToPointsConverter<Point>, RightConverter>)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName());
else if constexpr (std::is_same_v<ColumnToLineStringsConverter<Point>, LeftConverter> || std::is_same_v<ColumnToLineStringsConverter<Point>, RightConverter>)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName());
else
{
auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst());

View File

@ -82,6 +82,11 @@ struct ReadWKTPointNameHolder
static constexpr const char * name = "readWKTPoint";
};
struct ReadWKTLineStringNameHolder
{
static constexpr const char * name = "readWKTLineString";
};
struct ReadWKTRingNameHolder
{
static constexpr const char * name = "readWKTRing";
@ -102,6 +107,30 @@ struct ReadWKTMultiPolygonNameHolder
REGISTER_FUNCTION(ReadWKT)
{
factory.registerFunction<FunctionReadWKT<DataTypePointName, CartesianPoint, PointSerializer<CartesianPoint>, ReadWKTPointNameHolder>>();
factory.registerFunction<FunctionReadWKT<DataTypeLineStringName, CartesianLineString, LineStringSerializer<CartesianPoint>, ReadWKTLineStringNameHolder>>(FunctionDocumentation
{
.description=R"(
Parses a Well-Known Text (WKT) representation of a LineString geometry and returns it in the internal ClickHouse format.
)",
.syntax = "readWKTLineString(wkt_string)",
.arguments{
{"wkt_string", "The input WKT string representing a LineString geometry."}
},
.returned_value = "The function returns a ClickHouse internal representation of the linestring geometry.",
.examples{
{"first call", "SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)');", R"(
readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')
[(1,1),(2,2),(3,3),(1,1)]
)"},
{"second call", "SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'));", R"(
toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'))
LineString
)"},
},
.categories{"Unique identifiers"}
});
factory.registerFunction<FunctionReadWKT<DataTypeRingName, CartesianRing, RingSerializer<CartesianPoint>, ReadWKTRingNameHolder>>();
factory.registerFunction<FunctionReadWKT<DataTypePolygonName, CartesianPolygon, PolygonSerializer<CartesianPoint>, ReadWKTPolygonNameHolder>>();
factory.registerFunction<FunctionReadWKT<DataTypeMultiPolygonName, CartesianMultiPolygon, MultiPolygonSerializer<CartesianPoint>, ReadWKTMultiPolygonNameHolder>>();

View File

@ -41,6 +41,14 @@ public:
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
/*
* Functions like recursiveRemoveLowCardinality don't pay enough attention to custom types and just erase
* the information about it during type conversions.
* While it is a big problem the quick solution would be just to disable default low cardinality implementation
* because it doesn't make a lot of sense for geo types.
*/
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
{
auto res_column = ColumnString::create();

View File

@ -596,6 +596,34 @@ void ActionsDAG::removeUnusedActions(const std::unordered_set<const Node *> & us
std::erase_if(inputs, [&](const Node * node) { return !visited_nodes.contains(node); });
}
void ActionsDAG::removeAliasesForFilter(const std::string & filter_name)
{
const auto & filter_node = findInOutputs(filter_name);
std::stack<Node *> stack;
stack.push(const_cast<Node *>(&filter_node));
std::unordered_set<const Node *> visited;
visited.insert(stack.top());
while (!stack.empty())
{
auto * node = stack.top();
stack.pop();
for (auto & child : node->children)
{
while (child->type == ActionType::ALIAS)
child = child->children.front();
if (!visited.contains(child))
{
stack.push(const_cast<Node *>(child));
visited.insert(child);
}
}
}
}
ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases)
{
auto actions = std::make_shared<ActionsDAG>();
@ -1704,7 +1732,7 @@ void ActionsDAG::mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs
}
}
ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split_nodes, bool create_split_nodes_mapping) const
ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split_nodes, bool create_split_nodes_mapping, bool avoid_duplicate_inputs) const
{
/// Split DAG into two parts.
/// (first_nodes, first_outputs) is a part which will have split_list in result.
@ -1718,6 +1746,14 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
/// List of nodes from current actions which are not inputs, but will be in second part.
NodeRawConstPtrs new_inputs;
/// Avoid new inputs to have the same name as existing inputs.
/// It's allowed for DAG but may break Block invariant 'columns with identical name must have identical structure'.
std::unordered_set<std::string_view> duplicate_inputs;
size_t duplicate_counter = 0;
if (avoid_duplicate_inputs)
for (const auto * input : inputs)
duplicate_inputs.insert(input->result_name);
struct Frame
{
const Node * node = nullptr;
@ -1830,7 +1866,8 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
input_node.result_name = child->result_name;
child_data.to_second = &second_nodes.emplace_back(std::move(input_node));
new_inputs.push_back(child);
if (child->type != ActionType::INPUT)
new_inputs.push_back(child);
}
}
@ -1886,7 +1923,32 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
for (const auto * input : new_inputs)
{
const auto & cur = data[input];
auto & cur = data[input];
if (avoid_duplicate_inputs)
{
bool is_name_updated = false;
while (!duplicate_inputs.insert(cur.to_first->result_name).second)
{
is_name_updated = true;
cur.to_first->result_name = fmt::format("{}_{}", input->result_name, duplicate_counter);
++duplicate_counter;
}
if (is_name_updated)
{
Node input_node;
input_node.type = ActionType::INPUT;
input_node.result_type = cur.to_first->result_type;
input_node.result_name = cur.to_first->result_name;
auto * new_input = &second_nodes.emplace_back(std::move(input_node));
cur.to_second->type = ActionType::ALIAS;
cur.to_second->children = {new_input};
cur.to_second = new_input;
}
}
second_inputs.push_back(cur.to_second);
first_outputs.push_back(cur.to_first);
}

View File

@ -195,6 +195,8 @@ public:
/// Remove actions that are not needed to compute output nodes with required names
void removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs = true, bool allow_constant_folding = true);
void removeAliasesForFilter(const std::string & filter_name);
/// Transform the current DAG in a way that leaf nodes get folded into their parents. It's done
/// because each projection can provide some columns as inputs to substitute certain sub-DAGs
/// (expressions). Consider the following example:
@ -343,7 +345,7 @@ public:
/// initial DAG : (a, b, c, d, e) -> (w, x, y, z) | 1 a 2 b 3 c 4 d 5 e 6 -> 1 2 3 4 5 6 w x y z
/// split (first) : (a, c, d) -> (i, j, k, w, y) | 1 a 2 b 3 c 4 d 5 e 6 -> 1 2 b 3 4 5 e 6 i j k w y
/// split (second) : (i, j, k, y, b, e) -> (x, y, z) | 1 2 b 3 4 5 e 6 i j k w y -> 1 2 3 4 5 6 w x y z
SplitResult split(std::unordered_set<const Node *> split_nodes, bool create_split_nodes_mapping = false) const;
SplitResult split(std::unordered_set<const Node *> split_nodes, bool create_split_nodes_mapping = false, bool avoid_duplicate_inputs = false) const;
/// Splits actions into two parts. Returned first half may be swapped with ARRAY JOIN.
SplitResult splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const;

View File

@ -740,12 +740,18 @@ struct ContextSharedPart : boost::noncopyable
void initializeTraceCollector(std::shared_ptr<TraceLog> trace_log)
{
if (!trace_log)
return;
if (!trace_collector.has_value())
throw Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector needs to be first created before initialization");
trace_collector->initialize(trace_log);
}
void createTraceCollector()
{
if (hasTraceCollector())
return;
trace_collector.emplace(std::move(trace_log));
trace_collector.emplace();
}
void addWarningMessage(const String & message) TSA_REQUIRES(mutex)
@ -3891,6 +3897,11 @@ void Context::initializeSystemLogs()
});
}
void Context::createTraceCollector()
{
shared->createTraceCollector();
}
void Context::initializeTraceCollector()
{
shared->initializeTraceCollector(getTraceLog());

View File

@ -1077,6 +1077,8 @@ public:
void initializeSystemLogs();
/// Call after initialization before using trace collector.
void createTraceCollector();
void initializeTraceCollector();
/// Call after unexpected crash happen.

View File

@ -279,6 +279,8 @@ Chain InterpreterInsertQuery::buildChain(
std::atomic_uint64_t * elapsed_counter_ms,
bool check_access)
{
IInterpreter::checkStorageSupportsTransactionsIfNeeded(table, getContext());
ProfileEvents::increment(ProfileEvents::InsertQueriesWithSubqueries);
ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries);

View File

@ -578,7 +578,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
settings.parallel_replicas_count,
settings.parallel_replica_offset,
std::move(custom_key_ast),
settings.parallel_replicas_custom_key_filter_type,
{settings.parallel_replicas_custom_key_filter_type,
settings.parallel_replicas_custom_key_range_lower,
settings.parallel_replicas_custom_key_range_upper},
storage->getInMemoryMetadataPtr()->columns,
context);
}

View File

@ -11,7 +11,6 @@
#include <Parsers/formatAST.h>
#include <Parsers/queryNormalization.h>
#include <Common/CurrentThread.h>
#include <Common/CurrentMemoryTracker.h>
#include <Common/Exception.h>
#include <Common/ProfileEvents.h>
#include <Common/QueryProfiler.h>
@ -211,12 +210,9 @@ void ThreadStatus::applyQuerySettings()
query_id_from_query_context = query_context_ptr->getCurrentQueryId();
initQueryProfiler();
max_untracked_memory = settings.max_untracked_memory;
if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast<UInt64>(max_untracked_memory))
max_untracked_memory = settings.memory_profiler_step;
min_untracked_memory = std::min<Int64>(settings.min_untracked_memory, max_untracked_memory);
updateUntrackedMemoryLimit(CurrentMemoryTracker::get());
untracked_memory_limit = settings.max_untracked_memory;
if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast<UInt64>(untracked_memory_limit))
untracked_memory_limit = settings.memory_profiler_step;
#if defined(OS_LINUX)
/// Set "nice" value if required.

View File

@ -1,5 +1,4 @@
#include "TraceCollector.h"
#include <Interpreters/TraceCollector.h>
#include <Core/Field.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
@ -14,8 +13,12 @@
namespace DB
{
TraceCollector::TraceCollector(std::shared_ptr<TraceLog> trace_log_)
: trace_log(std::move(trace_log_))
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
TraceCollector::TraceCollector()
{
TraceSender::pipe.open();
@ -28,6 +31,23 @@ TraceCollector::TraceCollector(std::shared_ptr<TraceLog> trace_log_)
thread = ThreadFromGlobalPool(&TraceCollector::run, this);
}
void TraceCollector::initialize(std::shared_ptr<TraceLog> trace_log_)
{
if (is_trace_log_initialized)
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector is already initialized");
trace_log_ptr = trace_log_;
is_trace_log_initialized.store(true, std::memory_order_release);
}
std::shared_ptr<TraceLog> TraceCollector::getTraceLog()
{
if (!is_trace_log_initialized.load(std::memory_order_acquire))
return nullptr;
return trace_log_ptr;
}
void TraceCollector::tryClosePipe()
{
try
@ -120,7 +140,7 @@ void TraceCollector::run()
ProfileEvents::Count increment;
readPODBinary(increment, in);
if (trace_log)
if (auto trace_log = getTraceLog())
{
// time and time_in_microseconds are both being constructed from the same timespec so that the
// times will be equal up to the precision of a second.

View File

@ -1,4 +1,5 @@
#pragma once
#include <atomic>
#include <Common/ThreadPool.h>
class StackTrace;
@ -16,11 +17,17 @@ class TraceLog;
class TraceCollector
{
public:
explicit TraceCollector(std::shared_ptr<TraceLog> trace_log_);
TraceCollector();
~TraceCollector();
void initialize(std::shared_ptr<TraceLog> trace_log_);
private:
std::shared_ptr<TraceLog> trace_log;
std::shared_ptr<TraceLog> getTraceLog();
std::atomic<bool> is_trace_log_initialized = false;
std::shared_ptr<TraceLog> trace_log_ptr;
ThreadFromGlobalPool thread;
void tryClosePipe();

View File

@ -7,7 +7,6 @@
#include <Interpreters/Context.h>
#include <DataTypes/DataTypesNumber.h>
#include <boost/rational.hpp>
@ -18,18 +17,19 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
extern const int INVALID_SETTING_VALUE;
}
ASTPtr getCustomKeyFilterForParallelReplica(
size_t replicas_count,
size_t replica_num,
ASTPtr custom_key_ast,
ParallelReplicasCustomKeyFilterType filter_type,
ParallelReplicasCustomKeyFilter filter,
const ColumnsDescription & columns,
const ContextPtr & context)
{
chassert(replicas_count > 1);
if (filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT)
if (filter.filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT)
{
// first we do modulo with replica count
auto modulo_function = makeASTFunction("positiveModulo", custom_key_ast, std::make_shared<ASTLiteral>(replicas_count));
@ -40,35 +40,80 @@ ASTPtr getCustomKeyFilterForParallelReplica(
return equals_function;
}
assert(filter_type == ParallelReplicasCustomKeyFilterType::RANGE);
chassert(filter.filter_type == ParallelReplicasCustomKeyFilterType::RANGE);
KeyDescription custom_key_description
= KeyDescription::getKeyFromAST(custom_key_ast, columns, context);
using RelativeSize = boost::rational<ASTSampleRatio::BigNum>;
RelativeSize size_of_universum = 0;
RelativeSize range_upper = RelativeSize(0);
RelativeSize range_lower = RelativeSize(filter.range_lower);
DataTypePtr custom_key_column_type = custom_key_description.data_types[0];
size_of_universum = RelativeSize(std::numeric_limits<UInt32>::max()) + RelativeSize(1);
if (custom_key_description.data_types.size() == 1)
{
if (typeid_cast<const DataTypeUInt64 *>(custom_key_column_type.get()))
size_of_universum = RelativeSize(std::numeric_limits<UInt64>::max()) + RelativeSize(1);
{
range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1)
: RelativeSize(std::numeric_limits<UInt64>::max()) + RelativeSize(1);
if (range_upper > RelativeSize(std::numeric_limits<UInt64>::max()) + RelativeSize(1))
throw Exception(
ErrorCodes::INVALID_SETTING_VALUE,
"Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt64) max value",
range_upper);
}
else if (typeid_cast<const DataTypeUInt32 *>(custom_key_column_type.get()))
size_of_universum = RelativeSize(std::numeric_limits<UInt32>::max()) + RelativeSize(1);
{
range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1)
: RelativeSize(std::numeric_limits<UInt32>::max()) + RelativeSize(1);
if (range_upper > RelativeSize(std::numeric_limits<UInt32>::max()) + RelativeSize(1))
throw Exception(
ErrorCodes::INVALID_SETTING_VALUE,
"Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt32) max value",
range_upper);
}
else if (typeid_cast<const DataTypeUInt16 *>(custom_key_column_type.get()))
size_of_universum = RelativeSize(std::numeric_limits<UInt16>::max()) + RelativeSize(1);
{
range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1)
: RelativeSize(std::numeric_limits<UInt16>::max()) + RelativeSize(1);
if (range_upper > RelativeSize(std::numeric_limits<UInt16>::max()) + RelativeSize(1))
throw Exception(
ErrorCodes::INVALID_SETTING_VALUE,
"Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt16) max value",
range_upper);
}
else if (typeid_cast<const DataTypeUInt8 *>(custom_key_column_type.get()))
size_of_universum = RelativeSize(std::numeric_limits<UInt8>::max()) + RelativeSize(1);
{
range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1)
: RelativeSize(std::numeric_limits<UInt8>::max()) + RelativeSize(1);
if (range_upper > RelativeSize(std::numeric_limits<UInt8>::max()) + RelativeSize(1))
throw Exception(
ErrorCodes::INVALID_SETTING_VALUE,
"Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt8) max value",
range_upper);
}
}
if (size_of_universum == RelativeSize(0))
if (range_upper == RelativeSize(0))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER,
"Invalid custom key column type: {}. Must be one unsigned integer type",
custom_key_column_type->getName());
if (range_lower >= range_upper)
throw Exception(
ErrorCodes::INVALID_SETTING_VALUE,
"Invalid custom key filter range: Range upper bound {} must be larger than range lower bound {}",
range_lower,
range_upper);
RelativeSize size_of_universum = range_upper - range_lower;
if (size_of_universum <= RelativeSize(replicas_count))
throw Exception(
ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range must be larger than than the number of replicas");
RelativeSize relative_range_size = RelativeSize(1) / replicas_count;
RelativeSize relative_range_offset = relative_range_size * RelativeSize(replica_num);
@ -76,19 +121,19 @@ ASTPtr getCustomKeyFilterForParallelReplica(
bool has_lower_limit = false;
bool has_upper_limit = false;
RelativeSize lower_limit_rational = relative_range_offset * size_of_universum;
RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum;
RelativeSize lower_limit_rational = range_lower + relative_range_offset * size_of_universum;
RelativeSize upper_limit_rational = range_lower + (relative_range_offset + relative_range_size) * size_of_universum;
UInt64 lower = boost::rational_cast<ASTSampleRatio::BigNum>(lower_limit_rational);
UInt64 upper = boost::rational_cast<ASTSampleRatio::BigNum>(upper_limit_rational);
if (lower > 0)
if (lower_limit_rational > range_lower)
has_lower_limit = true;
if (upper_limit_rational < size_of_universum)
if (upper_limit_rational < range_upper)
has_upper_limit = true;
assert(has_lower_limit || has_upper_limit);
chassert(has_lower_limit || has_upper_limit);
/// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed.
std::shared_ptr<ASTFunction> lower_function;
@ -110,7 +155,7 @@ ASTPtr getCustomKeyFilterForParallelReplica(
return upper_function;
}
assert(upper_function && lower_function);
chassert(upper_function && lower_function);
return makeASTFunction("and", std::move(lower_function), std::move(upper_function));
}

View File

@ -6,16 +6,24 @@
#include <Storages/IStorage.h>
#include <Core/SettingsEnums.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
struct ParallelReplicasCustomKeyFilter
{
ParallelReplicasCustomKeyFilterType filter_type;
UInt64 range_lower;
UInt64 range_upper;
};
/// Get AST for filter created from custom_key
/// replica_num is the number of the replica for which we are generating filter starting from 0
ASTPtr getCustomKeyFilterForParallelReplica(
size_t replicas_count,
size_t replica_num,
ASTPtr custom_key_ast,
ParallelReplicasCustomKeyFilterType filter_type,
ParallelReplicasCustomKeyFilter filter,
const ColumnsDescription & columns,
const ContextPtr & context);

View File

@ -498,12 +498,14 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage,
LOG_TRACE(getLogger("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value);
auto parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica(
settings.parallel_replicas_count,
settings.parallel_replica_offset,
std::move(custom_key_ast),
settings.parallel_replicas_custom_key_filter_type,
storage->getInMemoryMetadataPtr()->columns,
query_context);
settings.parallel_replicas_count,
settings.parallel_replica_offset,
std::move(custom_key_ast),
{settings.parallel_replicas_custom_key_filter_type,
settings.parallel_replicas_custom_key_range_lower,
settings.parallel_replicas_custom_key_range_upper},
storage->getInMemoryMetadataPtr()->columns,
query_context);
return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context);
}

View File

@ -50,6 +50,8 @@ FilterStep::FilterStep(
, filter_column_name(std::move(filter_column_name_))
, remove_filter_column(remove_filter_column_)
{
actions_dag = actions_dag->clone();
actions_dag->removeAliasesForFilter(filter_column_name);
}
void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)

View File

@ -118,7 +118,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
outputs.resize(size);
}
auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true);
auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true, true);
/// This is the leak of abstraction.
/// Splited actions may have inputs which are needed only for PREWHERE.

View File

@ -25,6 +25,7 @@
#include <Poco/Logger.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/Logger.h>
#include <Common/logger_useful.h>
#include <Common/Exception.h>
#include <Storages/AlterCommands.h>
@ -42,6 +43,7 @@
#include <cstddef>
#include <filesystem>
#include <memory>
#include <utility>
@ -185,11 +187,11 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_,
bool read_only_)
: IStorage(table_id_)
, WithContext(context_->getGlobalContext())
, log(getLogger(fmt::format("StorageEmbeddedRocksDB ({})", getStorageID().getNameForLogs())))
, primary_key{primary_key_}
, rocksdb_dir(std::move(rocksdb_dir_))
, ttl(ttl_)
, read_only(read_only_)
, log(getLogger(fmt::format("StorageEmbeddedRocksDB ({})", getStorageID().getNameForLogs())))
{
setInMemoryMetadata(metadata_);
setSettings(std::move(settings_));
@ -352,6 +354,72 @@ bool StorageEmbeddedRocksDB::optimize(
return true;
}
static_assert(rocksdb::DEBUG_LEVEL == 0);
static_assert(rocksdb::HEADER_LEVEL == 5);
static constexpr std::array<std::pair<DB::LogsLevel, Poco::Message::Priority>, 6> rocksdb_logger_map = {
std::make_pair(DB::LogsLevel::debug, Poco::Message::Priority::PRIO_DEBUG),
std::make_pair(DB::LogsLevel::information, Poco::Message::Priority::PRIO_INFORMATION),
std::make_pair(DB::LogsLevel::warning, Poco::Message::Priority::PRIO_WARNING),
std::make_pair(DB::LogsLevel::error, Poco::Message::Priority::PRIO_ERROR),
std::make_pair(DB::LogsLevel::fatal, Poco::Message::Priority::PRIO_FATAL),
/// Same as default logger does for HEADER_LEVEL
std::make_pair(DB::LogsLevel::information, Poco::Message::Priority::PRIO_INFORMATION),
};
class StorageEmbeddedRocksDBLogger : public rocksdb::Logger
{
public:
explicit StorageEmbeddedRocksDBLogger(const rocksdb::InfoLogLevel log_level, LoggerRawPtr log_)
: rocksdb::Logger(log_level)
, log(log_)
{}
void Logv(const char * format, va_list ap) override
__attribute__((format(printf, 2, 0)))
{
Logv(rocksdb::InfoLogLevel::DEBUG_LEVEL, format, ap);
}
void Logv(const rocksdb::InfoLogLevel log_level, const char * format, va_list ap) override
__attribute__((format(printf, 3, 0)))
{
if (log_level < GetInfoLogLevel())
return;
auto level = rocksdb_logger_map[log_level];
/// stack buffer was enough
{
va_list backup_ap;
va_copy(backup_ap, ap);
std::array<char, 1024> stack;
if (vsnprintf(stack.data(), stack.size(), format, backup_ap) < static_cast<int>(stack.size()))
{
va_end(backup_ap);
LOG_IMPL(log, level.first, level.second, "{}", stack.data());
return;
}
va_end(backup_ap);
}
/// let's try with a bigger dynamic buffer (but not too huge, since
/// some of rocksdb internal code has also such a limitation, i..e
/// HdfsLogger)
{
va_list backup_ap;
va_copy(backup_ap, ap);
static constexpr int buffer_size = 30000;
std::unique_ptr<char[]> buffer(new char[buffer_size]);
if (vsnprintf(buffer.get(), buffer_size, format, backup_ap) >= buffer_size)
buffer[buffer_size - 1] = 0;
va_end(backup_ap);
LOG_IMPL(log, level.first, level.second, "{}", buffer.get());
}
}
private:
LoggerRawPtr log;
};
void StorageEmbeddedRocksDB::initDB()
{
rocksdb::Status status;
@ -448,6 +516,7 @@ void StorageEmbeddedRocksDB::initDB()
}
}
merged.info_log = std::make_shared<StorageEmbeddedRocksDBLogger>(merged.info_log_level, log.get());
merged.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options));
if (ttl > 0)

View File

@ -114,17 +114,19 @@ public:
private:
SinkToStoragePtr getSink(ContextPtr context, const StorageMetadataPtr & metadata_snapshot);
LoggerPtr log;
MultiVersion<RocksDBSettings> storage_settings;
const String primary_key;
using RocksDBPtr = std::unique_ptr<rocksdb::DB>;
RocksDBPtr rocksdb_ptr;
mutable SharedMutex rocksdb_ptr_mx;
String rocksdb_dir;
Int32 ttl;
bool read_only;
void initDB();
LoggerPtr log;
};
}

View File

@ -904,11 +904,13 @@ void StorageDistributed::read(
[my_custom_key_ast = std::move(custom_key_ast),
column_description = this->getInMemoryMetadataPtr()->columns,
custom_key_type = settings.parallel_replicas_custom_key_filter_type.value,
custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value,
custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value,
context = local_context,
replica_count = modified_query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr
{
return getCustomKeyFilterForParallelReplica(
replica_count, replica_num - 1, my_custom_key_ast, custom_key_type, column_description, context);
replica_count, replica_num - 1, my_custom_key_ast, {custom_key_type, custom_key_range_lower, custom_key_range_upper}, column_description, context);
};
}
}

View File

@ -50,6 +50,12 @@ namespace ErrorCodes
namespace
{
struct GenerateRandomState
{
std::atomic<UInt64> add_total_rows = 0;
};
using GenerateRandomStatePtr = std::shared_ptr<GenerateRandomState>;
void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_of_type, pcg64 & rng, [[maybe_unused]] bool flip_bytes = false)
{
size_t size = limit * size_of_type;
@ -532,10 +538,24 @@ ColumnPtr fillColumnWithRandomData(
class GenerateSource : public ISource
{
public:
GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_)
GenerateSource(
UInt64 block_size_,
UInt64 max_array_length_,
UInt64 max_string_length_,
UInt64 random_seed_,
Block block_header_,
ContextPtr context_,
GenerateRandomStatePtr state_)
: ISource(Nested::flattenNested(prepareBlockToFill(block_header_)))
, block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_)
, block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {}
, block_size(block_size_)
, max_array_length(max_array_length_)
, max_string_length(max_string_length_)
, block_to_fill(std::move(block_header_))
, rng(random_seed_)
, context(context_)
, shared_state(state_)
{
}
String getName() const override { return "GenerateRandom"; }
@ -549,7 +569,15 @@ protected:
columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context));
columns = Nested::flattenNested(block_to_fill.cloneWithColumns(columns)).getColumns();
return {std::move(columns), block_size};
UInt64 total_rows = shared_state->add_total_rows.fetch_and(0);
if (total_rows)
addTotalRowsApprox(total_rows);
auto chunk = Chunk{std::move(columns), block_size};
progress(chunk.getNumRows(), chunk.bytes());
return chunk;
}
private:
@ -561,6 +589,7 @@ private:
pcg64 rng;
ContextPtr context;
GenerateRandomStatePtr shared_state;
static Block & prepareBlockToFill(Block & block)
{
@ -648,9 +677,6 @@ Pipe StorageGenerateRandom::read(
{
storage_snapshot->check(column_names);
Pipes pipes;
pipes.reserve(num_streams);
const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns();
Block block_header;
for (const auto & name : column_names)
@ -679,16 +705,24 @@ Pipe StorageGenerateRandom::read(
}
}
UInt64 query_limit = query_info.limit;
if (query_limit && num_streams * max_block_size > query_limit)
{
/// We want to avoid spawning more streams than necessary
num_streams = std::min(num_streams, static_cast<size_t>(((query_limit + max_block_size - 1) / max_block_size)));
}
Pipes pipes;
pipes.reserve(num_streams);
/// Will create more seed values for each source from initial seed.
pcg64 generate(random_seed);
auto shared_state = std::make_shared<GenerateRandomState>(query_info.limit);
for (UInt64 i = 0; i < num_streams; ++i)
{
auto source = std::make_shared<GenerateSource>(max_block_size, max_array_length, max_string_length, generate(), block_header, context);
if (i == 0 && query_info.limit)
source->addTotalRowsApprox(query_info.limit);
auto source = std::make_shared<GenerateSource>(
max_block_size, max_array_length, max_string_length, generate(), block_header, context, shared_state);
pipes.emplace_back(std::move(source));
}

View File

@ -16,7 +16,9 @@ namespace
struct ZerosState
{
explicit ZerosState(UInt64 limit) : add_total_rows(limit) { }
std::atomic<UInt64> num_generated_rows = 0;
std::atomic<UInt64> add_total_rows = 0;
};
using ZerosStatePtr = std::shared_ptr<ZerosState>;
@ -42,10 +44,13 @@ protected:
auto column_ptr = column;
size_t column_size = column_ptr->size();
if (state)
UInt64 total_rows = state->add_total_rows.fetch_and(0);
if (total_rows)
addTotalRowsApprox(total_rows);
if (limit)
{
auto generated_rows = state->num_generated_rows.fetch_add(column_size, std::memory_order_acquire);
if (generated_rows >= limit)
return {};
@ -103,36 +108,25 @@ Pipe StorageSystemZeros::read(
{
storage_snapshot->check(column_names);
bool use_multiple_streams = multithreaded;
UInt64 query_limit = limit ? *limit : 0;
if (query_info.limit)
query_limit = query_limit ? std::min(query_limit, query_info.limit) : query_info.limit;
if (limit && *limit < max_block_size)
{
max_block_size = static_cast<size_t>(*limit);
use_multiple_streams = false;
}
if (query_limit && query_limit < max_block_size)
max_block_size = query_limit;
if (!use_multiple_streams)
if (!multithreaded)
num_streams = 1;
else if (query_limit && num_streams * max_block_size > query_limit)
/// We want to avoid spawning more streams than necessary
num_streams = std::min(num_streams, static_cast<size_t>(((query_limit + max_block_size - 1) / max_block_size)));
ZerosStatePtr state = std::make_shared<ZerosState>(query_limit);
Pipe res;
ZerosStatePtr state;
if (limit)
state = std::make_shared<ZerosState>();
for (size_t i = 0; i < num_streams; ++i)
{
auto source = std::make_shared<ZerosSource>(max_block_size, limit ? *limit : 0, state);
if (i == 0)
{
if (limit)
source->addTotalRowsApprox(*limit);
else if (query_info.limit)
source->addTotalRowsApprox(query_info.limit);
}
auto source = std::make_shared<ZerosSource>(max_block_size, query_limit, state);
res.addSource(std::move(source));
}

View File

@ -166,8 +166,14 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType
{
ExpressionAndSets result;
auto ttl_string = queryToString(ast);
auto syntax_analyzer_result = TreeRewriter(context).analyze(ast, columns);
ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context);
auto context_copy = Context::createCopy(context);
/// FIXME All code here will work with old analyzer, however for TTL
/// with subqueries it's possible that new analyzer will be enabled in ::read method
/// of underlying storage when all other parts of infra are not ready for it
/// (built with old analyzer).
context_copy->setSetting("allow_experimental_analyzer", Field{0});
auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns);
ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy);
auto dag = analyzer.getActionsDAG(false);
const auto * col = &dag->findInOutputs(ast->getColumnName());
@ -177,7 +183,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType
dag->getOutputs() = {col};
dag->removeUnusedActions();
result.expression = std::make_shared<ExpressionActions>(dag, ExpressionActionsSettings::fromContext(context));
result.expression = std::make_shared<ExpressionActions>(dag, ExpressionActionsSettings::fromContext(context_copy));
result.sets = analyzer.getPreparedSets();
return result;

View File

@ -127,13 +127,11 @@ close it.
to_pop.append(i)
elif pr.head.ref.startswith(f"backport/{self.name}"):
self.backport_pr = pr
self._backported = True
to_pop.append(i)
else:
assert False, f"BUG! Invalid PR's branch [{pr.head.ref}]"
# Cherry-pick or backport PR found, set @backported flag for current release branch
self._backported = True
for i in reversed(to_pop):
# Going from the tail to keep the order and pop greater index first
prs.pop(i)
@ -218,6 +216,7 @@ close it.
self.name,
self.pr.number,
)
self._backported = True
return
except CalledProcessError:
# There are most probably conflicts, they'll be resolved in PR
@ -247,7 +246,6 @@ close it.
self.cherrypick_pr.add_to_labels(Labels.PR_CRITICAL_BUGFIX)
elif Labels.PR_BUGFIX in [label.name for label in self.pr.labels]:
self.cherrypick_pr.add_to_labels(Labels.PR_BUGFIX)
self._backported = True
self._assign_new_pr(self.cherrypick_pr)
# update cherrypick PR to get the state for PR.mergable
self.cherrypick_pr.update()
@ -359,10 +357,10 @@ class Backport:
self._fetch_from = fetch_from
self.dry_run = dry_run
self.must_create_backport_label = (
Labels.MUST_BACKPORT
self.must_create_backport_labels = (
[Labels.MUST_BACKPORT]
if self._repo_name == self._fetch_from
else Labels.MUST_BACKPORT_CLOUD
else [Labels.MUST_BACKPORT_CLOUD, Labels.MUST_BACKPORT]
)
self.backport_created_label = (
Labels.PR_BACKPORTS_CREATED
@ -468,7 +466,7 @@ class Backport:
query_args = {
"query": f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}",
"label": ",".join(
self.labels_to_backport + [self.must_create_backport_label]
self.labels_to_backport + self.must_create_backport_labels
),
"merged": [since_date, tomorrow],
}
@ -492,7 +490,10 @@ class Backport:
def process_pr(self, pr: PullRequest) -> None:
pr_labels = [label.name for label in pr.labels]
if self.must_create_backport_label in pr_labels:
if (
any(label in pr_labels for label in self.must_create_backport_labels)
or self._repo_name != self._fetch_from
):
branches = [
ReleaseBranch(br, pr, self.repo, self.backport_created_label)
for br in self.release_branches
@ -532,8 +533,8 @@ class Backport:
br.process(self.dry_run)
for br in branches:
assert br.backported, f"BUG! backport to branch [{br}] failed"
self.mark_pr_backported(pr)
if br.backported:
self.mark_pr_backported(pr)
def mark_pr_backported(self, pr: PullRequest) -> None:
if self.dry_run:

View File

@ -424,6 +424,7 @@ def _configure_jobs(
s3: S3Helper,
pr_info: PRInfo,
ci_settings: CiSettings,
skip_jobs: bool,
) -> CiCache:
"""
returns CICache instance with configured job's data
@ -434,11 +435,14 @@ def _configure_jobs(
"""
# get all jobs
job_configs = CI_CONFIG.get_workflow_jobs_with_configs(
is_mq=pr_info.is_merge_queue,
is_docs_only=pr_info.has_changes_in_documentation_only(),
is_master=pr_info.is_master,
)
if not skip_jobs:
job_configs = CI_CONFIG.get_workflow_jobs_with_configs(
is_mq=pr_info.is_merge_queue,
is_docs_only=pr_info.has_changes_in_documentation_only(),
is_master=pr_info.is_master,
)
else:
job_configs = {}
# filter jobs in accordance with ci settings
job_configs = ci_settings.apply(
@ -451,7 +455,9 @@ def _configure_jobs(
# check jobs in ci cache
ci_cache = CiCache.calc_digests_and_create(
s3, job_configs, cache_enabled=not ci_settings.no_ci_cache and CI
s3,
job_configs,
cache_enabled=not ci_settings.no_ci_cache and not skip_jobs and CI,
)
ci_cache.update()
ci_cache.apply(job_configs, is_release=pr_info.is_release)
@ -975,6 +981,7 @@ def main() -> int:
s3,
pr_info,
ci_settings,
args.skip_jobs,
)
ci_cache.print_status()
@ -993,15 +1000,15 @@ def main() -> int:
result["ci_settings"] = ci_settings.as_dict()
if not args.skip_jobs:
result["stages_data"] = _generate_ci_stage_config(ci_cache.jobs_to_do)
result["jobs_data"] = {
"jobs_to_do": list(ci_cache.jobs_to_do),
"jobs_to_skip": ci_cache.jobs_to_skip,
"digests": ci_cache.job_digests,
"jobs_params": {
job: {"batches": config.batches, "num_batches": config.num_batches}
for job, config in ci_cache.jobs_to_do.items()
},
}
result["jobs_data"] = {
"jobs_to_do": list(ci_cache.jobs_to_do),
"jobs_to_skip": ci_cache.jobs_to_skip,
"digests": ci_cache.job_digests,
"jobs_params": {
job: {"batches": config.batches, "num_batches": config.num_batches}
for job, config in ci_cache.jobs_to_do.items()
},
}
result["docker_data"] = docker_data
### CONFIGURE action: end

View File

@ -53,6 +53,8 @@ class CILabels(metaclass=WithIter):
CI_SET_SYNC = "ci_set_sync"
CI_SET_ARM = "ci_set_arm"
CI_SET_REQUIRED = "ci_set_required"
CI_SET_NORMAL_BUILDS = "ci_set_normal_builds"
CI_SET_SPECIAL_BUILDS = "ci_set_special_builds"
CI_SET_NON_REQUIRED = "ci_set_non_required"
CI_SET_OLD_ANALYZER = "ci_set_old_analyzer"
@ -685,10 +687,8 @@ class CIConfig:
def get_job_parents(self, check_name: str) -> List[str]:
res = []
check_name = normalize_string(check_name)
for config in (
self.build_config,
self.builds_report_config,
self.test_configs,
self.other_jobs_configs,
):
@ -899,6 +899,40 @@ CI_CONFIG = CIConfig(
]
),
CILabels.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS),
CILabels.CI_SET_NORMAL_BUILDS: LabelConfig(
run_jobs=[
JobNames.STYLE_CHECK,
JobNames.BUILD_CHECK,
Build.PACKAGE_RELEASE,
Build.PACKAGE_AARCH64,
Build.PACKAGE_ASAN,
Build.PACKAGE_UBSAN,
Build.PACKAGE_TSAN,
Build.PACKAGE_MSAN,
Build.PACKAGE_DEBUG,
Build.BINARY_RELEASE,
Build.PACKAGE_RELEASE_COVERAGE,
Build.FUZZERS,
]
),
CILabels.CI_SET_SPECIAL_BUILDS: LabelConfig(
run_jobs=[
JobNames.STYLE_CHECK,
JobNames.BUILD_CHECK_SPECIAL,
Build.BINARY_TIDY,
Build.BINARY_DARWIN,
Build.BINARY_AARCH64,
Build.BINARY_AARCH64_V80COMPAT,
Build.BINARY_FREEBSD,
Build.BINARY_DARWIN_AARCH64,
Build.BINARY_PPC64LE,
Build.BINARY_RISCV64,
Build.BINARY_S390X,
Build.BINARY_LOONGARCH64,
Build.BINARY_AMD64_COMPAT,
Build.BINARY_AMD64_MUSL,
]
),
CILabels.CI_SET_NON_REQUIRED: LabelConfig(
run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS]
),

View File

@ -309,9 +309,6 @@ def main():
state, description, test_results, additional_logs = process_results(
result_path, server_log_path
)
# FIXME (alesapin)
if "azure" in check_name:
state = "success"
else:
print(
"This is validate bugfix or flaky check run, but no changes test to run - skip with success"

View File

@ -808,10 +808,10 @@ class SettingsRandomizer:
"merge_tree_coarse_index_granularity": lambda: random.randint(2, 32),
"optimize_distinct_in_order": lambda: random.randint(0, 1),
"max_bytes_before_external_sort": threshold_generator(
0.3, 0.5, 1, 10 * 1024 * 1024 * 1024
0.3, 0.5, 0, 10 * 1024 * 1024 * 1024
),
"max_bytes_before_external_group_by": threshold_generator(
0.3, 0.5, 1, 10 * 1024 * 1024 * 1024
0.3, 0.5, 0, 10 * 1024 * 1024 * 1024
),
"max_bytes_before_remerge_sort": lambda: random.randint(1, 3000000000),
"min_compress_block_size": lambda: random.randint(1, 1048576 * 3),
@ -850,6 +850,11 @@ class SettingsRandomizer:
"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability": lambda: round(
random.random(), 2
),
"prefer_external_sort_block_bytes": lambda: random.choice([0, 1, 100000000]),
"cross_join_min_rows_to_compress": lambda: random.choice([0, 1, 100000000]),
"cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]),
"min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]),
"max_parsing_threads": lambda: random.choice([0, 1, 10]),
}
@staticmethod

View File

@ -0,0 +1,7 @@
<clickhouse>
<rocksdb>
<options>
<info_log_level>DEBUG_LEVEL</info_log_level>
</options>
</rocksdb>
</clickhouse>

View File

@ -68,6 +68,7 @@ ln -sf $SRC_PATH/config.d/zero_copy_destructive_operations.xml $DEST_SERVER_PATH
ln -sf $SRC_PATH/config.d/block_number.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/handlers.yaml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/
# Not supported with fasttest.
if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ]

View File

@ -98,7 +98,8 @@ def create_destination_table(node, table_name, replicated):
)
def test_both_mergtree(start_cluster):
def test_both_mergetree(start_cluster):
cleanup([replica1, replica2])
create_source_table(replica1, "source", False)
create_destination_table(replica1, "destination", False)
@ -120,12 +121,13 @@ def test_both_mergtree(start_cluster):
def test_all_replicated(start_cluster):
cleanup([replica1, replica2])
create_source_table(replica1, "source", True)
create_destination_table(replica1, "destination", True)
create_destination_table(replica2, "destination", True)
replica1.query("SYSTEM SYNC REPLICA destination")
replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source")
replica2.query("SYSTEM SYNC REPLICA destination")
assert_eq_with_retry(
replica1,
@ -154,12 +156,13 @@ def test_all_replicated(start_cluster):
def test_only_destination_replicated(start_cluster):
cleanup([replica1, replica2])
create_source_table(replica1, "source", False)
create_destination_table(replica1, "destination", True)
create_destination_table(replica2, "destination", True)
replica1.query("SYSTEM SYNC REPLICA destination")
replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source")
replica2.query("SYSTEM SYNC REPLICA destination")
assert_eq_with_retry(
replica1,
@ -188,6 +191,7 @@ def test_only_destination_replicated(start_cluster):
def test_not_work_on_different_disk(start_cluster):
cleanup([replica1, replica2])
# Replace and move should not work on replace
create_source_table(replica1, "source", False)
create_destination_table(replica2, "destination", False)

View File

@ -45,7 +45,9 @@ def test_failed_async_inserts(started_cluster):
ignore_error=True,
)
select_query = "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = '4Mi'"
select_query = (
"SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery'"
)
assert node.query(select_query) == "4\n"

View File

@ -26,7 +26,7 @@ def create_tables(cluster):
n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster}")
n1.query(
f"CREATE TABLE test_table ON CLUSTER {cluster} (key Int32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))"
f"CREATE TABLE test_table ON CLUSTER {cluster} (key UInt32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))"
)
n1.query(
f"""

View File

@ -28,10 +28,10 @@ def create_tables(cluster, table_name):
node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC")
node1.query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)"
f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)"
)
node3.query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)"
f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)"
)
# populate data

View File

@ -33,7 +33,7 @@ def create_tables(table_name):
for i in range(0, 4):
nodes[i].query(f"DROP TABLE IF EXISTS {table_name} SYNC")
nodes[i].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r{i+1}') ORDER BY (key)"
f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r{i+1}') ORDER BY (key)"
)
# populate data

View File

@ -0,0 +1 @@
#!/usr/bin/env python3

View File

@ -0,0 +1,11 @@
<clickhouse>
<profiles>
<default>
<allow_experimental_analyzer>1</allow_experimental_analyzer>
<allow_experimental_parallel_reading_from_replicas>1</allow_experimental_parallel_reading_from_replicas>
<cluster_for_parallel_replicas>default</cluster_for_parallel_replicas>
<max_parallel_replicas>100</max_parallel_replicas>
<use_hedged_requests>0</use_hedged_requests>
</default>
</profiles>
</clickhouse>

View File

@ -0,0 +1,6 @@
<clickhouse>
<macros>
<replica>node1</replica>
<shard>default</shard>
</macros>
</clickhouse>

View File

@ -0,0 +1,51 @@
#!/usr/bin/env python3
import logging
import random
import string
import time
import pytest
from multiprocessing.dummy import Pool
from helpers.cluster import ClickHouseCluster
import minio
cluster = ClickHouseCluster(__file__)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.add_instance(
"node1",
main_configs=["configs/node1_macro.xml"],
user_configs=[
"configs/enable_parallel_replicas.xml",
],
with_minio=True,
with_zookeeper=True,
)
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_replicated_db_and_ttl(started_cluster):
node1 = cluster.instances["node1"]
node1.query("DROP DATABASE default")
node1.query("CREATE DATABASE default ENGINE Replicated('/replicated')")
node1.query(
"CREATE TABLE 02908_main (a UInt32) ENGINE = ReplicatedMergeTree ORDER BY a"
)
node1.query(
"CREATE TABLE 02908_dependent (a UInt32, ts DateTime) ENGINE = ReplicatedMergeTree ORDER BY a TTL ts + 1 WHERE a IN (SELECT a FROM 02908_main)"
)
node1.query("INSERT INTO 02908_main VALUES (1)")
node1.query("INSERT INTO 02908_dependent VALUES (1, now())")
node1.query("SELECT * FROM 02908_dependent")

View File

@ -136,10 +136,7 @@ def test_select_clamps_settings():
)
assert (
distributed.query(
query,
settings={"max_memory_usage": 1, "min_untracked_memory": 4 * 1024 * 1024},
)
distributed.query(query, settings={"max_memory_usage": 1})
== "node1\tmax_memory_usage\t11111111\n"
"node1\treadonly\t0\n"
"node2\tmax_memory_usage\t0\n"

View File

@ -4,10 +4,10 @@ CREATE TABLE nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeT
INSERT INTO nested SELECT number, number % 2, range(number % 10) FROM system.numbers LIMIT 100000;
ALTER TABLE nested ADD COLUMN n.b Array(UInt64);
SELECT DISTINCT n.b FROM nested PREWHERE filter;
SELECT DISTINCT n.b FROM nested PREWHERE filter ORDER BY ALL;
ALTER TABLE nested ADD COLUMN n.c Array(UInt64) DEFAULT arrayMap(x -> x * 2, n.a);
SELECT DISTINCT n.c FROM nested PREWHERE filter;
SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter;
SELECT DISTINCT n.c FROM nested PREWHERE filter ORDER BY ALL;
SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter ORDER BY ALL;
DROP TABLE nested;

View File

@ -7,8 +7,7 @@
-- sizeof(HLL) is (2^K * 6 / 8)
-- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400
SET use_uncompressed_cache = 0;
SET min_untracked_memory = '4Mi';
SET use_uncompressed_cache = 0;
-- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements
SELECT 'UInt32';

View File

@ -44,12 +44,12 @@ Filter
9 10 1
> one condition of filter should be pushed down after aggregating, other condition is aliased
Filter column
ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4))
ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2
Aggregating
Filter column: notEquals(y, 0)
> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased
Filter column
ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))
ALIAS notEquals(__table1.s, 4_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))
Aggregating
Filter column: notEquals(__table1.y, 0_UInt8)
0 1
@ -63,12 +63,12 @@ Filter column: notEquals(__table1.y, 0_UInt8)
9 10
> one condition of filter should be pushed down after aggregating, other condition is casted
Filter column
FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2
FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4))
Aggregating
Filter column: notEquals(y, 0)
> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted
Filter column
FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2
FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8))
Aggregating
Filter column: notEquals(__table1.y, 0_UInt8)
0 1
@ -82,12 +82,12 @@ Filter column: notEquals(__table1.y, 0_UInt8)
9 10
> one condition of filter should be pushed down after aggregating, other two conditions are ANDed
Filter column
FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))
FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))
Aggregating
Filter column: notEquals(y, 0)
> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed
Filter column
FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))
FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))
Aggregating
Filter column: notEquals(__table1.y, 0_UInt8)
0 1
@ -100,12 +100,12 @@ Filter column: notEquals(__table1.y, 0_UInt8)
9 10
> two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased
Filter column
ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))
ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))
Aggregating
Filter column: and(notEquals(y, 0), minus(y, 4))
> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased
Filter column
ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))
ALIAS notEquals(__table1.s, 8_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))
Aggregating
Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))
0 1

View File

@ -49,14 +49,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s != 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2"
echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased"
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
explain actions = 1 select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s != 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
@ -69,14 +69,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4))"
echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted"
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
explain actions = 1 select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8))"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
@ -89,14 +89,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 8 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))"
echo "> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed"
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q "
explain actions = 1 select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 8 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
@ -109,14 +109,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s != 8 and y - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))"
grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))"
echo "> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased"
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q "
explain actions = 1 select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s != 8 and y - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))"
grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y

View File

@ -7,6 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "create table insert_big_json(a String, b String) engine=MergeTree() order by tuple()";
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 --max_parsing_threads=2 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
${CLICKHOUSE_CLIENT} -q "drop table insert_big_json"

View File

@ -4,7 +4,7 @@ CREATE TABLE system.aggregate_function_combinators
`is_internal` UInt8
)
ENGINE = SystemAggregateFunctionCombinators
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all available aggregate function combinators, which could be applied to aggregate functions and change the way they work.'
CREATE TABLE system.asynchronous_inserts
(
`query` String,
@ -17,7 +17,7 @@ CREATE TABLE system.asynchronous_inserts
`entries.bytes` Array(UInt64)
)
ENGINE = SystemAsynchronousInserts
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about pending asynchronous inserts in queue in server\'s memory.'
CREATE TABLE system.asynchronous_metrics
(
`metric` String,
@ -25,14 +25,14 @@ CREATE TABLE system.asynchronous_metrics
`description` String
)
ENGINE = SystemAsynchronousMetrics
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains metrics that are calculated periodically in the background. For example, the amount of RAM in use.'
CREATE TABLE system.build_options
(
`name` String,
`value` String
)
ENGINE = SystemBuildOptions
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all build flags, compiler options and commit hash for used build.'
CREATE TABLE system.clusters
(
`cluster` String,
@ -55,14 +55,14 @@ CREATE TABLE system.clusters
`name` String ALIAS cluster
)
ENGINE = SystemClusters
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about clusters defined in the configuration file or generated by a Replicated database.'
CREATE TABLE system.collations
(
`name` String,
`language` Nullable(String)
)
ENGINE = SystemTableCollations
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all available collations for alphabetical comparison of strings.'
CREATE TABLE system.columns
(
`database` String,
@ -88,13 +88,13 @@ CREATE TABLE system.columns
`datetime_precision` Nullable(UInt64)
)
ENGINE = SystemColumns
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Lists all columns from all tables of the current server.'
CREATE TABLE system.contributors
(
`name` String
)
ENGINE = SystemContributors
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all ClickHouse contributors <3'
CREATE TABLE system.current_roles
(
`role_name` String,
@ -102,7 +102,7 @@ CREATE TABLE system.current_roles
`is_default` UInt8
)
ENGINE = SystemCurrentRoles
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains active roles of a current user. SET ROLE changes the contents of this table.'
CREATE TABLE system.data_skipping_indices
(
`database` String,
@ -117,7 +117,7 @@ CREATE TABLE system.data_skipping_indices
`marks` UInt64
)
ENGINE = SystemDataSkippingIndices
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains all the information about all the data skipping indices in tables, similar to system.columns.'
CREATE TABLE system.data_type_families
(
`name` String,
@ -125,7 +125,7 @@ CREATE TABLE system.data_type_families
`alias_to` String
)
ENGINE = SystemTableDataTypeFamilies
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all available native data types along with all the aliases used for compatibility with other DBMS.'
CREATE TABLE system.databases
(
`name` String,
@ -138,7 +138,7 @@ CREATE TABLE system.databases
`database` String ALIAS name
)
ENGINE = SystemDatabases
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Lists all databases of the current server.'
CREATE TABLE system.detached_parts
(
`database` String,
@ -155,7 +155,7 @@ CREATE TABLE system.detached_parts
`level` Nullable(UInt32)
)
ENGINE = SystemDetachedParts
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all parts which are being found in /detached directory along with a reason why it was detached. ClickHouse server doesn\'t use such parts anyhow.'
CREATE TABLE system.dictionaries
(
`database` String,
@ -185,7 +185,7 @@ CREATE TABLE system.dictionaries
`comment` String
)
ENGINE = SystemDictionaries
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about dictionaries.'
CREATE TABLE system.disks
(
`name` String,
@ -205,7 +205,7 @@ CREATE TABLE system.disks
`cache_path` String
)
ENGINE = SystemDisks
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about disks defined in the server configuration.'
CREATE TABLE system.distributed_ddl_queue
(
`entry` String,
@ -225,7 +225,7 @@ CREATE TABLE system.distributed_ddl_queue
`query_duration_ms` Nullable(UInt64)
)
ENGINE = SystemDDLWorkerQueue
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about distributed DDL queries (ON CLUSTER clause) that were executed on a cluster.'
CREATE TABLE system.distribution_queue
(
`database` String,
@ -241,7 +241,7 @@ CREATE TABLE system.distribution_queue
`last_exception_time` DateTime
)
ENGINE = SystemDistributionQueue
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about local files that are in the queue to be sent to the shards. These local files contain new parts that are created by inserting new data into the Distributed table in asynchronous mode.'
CREATE TABLE system.enabled_roles
(
`role_name` String,
@ -250,7 +250,7 @@ CREATE TABLE system.enabled_roles
`is_default` UInt8
)
ENGINE = SystemEnabledRoles
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains all active roles at the moment, including current role of the current user and granted roles for current role.'
CREATE TABLE system.errors
(
`name` String,
@ -262,7 +262,7 @@ CREATE TABLE system.errors
`remote` UInt8
)
ENGINE = SystemErrors
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all errors which have ever happened including the error code, last time and message with unsymbolized stacktrace.'
CREATE TABLE system.events
(
`event` String,
@ -271,7 +271,7 @@ CREATE TABLE system.events
`name` String ALIAS event
)
ENGINE = SystemEvents
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains profiling events and their current value.'
CREATE TABLE system.formats
(
`name` String,
@ -281,7 +281,7 @@ CREATE TABLE system.formats
`supports_parallel_formatting` UInt8
)
ENGINE = SystemFormats
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all the formats along with flags whether a format is suitable for input/output or whether it supports parallelization.'
CREATE TABLE system.functions
(
`name` String,
@ -299,7 +299,7 @@ CREATE TABLE system.functions
`categories` String
)
ENGINE = SystemFunctions
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all available ordinary and aggregate functions with their descriptions.'
CREATE TABLE system.graphite_retentions
(
`config_name` String,
@ -314,7 +314,7 @@ CREATE TABLE system.graphite_retentions
`Tables.table` Array(String)
)
ENGINE = SystemGraphite
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about parameters graphite_rollup which are used in tables with *GraphiteMergeTree engines.'
CREATE TABLE system.licenses
(
`library_name` String,
@ -323,14 +323,14 @@ CREATE TABLE system.licenses
`license_text` String
)
ENGINE = SystemLicenses
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains licenses of third-party libraries that are located in the contrib directory of ClickHouse sources.'
CREATE TABLE system.macros
(
`macro` String,
`substitution` String
)
ENGINE = SystemMacros
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all macros defined in server configuration.'
CREATE TABLE system.merge_tree_settings
(
`name` String,
@ -344,7 +344,7 @@ CREATE TABLE system.merge_tree_settings
`is_obsolete` UInt8
)
ENGINE = SystemMergeTreeSettings
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all MergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query.'
CREATE TABLE system.merges
(
`database` String,
@ -373,7 +373,7 @@ CREATE TABLE system.merges
`merge_algorithm` String
)
ENGINE = SystemMerges
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row.'
CREATE TABLE system.metrics
(
`metric` String,
@ -382,7 +382,7 @@ CREATE TABLE system.metrics
`name` String ALIAS metric
)
ENGINE = SystemMetrics
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains metrics which can be calculated instantly, or have a current value. For example, the number of simultaneously processed queries or the current replica delay. This table is always up to date.'
CREATE TABLE system.moves
(
`database` String,
@ -395,7 +395,7 @@ CREATE TABLE system.moves
`thread_id` UInt64
)
ENGINE = SystemMoves
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row.'
CREATE TABLE system.mutations
(
`database` String,
@ -414,25 +414,25 @@ CREATE TABLE system.mutations
`latest_fail_reason` String
)
ENGINE = SystemMutations
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of mutations and their progress. Each mutation command is represented by a single row.'
CREATE TABLE system.numbers
(
`number` UInt64
)
ENGINE = SystemNumbers
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.'
CREATE TABLE system.numbers_mt
(
`number` UInt64
)
ENGINE = SystemNumbers
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Multithreaded version of `system.numbers`. Numbers order is not guaranteed.'
CREATE TABLE system.one
(
`dummy` UInt8
)
ENGINE = SystemOne
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.'
CREATE TABLE system.part_moves_between_shards
(
`database` String,
@ -451,7 +451,7 @@ CREATE TABLE system.part_moves_between_shards
`last_exception` String
)
ENGINE = SystemShardMoves
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about parts which are currently in a process of moving between shards and their progress.'
CREATE TABLE system.parts
(
`partition` String,
@ -522,7 +522,7 @@ CREATE TABLE system.parts
`part_name` String ALIAS name
)
ENGINE = SystemParts
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of currently existing (both active and inactive) parts of all *-MergeTree tables. Each part is represented by a single row.'
CREATE TABLE system.parts_columns
(
`partition` String,
@ -582,7 +582,7 @@ CREATE TABLE system.parts_columns
`part_name` String ALIAS name
)
ENGINE = SystemPartsColumns
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of columns of all currently existing parts of all MergeTree tables. Each column is represented by a single row.'
CREATE TABLE system.processes
(
`is_initial_query` UInt8,
@ -630,7 +630,7 @@ CREATE TABLE system.processes
`Settings.Values` Array(String) ALIAS mapValues(Settings)
)
ENGINE = SystemProcesses
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of currently executing processes (queries) with their progress.'
CREATE TABLE system.projection_parts
(
`partition` String,
@ -698,7 +698,7 @@ CREATE TABLE system.projection_parts
`part_name` String ALIAS name
)
ENGINE = SystemProjectionParts
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of currently existing projection parts (a copy of some part containing aggregated data or just sorted in different order) created for all the projections for all tables within a cluster.'
CREATE TABLE system.projection_parts_columns
(
`partition` String,
@ -754,7 +754,7 @@ CREATE TABLE system.projection_parts_columns
`part_name` String ALIAS name
)
ENGINE = SystemProjectionPartsColumns
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of columns of all currently existing projection parts of all MergeTree tables. Each column is represented by a single row.'
CREATE TABLE system.quota_limits
(
`quota_name` String,
@ -773,7 +773,7 @@ CREATE TABLE system.quota_limits
`max_failed_sequential_authentications` Nullable(UInt64)
)
ENGINE = SystemQuotaLimits
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about maximums for all intervals of all quotas. Any number of rows or zero can correspond to specific quota.'
CREATE TABLE system.quota_usage
(
`quota_name` String,
@ -805,7 +805,7 @@ CREATE TABLE system.quota_usage
`max_failed_sequential_authentications` Nullable(UInt64)
)
ENGINE = SystemQuotaUsage
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains quota usage by the current user: how much is used and how much is left.'
CREATE TABLE system.quotas
(
`name` String,
@ -818,7 +818,7 @@ CREATE TABLE system.quotas
`apply_to_except` Array(String)
)
ENGINE = SystemQuotas
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about quotas.'
CREATE TABLE system.quotas_usage
(
`quota_name` String,
@ -851,7 +851,7 @@ CREATE TABLE system.quotas_usage
`max_failed_sequential_authentications` Nullable(UInt64)
)
ENGINE = SystemQuotasUsage
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains quota usage by all users.'
CREATE TABLE system.replicas
(
`database` String,
@ -891,7 +891,7 @@ CREATE TABLE system.replicas
`replica_is_active` Map(String, UInt8)
)
ENGINE = SystemReplicas
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information and status of all table replicas on current server. Each replica is represented by a single row.'
CREATE TABLE system.replicated_fetches
(
`database` String,
@ -912,7 +912,7 @@ CREATE TABLE system.replicated_fetches
`thread_id` UInt64
)
ENGINE = SystemReplicatedFetches
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about currently running background fetches.'
CREATE TABLE system.replicated_merge_tree_settings
(
`name` String,
@ -926,7 +926,7 @@ CREATE TABLE system.replicated_merge_tree_settings
`is_obsolete` UInt8
)
ENGINE = SystemReplicatedMergeTreeSettings
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all ReplicatedMergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query. '
CREATE TABLE system.replication_queue
(
`database` String,
@ -952,7 +952,7 @@ CREATE TABLE system.replication_queue
`merge_type` String
)
ENGINE = SystemReplicationQueue
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about tasks from replication queues stored in ClickHouse Keeper, or ZooKeeper, for each table replica.'
CREATE TABLE system.role_grants
(
`user_name` Nullable(String),
@ -963,7 +963,7 @@ CREATE TABLE system.role_grants
`with_admin_option` UInt8
)
ENGINE = SystemRoleGrants
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains the role grants for users and roles. To add entries to this table, use `GRANT role TO user`. Using this table you may find out which roles are assigned to which users or which roles a user has.'
CREATE TABLE system.roles
(
`name` String,
@ -971,7 +971,7 @@ CREATE TABLE system.roles
`storage` String
)
ENGINE = SystemRoles
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all roles created at the server.'
CREATE TABLE system.row_policies
(
`name` String,
@ -987,7 +987,7 @@ CREATE TABLE system.row_policies
`apply_to_except` Array(String)
)
ENGINE = SystemRowPolicies
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains filters for one particular table, as well as a list of roles and/or users which should use this row policy.'
CREATE TABLE system.settings
(
`name` String,
@ -1003,7 +1003,7 @@ CREATE TABLE system.settings
`is_obsolete` UInt8
)
ENGINE = SystemSettings
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all user-level settings (which can be modified in a scope of query or session), their current and default values along with descriptions.'
CREATE TABLE system.settings_profile_elements
(
`profile_name` Nullable(String),
@ -1018,7 +1018,7 @@ CREATE TABLE system.settings_profile_elements
`inherit_profile` Nullable(String)
)
ENGINE = SystemSettingsProfileElements
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Describes the content of each settings profile configured on the server. Including settings constraints, roles and users for which the settings are applied, and parent settings profiles.'
CREATE TABLE system.settings_profiles
(
`name` String,
@ -1030,7 +1030,7 @@ CREATE TABLE system.settings_profiles
`apply_to_except` Array(String)
)
ENGINE = SystemSettingsProfiles
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains properties of configured setting profiles.'
CREATE TABLE system.stack_trace
(
`thread_name` String,
@ -1039,7 +1039,7 @@ CREATE TABLE system.stack_trace
`trace` Array(UInt64)
)
ENGINE = SystemStackTrace
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Allows to obtain an unsymbolized stacktrace from all the threads of the server process.'
CREATE TABLE system.storage_policies
(
`policy_name` String,
@ -1054,7 +1054,7 @@ CREATE TABLE system.storage_policies
`load_balancing` Enum8('ROUND_ROBIN' = 0, 'LEAST_USED' = 1)
)
ENGINE = SystemStoragePolicies
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains information about storage policies and volumes defined in the server configuration.'
CREATE TABLE system.table_engines
(
`name` String,
@ -1068,7 +1068,7 @@ CREATE TABLE system.table_engines
`supports_parallel_insert` UInt8
)
ENGINE = SystemTableEngines
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all available table engines along with information whether a particular table engine supports some specific features (e.g. settings, skipping indices, projections, replication, TTL, deduplication, parallel insert, etc.)'
CREATE TABLE system.table_functions
(
`name` String,
@ -1076,7 +1076,7 @@ CREATE TABLE system.table_functions
`allow_readonly` UInt8
)
ENGINE = SystemTableFunctions
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all available table functions with their descriptions.'
CREATE TABLE system.tables
(
`database` String,
@ -1115,13 +1115,13 @@ CREATE TABLE system.tables
`table` String ALIAS name
)
ENGINE = SystemTables
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Lists all tables of the current server.'
CREATE TABLE system.time_zones
(
`time_zone` String
)
ENGINE = SystemTimeZones
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of time zones that are supported by the ClickHouse server. This list of timezones might vary depending on the version of ClickHouse.'
CREATE TABLE system.user_directories
(
`name` String,
@ -1130,7 +1130,7 @@ CREATE TABLE system.user_directories
`precedence` UInt64
)
ENGINE = SystemUserDirectories
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains the information about configured user directories - directories on the file system from which ClickHouse server is allowed to read user provided data.'
CREATE TABLE system.users
(
`name` String,
@ -1151,22 +1151,22 @@ CREATE TABLE system.users
`default_database` String
)
ENGINE = SystemUsers
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains a list of all users profiles either configured at the server through a configuration file or created via SQL.'
CREATE TABLE system.warnings
(
`message` String
)
ENGINE = SystemWarnings
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Contains warnings about server configuration to be displayed by clickhouse-client right after it connects to the server.'
CREATE TABLE system.zeros
(
`zero` UInt8
)
ENGINE = SystemZeros
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Produces unlimited number of non-materialized zeros.'
CREATE TABLE system.zeros_mt
(
`zero` UInt8
)
ENGINE = SystemZeros
COMMENT 'SYSTEM TABLE is built on the fly.'
COMMENT 'Multithreaded version of system.zeros.'

View File

@ -1 +1 @@
CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB\nCOMMENT \'Contains a list of metrics exposed from embedded RocksDB.\'

View File

@ -87,7 +87,7 @@ Sorting (Stream): a ASC, b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, b ASC
Sorting (Stream): a ASC, b ASC
-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns
Sorting (Stream): __table1.a ASC
Sorting (Stream): __table1.a ASC

View File

@ -24,6 +24,7 @@ function check_refcnt_for_table()
local log_file
log_file=$(mktemp "$CUR_DIR/clickhouse-tests.XXXXXX.log")
local args=(
--allow_repeated_settings
--format Null
--max_threads 1
--max_block_size 1

View File

@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is
rm -f $path/count.txt
$CLICKHOUSE_CLIENT -q "detach table rmt2 sync"
$CLICKHOUSE_CLIENT --send_logs_level='fatal' -q "attach table rmt2"
$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level='fatal' -q "attach table rmt2"
$CLICKHOUSE_CLIENT -q "select reason, name from system.detached_parts where database='$CLICKHOUSE_DATABASE' and table='rmt2'"

View File

@ -20,12 +20,12 @@ SETTINGS_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, allow_experimental
# Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated
# with EXPLAIN PLAN, we need to check the logs.
${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l
${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l
${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l
${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE"
${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l
${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l
${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l
${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE"

View File

@ -30,7 +30,7 @@ run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas
$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key"
$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y UInt32) ENGINE = MergeTree ORDER BY cityHash64(x)"
$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key SELECT toString(number), number % 3 FROM numbers(1000)"
function run_count_with_custom_key {
@ -41,6 +41,6 @@ run_count_with_custom_key "y"
run_count_with_custom_key "cityHash64(y)"
run_count_with_custom_key "cityHash64(y) + 1"
$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with"
$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --allow_repeated_settings --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with"
$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key"

View File

@ -24,10 +24,10 @@ Positions: 3 0 1
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table1.id UInt64 : 3
ALIAS value :: 1 -> __table1.value String : 0
FUNCTION equals(__table1.id : 3, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1
Positions: 1 3 0
ALIAS id : 0 -> __table1.id UInt64 : 3
ALIAS value :: 1 -> __table1.value String : 4
FUNCTION equals(id :: 0, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1
Positions: 1 3 4
ReadFromMergeTree (default.test_table)
Header: id UInt64
value String

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long, zookeeper, no-parallel, no-fasttest
# Tags: long, zookeeper, no-parallel, no-fasttest, no-asan
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -58,9 +58,9 @@ function filter_temporary_locks()
function insert_duplicates() {
$CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --send_logs_level="error" &
$CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --allow_repeated_settings --send_logs_level="error" &
$CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --send_logs_level="error"
$CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --allow_repeated_settings --send_logs_level="error"
wait
@ -137,8 +137,8 @@ function list_keeper_nodes() {
list_keeper_nodes "${table_shared_id}"
$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --send_logs_level="error" &
$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --send_logs_level="error" &
$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --allow_repeated_settings --send_logs_level="error" &
$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --allow_repeated_settings --send_logs_level="error" &
wait
list_keeper_nodes "${table_shared_id}"

View File

@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192"
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1"
function test1_insert()
{
@ -115,11 +115,11 @@ run 0
$CH_CLIENT -q "drop table test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;"
run 1
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;"
run 1
$CH_CLIENT -q "drop table test;"

View File

@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192"
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1"
function test4_insert()
{
@ -61,11 +61,11 @@ run 0
$CH_CLIENT -q "drop table test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;"
run 1
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;"
run 1
$CH_CLIENT -q "drop table test;"

View File

@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 "
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1"
function test5_insert()
{
@ -63,11 +63,11 @@ run 0
$CH_CLIENT -q "drop table test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;"
run 1
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;"
run 1
$CH_CLIENT -q "drop table test;"

View File

@ -7,7 +7,8 @@ CLICKHOUSE_LOG_COMMENT=
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 "
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1"
function test6_insert()
{
@ -57,11 +58,11 @@ run 0
$CH_CLIENT -q "drop table test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;"
run 1
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;"
run 1
$CH_CLIENT -q "drop table test;"

View File

@ -0,0 +1,290 @@
-- { echoOn }
-- inequality operation
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
--
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
--
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
-- BETWEEN
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
1 10 alpha 0 0
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
1 10 alpha 0 0
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
2 15 beta 2 10 beta
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
2 15 beta 2 10 beta
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
0 0 1 5 ALPHA
0 0 4 25 delta
2 15 beta 2 10 beta
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
0 0 1 5 ALPHA
0 0 4 25 delta
2 15 beta 2 10 beta
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
0 0 1 5 ALPHA
0 0 4 25 delta
1 10 alpha 0 0
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
0 0 1 5 ALPHA
0 0 4 25 delta
1 10 alpha 0 0
2 15 beta 2 10 beta
3 20 gamma 0 0
--
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
-- Stupid condition
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 2 10 beta
3 20 gamma 0 0
-- Window functions with stupid condition
SET join_algorithm='hash';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
1 10 alpha 1 5 ALPHA 15 3
2 15 beta 2 10 beta 15 3
3 20 gamma 0 0 15 3
SET join_algorithm='grace_hash';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
1 10 alpha 1 5 ALPHA 15 3
2 15 beta 2 10 beta 15 3
3 20 gamma 0 0 15 3
SET join_algorithm='hash';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
1 10 alpha 1 5 ALPHA 12.5 3
2 15 beta 2 10 beta 12.5 3
SET join_algorithm='grace_hash';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
1 10 alpha 1 5 ALPHA 12.5 3
2 15 beta 2 10 beta 12.5 3
SET join_algorithm='hash';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
0 0 4 25 delta 8.333333333333334 7
1 10 alpha 1 5 ALPHA 8.333333333333334 7
2 15 beta 2 10 beta 8.333333333333334 7
SET join_algorithm='grace_hash';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
0 0 4 25 delta 8.333333333333334 7
1 10 alpha 1 5 ALPHA 8.333333333333334 7
2 15 beta 2 10 beta 8.333333333333334 7
SET join_algorithm='hash';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
0 0 4 25 delta 11.25 7
1 10 alpha 1 5 ALPHA 11.25 7
2 15 beta 2 10 beta 11.25 7
3 20 gamma 0 0 11.25 7
SET join_algorithm='grace_hash';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
0 0 4 25 delta 11.25 7
1 10 alpha 1 5 ALPHA 11.25 7
2 15 beta 2 10 beta 11.25 7
3 20 gamma 0 0 11.25 7
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;

View File

@ -0,0 +1,82 @@
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (
key UInt32,
a UInt32,
attr String
) ENGINE = MergeTree ORDER BY key;
CREATE TABLE t2 (
key UInt32,
a UInt32,
attr String
) ENGINE = MergeTree ORDER BY key;
INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, 'gamma');
INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta');
SET allow_experimental_analyzer=1;
SET allow_experimental_join_condition=1;
SET join_use_nulls=0;
-- { echoOn }
-- inequality operation
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
--
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL;
{% endfor -%}
{% endfor -%}
--
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
-- BETWEEN
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
--
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
-- Stupid condition
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
-- Window functions with stupid condition
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 {{ join_type }} JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;

View File

@ -0,0 +1,90 @@
-- { echoOn }
-- Support for query lower
SET join_algorithm='hash';
SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='hash';
SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
SET join_algorithm='grace_hash';
SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
1 10 alpha 1 5 ALPHA
SET join_algorithm='hash';
SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
SET join_algorithm='grace_hash';
SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
SET join_algorithm='hash';
SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
SET join_algorithm='grace_hash';
SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
0 0 2 10 beta
0 0 4 25 delta
1 10 alpha 1 5 ALPHA
2 15 beta 0 0
3 20 gamma 0 0
-- Subquery JOIN
SET join_algorithm='hash';
SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) LEFT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
1 10 alpha 1 5 ALPHA 1 100
2 15 beta 0 0 0 0
3 20 gamma 0 0 0 0
SET join_algorithm='grace_hash';
SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) LEFT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
1 10 alpha 1 5 ALPHA 1 100
2 15 beta 0 0 0 0
3 20 gamma 0 0 0 0
SET join_algorithm='hash';
SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) INNER JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
1 10 alpha 1 5 ALPHA 1 100
SET join_algorithm='grace_hash';
SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) INNER JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
1 10 alpha 1 5 ALPHA 1 100
SET join_algorithm='hash';
SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) RIGHT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
0 0 0 0 0 10
0 0 0 0 2 1000
1 10 alpha 1 5 ALPHA 1 100
SET join_algorithm='grace_hash';
SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) RIGHT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
0 0 0 0 0 10
0 0 0 0 2 1000
1 10 alpha 1 5 ALPHA 1 100
SET join_algorithm='hash';
SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) FULL JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
0 0 0 0 0 10
0 0 0 0 2 1000
0 0 2 10 beta 0 0
0 0 4 25 delta 0 0
1 10 alpha 1 5 ALPHA 1 100
2 15 beta 0 0 0 0
3 20 gamma 0 0 0 0
SET join_algorithm='grace_hash';
SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) FULL JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
0 0 0 0 0 10
0 0 0 0 2 1000
0 0 2 10 beta 0 0
0 0 4 25 delta 0 0
1 10 alpha 1 5 ALPHA 1 100
2 15 beta 0 0 0 0
3 20 gamma 0 0 0 0
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;

View File

@ -0,0 +1,44 @@
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (
key UInt32,
a UInt32,
attr String
) ENGINE = MergeTree ORDER BY key;
CREATE TABLE t2 (
key UInt32,
a UInt32,
attr String
) ENGINE = MergeTree ORDER BY key;
INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, 'gamma');
INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta');
SET allow_experimental_analyzer=1;
SET allow_experimental_join_condition=1;
SET join_use_nulls=0;
-- { echoOn }
-- Support for query lower
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT * FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
-- Subquery JOIN
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT * FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) {{ join_type }} JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL;
{% endfor -%}
{% endfor -%}
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;

View File

@ -0,0 +1,163 @@
-- { echoOn }
-- These queries work
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
2 15 2 10
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
2 15 2 10
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
2 15 2 10
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
2 15 2 10
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
1 10 0 0
2 15 0 0
3 20 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
1 10 0 0
2 15 0 0
3 20 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 2 10
0 0 4 25
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 2 10
0 0 4 25
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 2 10
0 0 4 25
1 10 0 0
2 15 0 0
3 20 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 2 10
0 0 4 25
1 10 0 0
2 15 0 0
3 20 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
2 15 2 10
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
2 15 2 10
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
2 15 2 10
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
2 15 2 10
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
2 15 2 10
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
2 15 2 10
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
2 15 2 10
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
2 15 2 10
SET join_algorithm='hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
1 10 0 0
2 15 2 10
3 20 0 0
SET join_algorithm='grace_hash';
SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
0 0 1 5
0 0 4 25
1 10 0 0
2 15 2 10
3 20 0 0

View File

@ -0,0 +1,51 @@
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (
key UInt32,
a UInt32
) ENGINE = MergeTree ORDER BY key;
CREATE TABLE t2 (
key UInt32,
a UInt32
) ENGINE = MergeTree ORDER BY key;
INSERT INTO t1 (key, a) VALUES (1, 10), (2, 15), (3, 20);
INSERT INTO t2 (key, a) VALUES (1, 5), (2, 10), (4, 25);
SET allow_experimental_analyzer=1;
SET allow_experimental_join_condition=1;
SET join_algorithm='hash';
-- { echoOn }
-- These queries work
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
{% endfor -%}
{% endfor -%}
{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%}
{% for algorithm in ['hash', 'grace_hash'] -%}
SET join_algorithm='{{ algorithm }}';
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL;
{% endfor -%}
{% endfor -%}

View File

@ -1,49 +0,0 @@
#!/usr/bin/expect -f
set basedir [file dirname $argv0]
set basename [file tail $argv0]
if {[info exists env(CLICKHOUSE_TMP)]} {
set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP)
} else {
set CLICKHOUSE_TMP "."
}
exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0
log_user 0
set timeout 60
match_max 100000
set stty_init "rows 25 cols 120"
expect_after {
-i $any_spawn_id eof { exp_continue }
-i $any_spawn_id timeout { exit 1 }
}
spawn clickhouse-local
expect ":) "
# Trivial SELECT with LIMIT from system.zeros shows progress bar.
send "SELECT * FROM system.zeros LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
send "SELECT * FROM system.zeros_mt LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
# As well as from generateRandom
send "SELECT * FROM generateRandom() LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
send "exit\r"
expect eof

Some files were not shown because too many files have changed in this diff Show More