Merge branch 'master' into tighten-limits-functional-tests

This commit is contained in:
Alexey Milovidov 2024-08-10 00:20:25 +02:00
commit 341f44a943
145 changed files with 4327 additions and 1386 deletions

View File

@ -428,12 +428,17 @@ if (NOT SANITIZE)
set (CMAKE_POSITION_INDEPENDENT_CODE OFF)
endif()
if (OS_LINUX AND NOT (ARCH_AARCH64 OR ARCH_S390X) AND NOT SANITIZE)
# Slightly more efficient code can be generated
# It's disabled for ARM because otherwise ClickHouse cannot run on Android.
if (NOT OS_ANDROID AND OS_LINUX AND NOT ARCH_S390X AND NOT SANITIZE)
# Using '-no-pie' builds executables with fixed addresses, resulting in slightly more efficient code
# and keeping binary addresses constant even with ASLR enabled.
# Disabled on Android as it requires PIE: https://source.android.com/docs/security/enhancements#android-5
# Disabled on IBM S390X due to build issues with 'no-pie'
# Disabled with sanitizers to avoid issues with maximum relocation size: https://github.com/ClickHouse/ClickHouse/pull/49145
set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie")
set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie -Wl,-no-pie")
else ()
message (WARNING "ClickHouse is built as PIE, system.trace_log will contain invalid addresses after server restart.")
endif ()
if (ENABLE_TESTS)

View File

@ -233,15 +233,26 @@ function run_tests()
set +e
TEST_ARGS=(
-j 2
--testname
--shard
--zookeeper
--check-zookeeper-session
--no-stateless
--hung-check
--print-time
--capture-client-stacktrace
"${ADDITIONAL_OPTIONS[@]}"
"$SKIP_TESTS_OPTION"
)
if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then
clickhouse-test --client="clickhouse-client --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 \
--max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" \
-j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --no-parallel-replicas --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \
"$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
else
clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \
"$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
TEST_ARGS+=(
--client="clickhouse-client --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'"
--no-parallel-replicas
)
fi
clickhouse-test "${TEST_ARGS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
set -e
}

View File

@ -264,11 +264,22 @@ function run_tests()
TIMEOUT=$((MAX_RUN_TIME - 800 > 8400 ? 8400 : MAX_RUN_TIME - 800))
START_TIME=${SECONDS}
set +e
timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s \
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
TEST_ARGS=(
--testname
--shard
--zookeeper
--check-zookeeper-session
--hung-check
--print-time
--no-drop-if-fail
--capture-client-stacktrace
--test-runs "$NUM_TRIES"
"${ADDITIONAL_OPTIONS[@]}"
)
timeout --preserve-status --signal TERM --kill-after 60m ${TIMEOUT}s clickhouse-test "${TEST_ARGS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
set -e
DURATION=$((SECONDS - START_TIME))

View File

@ -61,6 +61,7 @@ Engines in the family:
- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md)
- [PostgreSQL](../../engines/table-engines/integrations/postgresql.md)
- [S3Queue](../../engines/table-engines/integrations/s3queue.md)
- [TimeSeries](../../engines/table-engines/integrations/time-series.md)
### Special Engines {#special-engines}

View File

@ -0,0 +1,295 @@
---
slug: /en/engines/table-engines/special/time_series
sidebar_position: 60
sidebar_label: TimeSeries
---
# TimeSeries Engine [Experimental]
A table engine storing time series, i.e. a set of values associated with timestamps and tags (or labels):
```
metric_name1[tag1=value1, tag2=value2, ...] = {timestamp1: value1, timestamp2: value2, ...}
metric_name2[...] = ...
```
:::info
This is an experimental feature that may change in backwards-incompatible ways in the future releases.
Enable usage of the TimeSeries table engine
with [allow_experimental_time_series_table](../../../operations/settings/settings.md#allow-experimental-time-series-table) setting.
Input the command `set allow_experimental_time_series_table = 1`.
:::
## Syntax {#syntax}
``` sql
CREATE TABLE name [(columns)] ENGINE=TimeSeries
[SETTINGS var1=value1, ...]
[DATA db.data_table_name | DATA ENGINE data_table_engine(arguments)]
[TAGS db.tags_table_name | TAGS ENGINE tags_table_engine(arguments)]
[METRICS db.metrics_table_name | METRICS ENGINE metrics_table_engine(arguments)]
```
## Usage {#usage}
It's easier to start with everything set by default (it's allowed to create a `TimeSeries` table without specifying a list of columns):
``` sql
CREATE TABLE my_table ENGINE=TimeSeries
```
Then this table can be used with the following protocols (a port must be assigned in the server configuration):
- [prometheus remote-write](../../../interfaces/prometheus.md#remote-write)
- [prometheus remote-read](../../../interfaces/prometheus.md#remote-read)
## Target tables {#target-tables}
A `TimeSeries` table doesn't have its own data, everything is stored in its target tables.
This is similar to how a [materialized view](../../../sql-reference/statements/create/view#materialized-view) works,
with the difference that a materialized view has one target table
whereas a `TimeSeries` table has three target tables named [data]{#data-table}, [tags]{#tags-table], and [metrics]{#metrics-table}.
The target tables can be either specified explicitly in the `CREATE TABLE` query
or the `TimeSeries` table engine can generate inner target tables automatically.
The target tables are the following:
1. The _data_ table {#data-table} contains time series associated with some identifier.
The _data_ table must have columns:
| Name | Mandatory? | Default type | Possible types | Description |
|---|---|---|---|---|
| `id` | [x] | `UUID` | any | Identifies a combination of a metric names and tags |
| `timestamp` | [x] | `DateTime64(3)` | `DateTime64(X)` | A time point |
| `value` | [x] | `Float64` | `Float32` or `Float64` | A value associated with the `timestamp` |
2. The _tags_ table {#tags-table} contains identifiers calculated for each combination of a metric name and tags.
The _tags_ table must have columns:
| Name | Mandatory? | Default type | Possible types | Description |
|---|---|---|---|---|
| `id` | [x] | `UUID` | any (must match the type of `id` in the [data]{#data-table} table) | An `id` identifies a combination of a metric name and tags. The DEFAULT expression specifies how to calculate such an identifier |
| `metric_name` | [x] | `LowCardinality(String)` | `String` or `LowCardinality(String)` | The name of a metric |
| `<tag_value_column>` | [ ] | `String` | `String` or `LowCardinality(String)` or `LowCardinality(Nullable(String))` | The value of a specific tag, the tag's name and the name of a corresponding column are specified in the [tags_to_columns](#settings) setting |
| `tags` | [x] | `Map(LowCardinality(String), String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Map of tags excluding the tag `__name__` containing the name of a metric and excluding tags with names enumerated in the [tags_to_columns](#settings) setting |
| `all_tags` | [ ] | `Map(String, String)` | `Map(String, String)` or `Map(LowCardinality(String), String)` or `Map(LowCardinality(String), LowCardinality(String))` | Ephemeral column, each row is a map of all the tags excluding only the tag `__name__` containing the name of a metric. The only purpose of that column is to be used while calculating `id` |
| `min_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Minimum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` |
| `max_time` | [ ] | `Nullable(DateTime64(3))` | `DateTime64(X)` or `Nullable(DateTime64(X))` | Maximum timestamp of time series with that `id`. The column is created if [store_min_time_and_max_time](#settings) is `true` |
3. The _metrics_ table {#metrics-table} contains some information about metrics been collected, the types of those metrics and their descriptions.
The _metrics_ table must have columns:
| Name | Mandatory? | Default type | Possible types | Description |
|---|---|---|---|---|
| `metric_family_name` | [x] | `String` | `String` or `LowCardinality(String)` | The name of a metric family |
| `type` | [x] | `String` | `String` or `LowCardinality(String)` | The type of a metric family, one of "counter", "gauge", "summary", "stateset", "histogram", "gaugehistogram" |
| `unit` | [x] | `String` | `String` or `LowCardinality(String)` | The unit used in a metric |
| `help` | [x] | `String` | `String` or `LowCardinality(String)` | The description of a metric |
Any row inserted into a `TimeSeries` table will be in fact stored in those three target tables.
A `TimeSeries` table contains all those columns from the [data]{#data-table}, [tags]{#tags-table}, [metrics]{#metrics-table} tables.
## Creation {#creation}
There are multiple ways to create a table with the `TimeSeries` table engine.
The simplest statement
``` sql
CREATE TABLE my_table ENGINE=TimeSeries
```
will actually create the following table (you can see that by executing `SHOW CREATE TABLE my_table`):
``` sql
CREATE TABLE my_table
(
`id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)),
`timestamp` DateTime64(3),
`value` Float64,
`metric_name` LowCardinality(String),
`tags` Map(LowCardinality(String), String),
`all_tags` Map(String, String),
`min_time` Nullable(DateTime64(3)),
`max_time` Nullable(DateTime64(3)),
`metric_family_name` String,
`type` String,
`unit` String,
`help` String
)
ENGINE = TimeSeries
DATA ENGINE = MergeTree ORDER BY (id, timestamp)
DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef'
TAGS ENGINE = AggregatingMergeTree PRIMARY KEY metric_name ORDER BY (metric_name, id)
TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef'
METRICS ENGINE = ReplacingMergeTree ORDER BY metric_family_name
METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef'
```
So the columns were generated automatically and also there are three inner UUIDs in this statement -
one per each inner target table that was created.
(Inner UUIDs are not shown normally until setting
[show_table_uuid_in_table_create_query_if_not_nil](../../../operations/settings/settings#show_table_uuid_in_table_create_query_if_not_nil)
is set.)
Inner target tables have names like `.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`,
`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, `.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`
and each target table has columns which is a subset of the columns of the main `TimeSeries` table:
``` sql
CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`
(
`id` UUID,
`timestamp` DateTime64(3),
`value` Float64
)
ENGINE = MergeTree
ORDER BY (id, timestamp)
```
``` sql
CREATE TABLE default.`.inner_id.tags.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`
(
`id` UUID DEFAULT reinterpretAsUUID(sipHash128(metric_name, all_tags)),
`metric_name` LowCardinality(String),
`tags` Map(LowCardinality(String), String),
`all_tags` Map(String, String) EPHEMERAL,
`min_time` SimpleAggregateFunction(min, Nullable(DateTime64(3))),
`max_time` SimpleAggregateFunction(max, Nullable(DateTime64(3)))
)
ENGINE = AggregatingMergeTree
PRIMARY KEY metric_name
ORDER BY (metric_name, id)
```
``` sql
CREATE TABLE default.`.inner_id.metrics.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`
(
`metric_family_name` String,
`type` String,
`unit` String,
`help` String
)
ENGINE = ReplacingMergeTree
ORDER BY metric_family_name
```
## Adjusting types of columns {#adjusting-column-types}
You can adjust the types of almost any column of the inner target tables by specifying them explicitly
while defining the main table. For example,
``` sql
CREATE TABLE my_table
(
timestamp DateTime64(6)
) ENGINE=TimeSeries
```
will make the inner [data]{#data-table} table store timestamp in microseconds instead of milliseconds:
``` sql
CREATE TABLE default.`.inner_id.data.xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`
(
`id` UUID,
`timestamp` DateTime64(6),
`value` Float64
)
ENGINE = MergeTree
ORDER BY (id, timestamp)
```
## The `id` column {#id-column}
The `id` column contains identifiers, every identifier is calculated for a combination of a metric name and tags.
The DEFAULT expression for the `id` column is an expression which will be used to calculate such identifiers.
Both the type of the `id` column and that expression can be adjusted by specifying them explicitly:
``` sql
CREATE TABLE my_table
(
id UInt64 DEFAULT sipHash64(metric_name, all_tags)
) ENGINE=TimeSeries
```
## The `tags` and `all_tags` columns {#tags-and-all-tags}
There are two columns containing maps of tags - `tags` and `all_tags`. In this example they mean the same, however they can be different
if setting `tags_to_columns` is used. This setting allows to specify that a specific tag should be stored in a separate column instead of storing
in a map inside the `tags` column:
``` sql
CREATE TABLE my_table ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'}
```
This statement will add columns
```
`instance` String,
`job` String
```
to the definition of both `my_table` and its inner [tags]{#tags-table} target table. In this case the `tags` column will not contain tags `instance` and `job`,
but the `all_tags` column will contain them. The `all_tags` column is ephemeral and its only purpose to be used in the DEFAULT expression
for the `id` column.
The types of columns can be adjusted by specifying them explicitly:
``` sql
CREATE TABLE my_table (instance LowCardinality(String), job LowCardinality(Nullable(String)))
ENGINE=TimeSeries SETTINGS = {'instance': 'instance', 'job': 'job'}
```
## Table engines of inner target tables {#inner-table-engines}
By default inner target tables use the following table engines:
- the [data]{#data-table} table uses [MergeTree](../mergetree-family/mergetree);
- the [tags]{#tags-table} table uses [AggregatingMergeTree](../mergetree-family/aggregatingmergetree) because the same data is often inserted multiple times to this table so we need a way
to remove duplicates, and also because it's required to do aggregation for columns `min_time` and `max_time`;
- the [metrics]{#metrics-table} table uses [ReplacingMergeTree](../mergetree-family/replacingmergetree) because the same data is often inserted multiple times to this table so we need a way
to remove duplicates.
Other table engines also can be used for inner target tables if it's specified so:
``` sql
CREATE TABLE my_table ENGINE=TimeSeries
DATA ENGINE=ReplicatedMergeTree
TAGS ENGINE=ReplicatedAggregatingMergeTree
METRICS ENGINE=ReplicatedReplacingMergeTree
```
## External target tables {#external-target-tables}
It's possible to make a `TimeSeries` table use a manually created table:
``` sql
CREATE TABLE data_for_my_table
(
`id` UUID,
`timestamp` DateTime64(3),
`value` Float64
)
ENGINE = MergeTree
ORDER BY (id, timestamp);
CREATE TABLE tags_for_my_table ...
CREATE TABLE metrics_for_my_table ...
CREATE TABLE my_table ENGINE=TimeSeries DATA data_for_my_table TAGS tags_for_my_table METRICS metrics_for_my_table;
```
## Settings {#settings}
Here is a list of settings which can be specified while defining a `TimeSeries` table:
| Name | Type | Default | Description |
|---|---|---|---|
| `tags_to_columns` | Map | {} | Map specifying which tags should be put to separate columns in the [tags]{#tags-table} table. Syntax: `{'tag1': 'column1', 'tag2' : column2, ...}` |
| `use_all_tags_column_to_generate_id` | Bool | true | When generating an expression to calculate an identifier of a time series, this flag enables using the `all_tags` column in that calculation |
| `store_min_time_and_max_time` | Bool | true | If set to true then the table will store `min_time` and `max_time` for each time series |
| `aggregate_min_time_and_max_time` | Bool | true | When creating an inner target `tags` table, this flag enables using `SimpleAggregateFunction(min, Nullable(DateTime64(3)))` instead of just `Nullable(DateTime64(3))` as the type of the `min_time` column, and the same for the `max_time` column |
| `filter_by_min_time_and_max_time` | Bool | true | If set to true then the table will use the `min_time` and `max_time` columns for filtering time series |
# Functions {#functions}
Here is a list of functions supporting a `TimeSeries` table as an argument:
- [timeSeriesData](../../../sql-reference/table-functions/timeSeriesData.md)
- [timeSeriesTags](../../../sql-reference/table-functions/timeSeriesTags.md)
- [timeSeriesMetrics](../../../sql-reference/table-functions/timeSeriesMetrics.md)

View File

@ -0,0 +1,160 @@
---
slug: /en/interfaces/prometheus
sidebar_position: 19
sidebar_label: Prometheus protocols
---
# Prometheus protocols
## Exposing metrics {#expose}
:::note
ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com.
:::
ClickHouse can expose its own metrics for scraping from Prometheus:
```xml
<prometheus>
<port>9363</port>
<endpoint>/metrics</endpoint>
<metrics>true</metrics>
<asynchronous_metrics>true</asynchronous_metrics>
<events>true</events>
<errors>true</errors>
</prometheus>
Section `<prometheus.handlers>` can be used to make more extended handlers.
This section is similar to [<http_handlers>](/en/interfaces/http) but works for prometheus protocols:
```xml
<prometheus>
<port>9363</port>
<handlers>
<my_rule_1>
<url>/metrics</url>
<handler>
<type>expose_metrics</type>
<metrics>true</metrics>
<asynchronous_metrics>true</asynchronous_metrics>
<events>true</events>
<errors>true</errors>
</handler>
</my_rule_1>
</handlers>
</prometheus>
```
Settings:
| Name | Default | Description |
|---|---|---|---|
| `port` | none | Port for serving the exposing metrics protocol. |
| `endpoint` | `/metrics` | HTTP endpoint for scraping metrics by prometheus server. Starts with `/`. Should not be used with the `<handlers>` section. |
| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [<http_handlers>](/en/interfaces/http) section. |
| `metrics` | true | Expose metrics from the [system.metrics](/en/operations/system-tables/metrics) table. |
| `asynchronous_metrics` | true | Expose current metrics values from the [system.asynchronous_metrics](/en/operations/system-tables/asynchronous_metrics) table. |
| `events` | true | Expose metrics from the [system.events](/en/operations/system-tables/events) table. |
| `errors` | true | Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](/en/operations/system-tables/errors) as well. |
Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server):
```bash
curl 127.0.0.1:9363/metrics
```
## Remote-write protocol {#remote-write}
ClickHouse supports the [remote-write](https://prometheus.io/docs/specs/remote_write_spec/) protocol.
Data are received by this protocol and written to a [TimeSeries](/en/engines/table-engines/special/time_series) table
(which should be created beforehand).
```xml
<prometheus>
<port>9363</port>
<handlers>
<my_rule_1>
<url>/write</url>
<handler>
<type>remote_write</type
<database>db_name</database>
<table>time_series_table</table>
</handler>
</my_rule_1>
</handlers>
</prometheus>
```
Settings:
| Name | Default | Description |
|---|---|---|---|
| `port` | none | Port for serving the `remote-write` protocol. |
| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [<http_handlers>](/en/interfaces/http) section. |
| `table` | none | The name of a [TimeSeries](/en/engines/table-engines/special/time_series) table to write data received by the `remote-write` protocol. This name can optionally contain the name of a database too. |
| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. |
## Remote-read protocol {#remote-read}
ClickHouse supports the [remote-read](https://prometheus.io/docs/prometheus/latest/querying/remote_read_api/) protocol.
Data are read from a [TimeSeries](/en/engines/table-engines/special/time_series) table and sent via this protocol.
```xml
<prometheus>
<port>9363</port>
<handlers>
<my_rule_1>
<url>/read</url>
<handler>
<type>remote_read</type
<database>db_name</database>
<table>time_series_table</table>
</handler>
</my_rule_1>
</handlers>
</prometheus>
```
Settings:
| Name | Default | Description |
|---|---|---|---|
| `port` | none | Port for serving the `remote-read` protocol. |
| `url` / `headers` / `method` | none | Filters used to find a matching handler for a request. Similar to the fields with the same names in the [<http_handlers>](/en/interfaces/http) section. |
| `table` | none | The name of a [TimeSeries](/en/engines/table-engines/special/time_series) table to read data to send by the `remote-read` protocol. This name can optionally contain the name of a database too. |
| `database` | none | The name of a database where the table specified in the `table` setting is located if it's not specified in the `table` setting. |
## Configuration for multiple protocols {#multiple-protocols}
Multiple protocols can be specified together in one place:
```xml
<prometheus>
<port>9363</port>
<handlers>
<my_rule_1>
<url>/metrics</url>
<handler>
<type>expose_metrics</type>
<metrics>true</metrics>
<asynchronous_metrics>true</asynchronous_metrics>
<events>true</events>
<errors>true</errors>
</handler>
</my_rule_1>
<my_rule_2>
<url>/write</url>
<handler>
<type>remote_write</type
<table>db_name.time_series_table</table>
</handler>
</my_rule_2>
<my_rule_3>
<url>/read</url>
<handler>
<type>remote_read</type
<table>db_name.time_series_table</table>
</handler>
</my_rule_3>
</handlers>
</prometheus>
```

View File

@ -2112,48 +2112,6 @@ The trailing slash is mandatory.
<path>/var/lib/clickhouse/</path>
```
## Prometheus {#prometheus}
:::note
ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com.
:::
Exposing metrics data for scraping from [Prometheus](https://prometheus.io).
Settings:
- `endpoint` HTTP endpoint for scraping metrics by prometheus server. Start from /.
- `port` Port for `endpoint`.
- `metrics` Expose metrics from the [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) table.
- `events` Expose metrics from the [system.events](../../operations/system-tables/events.md#system_tables-events) table.
- `asynchronous_metrics` Expose current metrics values from the [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) table.
- `errors` - Expose the number of errors by error codes occurred since the last server restart. This information could be obtained from the [system.errors](../../operations/system-tables/asynchronous_metrics.md#system_tables-errors) as well.
**Example**
``` xml
<clickhouse>
<listen_host>0.0.0.0</listen_host>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<!-- highlight-start -->
<prometheus>
<endpoint>/metrics</endpoint>
<port>9363</port>
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>true</asynchronous_metrics>
<errors>true</errors>
</prometheus>
<!-- highlight-end -->
</clickhouse>
```
Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server):
```bash
curl 127.0.0.1:9363/metrics
```
## query_log {#query-log}
Setting for logging queries received with the [log_queries=1](../../operations/settings/settings.md) setting.

View File

@ -5626,3 +5626,14 @@ Default value: `False`
Disable all insert and mutations (alter table update / alter table delete / alter table drop partition). Set to true, can make this node focus on reading queries.
Default value: `false`.
## allow_experimental_time_series_table {#allow-experimental-time-series-table}
Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine.
Possible values:
- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled.
- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled.
Default value: `0`.

View File

@ -52,6 +52,48 @@ Result:
└───────────────────────────────┴───────────────┘
```
## LineString
`LineString` is a line stored as an array of points: [Array](array.md)([Point](#point)).
**Example**
Query:
```sql
CREATE TABLE geo_linestring (l LineString) ENGINE = Memory();
INSERT INTO geo_linestring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]);
SELECT l, toTypeName(l) FROM geo_linestring;
```
Result:
``` text
┌─r─────────────────────────────┬─toTypeName(r)─┐
│ [(0,0),(10,0),(10,10),(0,10)] │ LineString │
└───────────────────────────────┴───────────────┘
```
## MultiLineString
`MultiLineString` is multiple lines stored as an array of `LineString`: [Array](array.md)([LineString](#linestring)).
**Example**
Query:
```sql
CREATE TABLE geo_multilinestring (l MultiLineString) ENGINE = Memory();
INSERT INTO geo_multilinestring VALUES([[(0, 0), (10, 0), (10, 10), (0, 10)], [(1, 1), (2, 2), (3, 3)]]);
SELECT l, toTypeName(l) FROM geo_multilinestring;
```
Result:
``` text
┌─l───────────────────────────────────────────────────┬─toTypeName(l)───┐
│ [[(0,0),(10,0),(10,10),(0,10)],[(1,1),(2,2),(3,3)]] │ MultiLineString │
└─────────────────────────────────────────────────────┴─────────────────┘
```
## Polygon
`Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring)). First element of outer array is the outer shape of polygon and all the following elements are holes.

View File

@ -6,11 +6,13 @@ title: "Functions for Working with Polygons"
## WKT
Returns a WKT (Well Known Text) geometric object from various [Geo Data Types](../../data-types/geo.md). Supported WKT objects are:
Returns a WKT (Well Known Text) geometric object from various [Geo Data Types](../../data-types/geo.md). Supported WKT objects are:
- POINT
- POLYGON
- MULTIPOLYGON
- LINESTRING
- MULTILINESTRING
**Syntax**
@ -26,12 +28,16 @@ WKT(geo_data)
- [Ring](../../data-types/geo.md#ring)
- [Polygon](../../data-types/geo.md#polygon)
- [MultiPolygon](../../data-types/geo.md#multipolygon)
- [LineString](../../data-types/geo.md#linestring)
- [MultiLineString](../../data-types/geo.md#multilinestring)
**Returned value**
- WKT geometric object `POINT` is returned for a Point.
- WKT geometric object `POLYGON` is returned for a Polygon
- WKT geometric object `MULTIPOLYGON` is returned for a MultiPolygon.
- WKT geometric object `MULTIPOLYGON` is returned for a MultiPolygon.
- WKT geometric object `LINESTRING` is returned for a LineString.
- WKT geometric object `MULTILINESTRING` is returned for a MultiLineString.
**Examples**
@ -84,7 +90,7 @@ SELECT
### Input parameters
String starting with `MULTIPOLYGON`
String starting with `MULTIPOLYGON`
### Returned value
@ -170,6 +176,34 @@ SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)');
[(1,1),(2,2),(3,3),(1,1)]
```
## readWKTMultiLineString
Parses a Well-Known Text (WKT) representation of a MultiLineString geometry and returns it in the internal ClickHouse format.
### Syntax
```sql
readWKTMultiLineString(wkt_string)
```
### Arguments
- `wkt_string`: The input WKT string representing a MultiLineString geometry.
### Returned value
The function returns a ClickHouse internal representation of the multilinestring geometry.
### Example
```sql
SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))');
```
```response
[[(1,1),(2,2),(3,3)],[(4,4),(5,5),(6,6)]]
```
## readWKTRing
Parses a Well-Known Text (WKT) representation of a Polygon geometry and returns a ring (closed linestring) in the internal ClickHouse format.
@ -219,7 +253,7 @@ UInt8, 0 for false, 1 for true
## polygonsDistanceSpherical
Calculates the minimal distance between two points where one point belongs to the first polygon and the second to another polygon. Spherical means that coordinates are interpreted as coordinates on a pure and ideal sphere, which is not true for the Earth. Using this type of coordinate system speeds up execution, but of course is not precise.
Calculates the minimal distance between two points where one point belongs to the first polygon and the second to another polygon. Spherical means that coordinates are interpreted as coordinates on a pure and ideal sphere, which is not true for the Earth. Using this type of coordinate system speeds up execution, but of course is not precise.
### Example

View File

@ -9,6 +9,7 @@ The following operations with [partitions](/docs/en/engines/table-engines/merget
- [DETACH PARTITION\|PART](#detach-partitionpart) — Moves a partition or part to the `detached` directory and forget it.
- [DROP PARTITION\|PART](#drop-partitionpart) — Deletes a partition or part.
- [DROP DETACHED PARTITION\|PART](#drop-detached-partitionpart) - Delete a part or all parts of a partition from `detached`.
- [FORGET PARTITION](#forget-partition) — Deletes a partition metadata from zookeeper if it's empty.
- [ATTACH PARTITION\|PART](#attach-partitionpart) — Adds a partition or part from the `detached` directory to the table.
- [ATTACH PARTITION FROM](#attach-partition-from) — Copies the data partition from one table to another and adds.
@ -68,7 +69,7 @@ ALTER TABLE mt DROP PART 'all_4_4_0';
## DROP DETACHED PARTITION\|PART
``` sql
ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART partition_expr
ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART ALL|partition_expr
```
Removes the specified part or all parts of the specified partition from `detached`.

View File

@ -186,7 +186,7 @@ Otherwise, you'll get `INVALID_JOIN_ON_EXPRESSION`.
:::
Clickhouse currently supports `ALL INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`.
Clickhouse currently supports `ALL/ANY/SEMI/ANTI INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`.
**Example**

View File

@ -0,0 +1,36 @@
---
slug: /en/sql-reference/table-functions/fuzzQuery
sidebar_position: 75
sidebar_label: fuzzQuery
---
# fuzzQuery
Perturbs the given query string with random variations.
``` sql
fuzzQuery(query[, max_query_length[, random_seed]])
```
**Arguments**
- `query` (String) - The source query to perform the fuzzing on.
- `max_query_length` (UInt64) - A maximum length the query can get during the fuzzing process.
- `random_seed` (UInt64) - A random seed for producing stable results.
**Returned Value**
A table object with a single column containing perturbed query strings.
## Usage Example
``` sql
SELECT * FROM fuzzQuery('SELECT materialize(\'a\' AS key) GROUP BY key') LIMIT 2;
```
```
┌─query──────────────────────────────────────────────────────────┐
1. │ SELECT 'a' AS key GROUP BY key │
2. │ EXPLAIN PIPELINE compact = true SELECT 'a' AS key GROUP BY key │
└────────────────────────────────────────────────────────────────┘
```

View File

@ -0,0 +1,28 @@
---
slug: /en/sql-reference/table-functions/timeSeriesData
sidebar_position: 145
sidebar_label: timeSeriesData
---
# timeSeriesData
`timeSeriesData(db_name.time_series_table)` - Returns the [data](../../engines/table-engines/integrations/time-series.md#data-table) table
used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md):
``` sql
CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA data_table
```
The function also works if the _data_ table is inner:
``` sql
CREATE TABLE db_name.time_series_table ENGINE=TimeSeries DATA INNER UUID '01234567-89ab-cdef-0123-456789abcdef'
```
The following queries are equivalent:
``` sql
SELECT * FROM timeSeriesData(db_name.time_series_table);
SELECT * FROM timeSeriesData('db_name.time_series_table');
SELECT * FROM timeSeriesData('db_name', 'time_series_table');
```

View File

@ -0,0 +1,28 @@
---
slug: /en/sql-reference/table-functions/timeSeriesMetrics
sidebar_position: 145
sidebar_label: timeSeriesMetrics
---
# timeSeriesMetrics
`timeSeriesMetrics(db_name.time_series_table)` - Returns the [metrics](../../engines/table-engines/integrations/time-series.md#metrics-table) table
used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md):
``` sql
CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS metrics_table
```
The function also works if the _metrics_ table is inner:
``` sql
CREATE TABLE db_name.time_series_table ENGINE=TimeSeries METRICS INNER UUID '01234567-89ab-cdef-0123-456789abcdef'
```
The following queries are equivalent:
``` sql
SELECT * FROM timeSeriesMetrics(db_name.time_series_table);
SELECT * FROM timeSeriesMetrics('db_name.time_series_table');
SELECT * FROM timeSeriesMetrics('db_name', 'time_series_table');
```

View File

@ -0,0 +1,28 @@
---
slug: /en/sql-reference/table-functions/timeSeriesTags
sidebar_position: 145
sidebar_label: timeSeriesTags
---
# timeSeriesTags
`timeSeriesTags(db_name.time_series_table)` - Returns the [tags](../../engines/table-engines/integrations/time-series.md#tags-table) table
used by table `db_name.time_series_table` which table engine is [TimeSeries](../../engines/table-engines/integrations/time-series.md):
``` sql
CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS tags_table
```
The function also works if the _tags_ table is inner:
``` sql
CREATE TABLE db_name.time_series_table ENGINE=TimeSeries TAGS INNER UUID '01234567-89ab-cdef-0123-456789abcdef'
```
The following queries are equivalent:
``` sql
SELECT * FROM timeSeriesTags(db_name.time_series_table);
SELECT * FROM timeSeriesTags('db_name.time_series_table');
SELECT * FROM timeSeriesTags('db_name', 'time_series_table');
```

View File

@ -11,7 +11,10 @@ class Client : public ClientApplicationBase
public:
using Arguments = ClientApplicationBase::Arguments;
Client() = default;
Client()
{
fuzzer = QueryFuzzer(randomSeed(), &std::cout, &std::cerr);
}
void initialize(Poco::Util::Application & self) override;

View File

@ -814,10 +814,11 @@ try
const size_t physical_server_memory = getMemoryAmount();
LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.",
LOG_INFO(log, "Available RAM: {}; logical cores: {}; used cores: {}.",
formatReadableSizeWithBinarySuffix(physical_server_memory),
getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores
std::thread::hardware_concurrency());
std::thread::hardware_concurrency(),
getNumberOfPhysicalCPUCores() // on ARM processors it can show only enabled at current moment cores
);
#if defined(__x86_64__)
String cpu_info;
@ -1623,7 +1624,7 @@ try
concurrent_threads_soft_limit = new_server_settings.concurrent_threads_soft_limit_num;
if (new_server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0)
{
auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency();
auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * getNumberOfPhysicalCPUCores();
if (value > 0 && value < concurrent_threads_soft_limit)
concurrent_threads_soft_limit = value;
}

View File

@ -477,6 +477,8 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info)
{
if (profile_info.hasAppliedLimit() && output_format)
output_format->setRowsBeforeLimit(profile_info.getRowsBeforeLimit());
if (profile_info.hasAppliedAggregation() && output_format)
output_format->setRowsBeforeAggregation(profile_info.getRowsBeforeAggregation());
}

View File

@ -2,7 +2,7 @@
#include <Client/Suggest.h>
#include <Client/QueryFuzzer.h>
#include <Common/QueryFuzzer.h>
#include <Common/DNSResolver.h>
#include <Common/InterruptListener.h>
#include <Common/ProgressIndication.h>

View File

@ -1319,7 +1319,7 @@ Progress Connection::receiveProgress() const
ProfileInfo Connection::receiveProfileInfo() const
{
ProfileInfo profile_info;
profile_info.read(*in);
profile_info.read(*in, server_revision);
return profile_info;
}

View File

@ -365,7 +365,7 @@ bool LocalConnection::poll(size_t)
{
while (pollImpl())
{
LOG_DEBUG(&Poco::Logger::get("LocalConnection"), "Executor timeout encountered, will retry");
LOG_TEST(&Poco::Logger::get("LocalConnection"), "Executor timeout encountered, will retry");
if (needSendProgressOrMetrics())
return true;

View File

@ -68,22 +68,21 @@ Field QueryFuzzer::getRandomField(int type)
{
case 0:
{
return bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values)
/ sizeof(*bad_int64_values))];
return bad_int64_values[fuzz_rand() % std::size(bad_int64_values)];
}
case 1:
{
static constexpr double values[]
= {NAN, INFINITY, -INFINITY, 0., -0., 0.0001, 0.5, 0.9999,
1., 1.0001, 2., 10.0001, 100.0001, 1000.0001, 1e10, 1e20,
FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % (sizeof(values) / sizeof(*values))];
FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % std::size(values)];
}
case 2:
{
static constexpr UInt64 scales[] = {0, 1, 2, 10};
return DecimalField<Decimal64>(
bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))],
static_cast<UInt32>(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))])
bad_int64_values[fuzz_rand() % std::size(bad_int64_values)],
static_cast<UInt32>(scales[fuzz_rand() % std::size(scales)])
);
}
default:
@ -165,7 +164,8 @@ Field QueryFuzzer::fuzzField(Field field)
{
size_t pos = fuzz_rand() % arr.size();
arr.erase(arr.begin() + pos);
std::cerr << "erased\n";
if (debug_stream)
*debug_stream << "erased\n";
}
if (fuzz_rand() % 5 == 0)
@ -174,12 +174,14 @@ Field QueryFuzzer::fuzzField(Field field)
{
size_t pos = fuzz_rand() % arr.size();
arr.insert(arr.begin() + pos, fuzzField(arr[pos]));
std::cerr << fmt::format("inserted (pos {})\n", pos);
if (debug_stream)
*debug_stream << fmt::format("inserted (pos {})\n", pos);
}
else
{
arr.insert(arr.begin(), getRandomField(0));
std::cerr << "inserted (0)\n";
if (debug_stream)
*debug_stream << "inserted (0)\n";
}
}
@ -197,7 +199,9 @@ Field QueryFuzzer::fuzzField(Field field)
{
size_t pos = fuzz_rand() % arr.size();
arr.erase(arr.begin() + pos);
std::cerr << "erased\n";
if (debug_stream)
*debug_stream << "erased\n";
}
if (fuzz_rand() % 5 == 0)
@ -206,12 +210,16 @@ Field QueryFuzzer::fuzzField(Field field)
{
size_t pos = fuzz_rand() % arr.size();
arr.insert(arr.begin() + pos, fuzzField(arr[pos]));
std::cerr << fmt::format("inserted (pos {})\n", pos);
if (debug_stream)
*debug_stream << fmt::format("inserted (pos {})\n", pos);
}
else
{
arr.insert(arr.begin(), getRandomField(0));
std::cerr << "inserted (0)\n";
if (debug_stream)
*debug_stream << "inserted (0)\n";
}
}
@ -344,7 +352,8 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast)
}
else
{
std::cerr << "No random column.\n";
if (debug_stream)
*debug_stream << "No random column.\n";
}
}
@ -378,7 +387,8 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast)
if (col)
impl->children.insert(pos, col);
else
std::cerr << "No random column.\n";
if (debug_stream)
*debug_stream << "No random column.\n";
}
// We don't have to recurse here to fuzz the children, this is handled by
@ -1361,11 +1371,15 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast)
collectFuzzInfoMain(ast);
fuzz(ast);
std::cout << std::endl;
WriteBufferFromOStream ast_buf(std::cout, 4096);
formatAST(*ast, ast_buf, false /*highlight*/);
ast_buf.finalize();
std::cout << std::endl << std::endl;
if (out_stream)
{
*out_stream << std::endl;
WriteBufferFromOStream ast_buf(*out_stream, 4096);
formatAST(*ast, ast_buf, false /*highlight*/);
ast_buf.finalize();
*out_stream << std::endl << std::endl;
}
}
}

View File

@ -35,9 +35,31 @@ struct ASTWindowDefinition;
* queries, so you want to feed it a lot of queries to get some interesting mix
* of them. Normally we feed SQL regression tests to it.
*/
struct QueryFuzzer
class QueryFuzzer
{
pcg64 fuzz_rand{randomSeed()};
public:
explicit QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = nullptr, std::ostream * debug_stream_ = nullptr)
: fuzz_rand(fuzz_rand_)
, out_stream(out_stream_)
, debug_stream(debug_stream_)
{
}
// This is the only function you have to call -- it will modify the passed
// ASTPtr to point to new AST with some random changes.
void fuzzMain(ASTPtr & ast);
ASTs getInsertQueriesForFuzzedTables(const String & full_query);
ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query);
void notifyQueryFailed(ASTPtr ast);
static bool isSuitableForFuzzing(const ASTCreateQuery & create);
private:
pcg64 fuzz_rand;
std::ostream * out_stream = nullptr;
std::ostream * debug_stream = nullptr;
// We add elements to expression lists with fixed probability. Some elements
// are so large, that the expected number of elements we add to them is
@ -66,10 +88,6 @@ struct QueryFuzzer
std::unordered_map<std::string, size_t> index_of_fuzzed_table;
std::set<IAST::Hash> created_tables_hashes;
// This is the only function you have to call -- it will modify the passed
// ASTPtr to point to new AST with some random changes.
void fuzzMain(ASTPtr & ast);
// Various helper functions follow, normally you shouldn't have to call them.
Field getRandomField(int type);
Field fuzzField(Field field);
@ -77,9 +95,6 @@ struct QueryFuzzer
ASTPtr getRandomExpressionList();
DataTypePtr fuzzDataType(DataTypePtr type);
DataTypePtr getRandomType();
ASTs getInsertQueriesForFuzzedTables(const String & full_query);
ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query);
void notifyQueryFailed(ASTPtr ast);
void replaceWithColumnLike(ASTPtr & ast);
void replaceWithTableLike(ASTPtr & ast);
void fuzzOrderByElement(ASTOrderByElement * elem);
@ -102,8 +117,6 @@ struct QueryFuzzer
void addTableLike(ASTPtr ast);
void addColumnLike(ASTPtr ast);
void collectFuzzInfoRecurse(ASTPtr ast);
static bool isSuitableForFuzzing(const ASTCreateQuery & create);
};
}

View File

@ -629,6 +629,7 @@ void HandledSignals::setupTerminateHandler()
void HandledSignals::setupCommonDeadlySignalHandlers()
{
/// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime.
/// NOTE: that it is also used by clickhouse-test wrapper
addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, true);
#if defined(SANITIZER)

View File

@ -1,4 +1,5 @@
#pragma once
/// Get number of CPU cores without hyper-threading.
/// The calculation respects possible cgroups limits.
unsigned getNumberOfPhysicalCPUCores();

View File

@ -81,6 +81,8 @@ static constexpr auto DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK = 54467;
static constexpr auto DBMS_MIN_REVISION_WITH_SYSTEM_KEYWORDS_TABLE = 54468;
static constexpr auto DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION = 54469;
/// Version of ClickHouse TCP protocol.
///
/// Should be incremented manually on protocol changes.
@ -88,6 +90,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SYSTEM_KEYWORDS_TABLE = 54468;
/// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION,
/// later is just a number for server version (one number instead of commit SHA)
/// for simplicity (sometimes it may be more convenient in some use cases).
static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54468;
static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54469;
}

View File

@ -1243,6 +1243,7 @@ class IColumn;
M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \
\
M(Bool, exact_rows_before_limit, false, "When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely", 0) \
M(Bool, rows_before_aggregation, false, "When enabled, ClickHouse will provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation", 0) \
M(UInt64, cross_to_inner_join_rewrite, 1, "Use inner join instead of comma/cross join if there are joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible", 0) \
\
M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \

View File

@ -75,6 +75,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
},
{"24.8",
{
{"rows_before_aggregation", true, true, "Provide exact value for rows_before_aggregation statistic, represents the number of rows read before aggregation"},
{"restore_replace_external_table_functions_to_null", false, false, "New setting."},
{"restore_replace_external_engines_to_null", false, false, "New setting."},
{"input_format_json_max_depth", 1000000, 1000, "It was unlimited in previous versions, but that was unsafe."},

View File

@ -24,6 +24,13 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory)
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeLineStringName>()));
});
// Custom type for multiple lines stored as Array(LineString)
factory.registerSimpleDataTypeCustom("MultiLineString", []
{
return std::make_pair(DataTypeFactory::instance().get("Array(LineString)"),
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeMultiLineStringName>()));
});
// Custom type for simple polygon without holes stored as Array(Point)
factory.registerSimpleDataTypeCustom("Ring", []
{

View File

@ -17,6 +17,12 @@ public:
DataTypeLineStringName() : DataTypeCustomFixedName("LineString") {}
};
class DataTypeMultiLineStringName : public DataTypeCustomFixedName
{
public:
DataTypeMultiLineStringName() : DataTypeCustomFixedName("MultiLineString") {}
};
class DataTypeRingName : public DataTypeCustomFixedName
{
public:

150
src/Disks/DiskFomAST.cpp Normal file
View File

@ -0,0 +1,150 @@
#include <Disks/DiskFomAST.h>
#include <Common/assert_cast.h>
#include <Common/filesystemHelpers.h>
#include <Disks/getDiskConfigurationFromAST.h>
#include <Disks/DiskSelector.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/isDiskFunction.h>
#include <Interpreters/Context.h>
#include <Parsers/IAST.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
std::string getOrCreateCustomDisk(DiskConfigurationPtr config, const std::string & serialization, ContextPtr context, bool attach)
{
Poco::Util::AbstractConfiguration::Keys disk_settings_keys;
config->keys(disk_settings_keys);
/// Check that no settings are defined when disk from the config is referred.
if (disk_settings_keys.empty())
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Disk function must have arguments. Invalid disk description.");
if (disk_settings_keys.size() == 1 && disk_settings_keys.front() == "name" && !attach)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Disk function `{}` must have other arguments apart from `name`, which describe disk configuration. Invalid disk description.",
serialization);
auto disk_settings_hash = sipHash128(serialization.data(), serialization.size());
std::string disk_name;
if (config->has("name"))
{
disk_name = config->getString("name");
}
else
{
/// We need a unique name for a created custom disk, but it needs to be the same
/// after table is reattached or server is restarted, so take a hash of the disk
/// configuration serialized ast as a disk name suffix.
disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(disk_settings_hash);
}
auto disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr {
auto result = DiskFactory::instance().create(
disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true);
/// Mark that disk can be used without storage policy.
result->markDiskAsCustom(disk_settings_hash);
return result;
});
if (!disk->isCustomDisk())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Disk `{}` already exists and is described by the config."
" It is impossible to redefine it.",
disk_name);
if (disk->getCustomDiskSettings() != disk_settings_hash && !attach)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"The disk `{}` is already configured as a custom disk in another table. It can't be redefined with different settings.",
disk_name);
if (!attach && !disk->isRemote())
{
static constexpr auto custom_local_disks_base_dir_in_config = "custom_local_disks_base_directory";
auto disk_path_expected_prefix = context->getConfigRef().getString(custom_local_disks_base_dir_in_config, "");
if (disk_path_expected_prefix.empty())
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Base path for custom local disks must be defined in config file by `{}`",
custom_local_disks_base_dir_in_config);
if (!pathStartsWith(disk->getPath(), disk_path_expected_prefix))
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Path of the custom local disk must be inside `{}` directory",
disk_path_expected_prefix);
}
return disk_name;
}
class DiskConfigurationFlattener
{
public:
struct Data
{
ContextPtr context;
bool attach;
};
static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, Data & data)
{
if (isDiskFunction(ast))
{
const auto * function = ast->as<ASTFunction>();
const auto * function_args_expr = assert_cast<const ASTExpressionList *>(function->arguments.get());
const auto & function_args = function_args_expr->children;
auto config = getDiskConfigurationFromAST(function_args, data.context);
auto disk_setting_string = serializeAST(*function);
auto disk_name = getOrCreateCustomDisk(config, disk_setting_string, data.context, data.attach);
ast = std::make_shared<ASTLiteral>(disk_name);
}
}
};
std::string DiskFomAST::createCustomDisk(const ASTPtr & disk_function_ast, ContextPtr context, bool attach)
{
if (!isDiskFunction(disk_function_ast))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected a disk function");
auto ast = disk_function_ast->clone();
using FlattenDiskConfigurationVisitor = InDepthNodeVisitor<DiskConfigurationFlattener, false>;
FlattenDiskConfigurationVisitor::Data data{context, attach};
FlattenDiskConfigurationVisitor{data}.visit(ast);
return assert_cast<const ASTLiteral &>(*ast).value.get<String>();
}
void DiskFomAST::ensureDiskIsNotCustom(const std::string & disk_name, ContextPtr context)
{
auto disk = context->getDisk(disk_name);
if (disk->isCustomDisk())
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Disk name `{}` is a custom disk that is used in other table. "
"That disk could not be used by a reference by other tables. The custom disk should be fully specified with a disk function.",
disk_name);
}
}

15
src/Disks/DiskFomAST.h Normal file
View File

@ -0,0 +1,15 @@
#pragma once
#include <string>
#include <Interpreters/Context_fwd.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
namespace DiskFomAST
{
void ensureDiskIsNotCustom(const std::string & name, ContextPtr context);
std::string createCustomDisk(const ASTPtr & disk_function, ContextPtr context, bool attach);
}
}

View File

@ -6,6 +6,8 @@
#include <Poco/Util/AbstractConfiguration.h>
#include <map>
#include <sstream>
#include <string_view>
namespace DB
{

View File

@ -464,9 +464,9 @@ public:
virtual void chmod(const String & /*path*/, mode_t /*mode*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk does not support chmod"); }
/// Was disk created to be used without storage configuration?
bool isCustomDisk() const { return is_custom_disk; }
void markDiskAsCustom() { is_custom_disk = true; }
bool isCustomDisk() const { return custom_disk_settings_hash != 0; }
UInt128 getCustomDiskSettings() const { return custom_disk_settings_hash; }
void markDiskAsCustom(UInt128 settings_hash) { custom_disk_settings_hash = settings_hash; }
virtual DiskPtr getDelegateDiskIfExists() const { return nullptr; }
@ -504,7 +504,8 @@ protected:
private:
ThreadPool copying_thread_pool;
bool is_custom_disk = false;
// 0 means the disk is not custom, the disk is predefined in the config
UInt128 custom_disk_settings_hash = 0;
/// Check access to the disk.
void checkAccess();

View File

@ -12,7 +12,6 @@
#include <Common/formatReadable.h>
#include <memory>
#include <mutex>
#include <unordered_map>
#include <unistd.h>
#include <boost/noncopyable.hpp>

View File

@ -1,121 +0,0 @@
#include <Disks/getOrCreateDiskFromAST.h>
#include <Common/logger_useful.h>
#include <Common/assert_cast.h>
#include <Common/filesystemHelpers.h>
#include <Disks/getDiskConfigurationFromAST.h>
#include <Disks/DiskSelector.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/isDiskFunction.h>
#include <Interpreters/Context.h>
#include <Parsers/IAST.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
namespace
{
std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context, bool attach)
{
const auto * function_args_expr = assert_cast<const ASTExpressionList *>(function.arguments.get());
const auto & function_args = function_args_expr->children;
auto config = getDiskConfigurationFromAST(function_args, context);
std::string disk_name;
if (config->has("name"))
{
disk_name = config->getString("name");
}
else
{
/// We need a unique name for a created custom disk, but it needs to be the same
/// after table is reattached or server is restarted, so take a hash of the disk
/// configuration serialized ast as a disk name suffix.
auto disk_setting_string = serializeAST(function);
disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX
+ toString(sipHash128(disk_setting_string.data(), disk_setting_string.size()));
}
auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr {
auto disk = DiskFactory::instance().create(
disk_name, *config, /* config_path */"", context, disks_map, /* attach */attach, /* custom_disk */true);
/// Mark that disk can be used without storage policy.
disk->markDiskAsCustom();
return disk;
});
if (!result_disk->isCustomDisk())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk with name `{}` already exist", disk_name);
if (!attach && !result_disk->isRemote())
{
static constexpr auto custom_local_disks_base_dir_in_config = "custom_local_disks_base_directory";
auto disk_path_expected_prefix = context->getConfigRef().getString(custom_local_disks_base_dir_in_config, "");
if (disk_path_expected_prefix.empty())
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Base path for custom local disks must be defined in config file by `{}`",
custom_local_disks_base_dir_in_config);
if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix))
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Path of the custom local disk must be inside `{}` directory",
disk_path_expected_prefix);
}
return disk_name;
}
class DiskConfigurationFlattener
{
public:
struct Data
{
ContextPtr context;
bool attach;
};
static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, Data & data)
{
if (isDiskFunction(ast))
{
auto disk_name = getOrCreateDiskFromDiskAST(*ast->as<ASTFunction>(), data.context, data.attach);
ast = std::make_shared<ASTLiteral>(disk_name);
}
}
};
/// Visits children first.
using FlattenDiskConfigurationVisitor = InDepthNodeVisitor<DiskConfigurationFlattener, false>;
}
std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr context, bool attach)
{
if (!isDiskFunction(disk_function))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected a disk function");
auto ast = disk_function->clone();
FlattenDiskConfigurationVisitor::Data data{context, attach};
FlattenDiskConfigurationVisitor{data}.visit(ast);
auto disk_name = assert_cast<const ASTLiteral &>(*ast).value.get<String>();
LOG_TRACE(getLogger("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name);
return disk_name;
}
}

View File

@ -1,18 +0,0 @@
#pragma once
#include <string>
#include <Interpreters/Context_fwd.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class ASTFunction;
/**
* Create a DiskPtr from disk AST function like disk(<disk_configuration>),
* add it to DiskSelector by a unique (but always the same for given configuration) disk name
* and return this name.
*/
std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr context, bool attach);
}

View File

@ -487,6 +487,8 @@ namespace JSONUtils
size_t rows,
size_t rows_before_limit,
bool applied_limit,
size_t rows_before_aggregation,
bool applied_aggregation,
const Stopwatch & watch,
const Progress & progress,
bool write_statistics,
@ -502,7 +504,12 @@ namespace JSONUtils
writeTitle("rows_before_limit_at_least", out, 1, " ");
writeIntText(rows_before_limit, out);
}
if (applied_aggregation)
{
writeFieldDelimiter(out, 2);
writeTitle("rows_before_aggregation", out, 1, " ");
writeIntText(rows_before_aggregation, out);
}
if (write_statistics)
{
writeFieldDelimiter(out, 2);

View File

@ -104,6 +104,8 @@ namespace JSONUtils
size_t rows,
size_t rows_before_limit,
bool applied_limit,
size_t rows_before_aggregation,
bool applied_aggregation,
const Stopwatch & watch,
const Progress & progress,
bool write_statistics,

View File

@ -31,6 +31,9 @@ namespace ErrorCodes
template <typename Point>
using LineString = boost::geometry::model::linestring<Point>;
template <typename Point>
using MultiLineString = boost::geometry::model::multi_linestring<LineString<Point>>;
template <typename Point>
using Ring = boost::geometry::model::ring<Point>;
@ -42,12 +45,14 @@ using MultiPolygon = boost::geometry::model::multi_polygon<Polygon<Point>>;
using CartesianPoint = boost::geometry::model::d2::point_xy<Float64>;
using CartesianLineString = LineString<CartesianPoint>;
using CartesianMultiLineString = MultiLineString<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 SphericalMultiLineString = MultiLineString<SphericalPoint>;
using SphericalRing = Ring<SphericalPoint>;
using SphericalPolygon = Polygon<SphericalPoint>;
using SphericalMultiPolygon = MultiPolygon<SphericalPoint>;
@ -113,6 +118,28 @@ struct ColumnToLineStringsConverter
}
};
/**
* Class which converts Column with type Array(Array(Tuple(Float64, Float64))) to a vector of boost multi_linestring type.
*/
template <typename Point>
struct ColumnToMultiLineStringsConverter
{
static std::vector<MultiLineString<Point>> convert(ColumnPtr col)
{
const IColumn::Offsets & offsets = typeid_cast<const ColumnArray &>(*col).getOffsets();
size_t prev_offset = 0;
std::vector<MultiLineString<Point>> answer(offsets.size());
auto all_linestrings = ColumnToLineStringsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
for (size_t iter = 0; iter < offsets.size() && iter < all_linestrings.size(); ++iter)
{
for (size_t linestring_iter = prev_offset; linestring_iter < offsets[iter]; ++linestring_iter)
answer[iter].emplace_back(std::move(all_linestrings[linestring_iter]));
prev_offset = offsets[iter];
}
return answer;
}
};
/**
* Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost ring type.
*/
@ -268,6 +295,38 @@ private:
ColumnUInt64::MutablePtr offsets;
};
/// Serialize Point, MultiLineString as MultiLineString
template <typename Point>
class MultiLineStringSerializer
{
public:
MultiLineStringSerializer()
: offsets(ColumnUInt64::create())
{}
explicit MultiLineStringSerializer(size_t n)
: offsets(ColumnUInt64::create(n))
{}
void add(const MultiLineString<Point> & multilinestring)
{
size += multilinestring.size();
offsets->insertValue(size);
for (const auto & linestring : multilinestring)
linestring_serializer.add(linestring);
}
ColumnPtr finalize()
{
return ColumnArray::create(linestring_serializer.finalize(), std::move(offsets));
}
private:
size_t size = 0;
LineStringSerializer<Point> linestring_serializer;
ColumnUInt64::MutablePtr offsets;
};
/// Almost the same as LineStringSerializer
/// Serialize Point, Ring as Ring
template <typename Point>
@ -411,6 +470,11 @@ static void callOnGeometryDataType(DataTypePtr type, F && f)
else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "LineString")
return f(ConverterType<ColumnToLineStringsConverter<Point>>());
/// We should take the name into consideration to avoid ambiguity.
/// Because for example both MultiLineString and Polygon are resolved to Array(Array(Point)).
else if (factory.get("MultiLineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "MultiLineString")
return f(ConverterType<ColumnToMultiLineStringsConverter<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>>());

View File

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

View File

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

View File

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

View File

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

View File

@ -87,6 +87,11 @@ struct ReadWKTLineStringNameHolder
static constexpr const char * name = "readWKTLineString";
};
struct ReadWKTMultiLineStringNameHolder
{
static constexpr const char * name = "readWKTMultiLineString";
};
struct ReadWKTRingNameHolder
{
static constexpr const char * name = "readWKTRing";
@ -131,6 +136,31 @@ Parses a Well-Known Text (WKT) representation of a LineString geometry and retur
},
.categories{"Unique identifiers"}
});
factory.registerFunction<FunctionReadWKT<DataTypeMultiLineStringName, CartesianMultiLineString, MultiLineStringSerializer<CartesianPoint>, ReadWKTMultiLineStringNameHolder>>(FunctionDocumentation
{
.description=R"(
Parses a Well-Known Text (WKT) representation of a MultiLineString geometry and returns it in the internal ClickHouse format.
)",
.syntax = "readWKTMultiLineString(wkt_string)",
.arguments{
{"wkt_string", "The input WKT string representing a MultiLineString geometry."}
},
.returned_value = "The function returns a ClickHouse internal representation of the multilinestring geometry.",
.examples{
{"first call", "SELECT readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))');", R"(
readWKTMultiLineString('MULTILINESTRING ((1 1, 2 2, 3 3), (4 4, 5 5, 6 6))')
[[(1,1),(2,2),(3,3)],[(4,4),(5,5),(6,6)]]
)"},
{"second call", "SELECT toTypeName(readWKTLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'));", R"(
toTypeName(readWKTLineString('MULTILINESTRING ((1 1, 2 2, 3 3, 1 1))'))
MultiLineString
)"},
},
.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

@ -153,12 +153,8 @@ public:
return true;
}
std::vector<double> xfreq(spec_len);
double step = 0.5 / (spec_len - 1);
for (size_t i = 0; i < spec_len; ++i)
xfreq[i] = i * step;
auto freq = xfreq[idx];
auto freq = idx * step;
period = std::round(1 / freq);
return true;

View File

@ -1,4 +1,4 @@
#include <Interpreters/HashJoin/HashJoinMethods.h>
#include <Interpreters/HashJoin/HashJoinMethodsImpl.h>
namespace DB
{

View File

@ -291,12 +291,13 @@ void HashJoin::dataMapInit(MapsVariant & map)
{
if (kind == JoinKind::Cross)
return;
joinDispatchInit(kind, strictness, map);
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); });
auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
joinDispatchInit(kind, strictness, map, prefer_use_maps_all);
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { map_.create(data->type); });
if (reserve_num)
{
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); });
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); });
}
if (!data)
@ -327,9 +328,10 @@ size_t HashJoin::getTotalRowCount() const
}
else
{
auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
for (const auto & map : data->maps)
{
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); });
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); });
}
}
@ -367,9 +369,10 @@ size_t HashJoin::getTotalByteCount() const
if (data->type != Type::CROSS)
{
auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
for (const auto & map : data->maps)
{
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); });
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); });
}
}
return res;
@ -520,6 +523,8 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
return true;
}
bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
size_t total_rows = 0;
size_t total_bytes = 0;
{
@ -592,7 +597,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
bool is_inserted = false;
if (kind != JoinKind::Cross)
{
joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map)
joinDispatch(kind, strictness, data->maps[onexpr_idx], prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & map)
{
size_t size = HashJoinMethods<kind_, strictness_, std::decay_t<decltype(map)>>::insertFromBlockImpl(
*this,
@ -608,10 +613,10 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
is_inserted);
if (flag_per_row)
used_flags->reinit<kind_, strictness_>(stored_block);
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map)>, MapsAll>>(stored_block);
else if (is_inserted)
/// Number of buckets + 1 value from zero storage
used_flags->reinit<kind_, strictness_>(size + 1);
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map)>, MapsAll>>(size + 1);
});
}
@ -873,7 +878,7 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block
keys.insert(std::move(key));
}
static_assert(!MapGetter<JoinKind::Left, JoinStrictness::Any>::flagged,
static_assert(!MapGetter<JoinKind::Left, JoinStrictness::Any, false>::flagged,
"joinGet are not protected from hash table changes between block processing");
std::vector<const MapsOne *> maps_vector;
@ -914,16 +919,34 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
materializeBlockInplace(block);
}
bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
{
std::vector<const std::decay_t<decltype(data->maps[0])> * > maps_vector;
for (size_t i = 0; i < table_join->getClauses().size(); ++i)
maps_vector.push_back(&data->maps[i]);
if (joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_)
if (joinDispatch(kind, strictness, maps_vector, prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & maps_vector_)
{
using MapType = typename MapGetter<kind_, strictness_>::Map;
Block remaining_block = HashJoinMethods<kind_, strictness_, MapType>::joinBlockImpl(
*this, block, sample_block_with_columns_to_add, maps_vector_);
Block remaining_block;
if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsAll *>>)
{
remaining_block = HashJoinMethods<kind_, strictness_, MapsAll>::joinBlockImpl(
*this, block, sample_block_with_columns_to_add, maps_vector_);
}
else if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsOne *>>)
{
remaining_block = HashJoinMethods<kind_, strictness_, MapsOne>::joinBlockImpl(
*this, block, sample_block_with_columns_to_add, maps_vector_);
}
else if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsAsof *>>)
{
remaining_block = HashJoinMethods<kind_, strictness_, MapsAsof>::joinBlockImpl(
*this, block, sample_block_with_columns_to_add, maps_vector_);
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown maps type");
}
if (remaining_block.rows())
not_processed = std::make_shared<ExtraBlock>(ExtraBlock{std::move(remaining_block)});
else
@ -1023,7 +1046,8 @@ public:
rows_added = fillColumnsFromMap(map, columns_right);
};
if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), fill_callback))
bool prefer_use_maps_all = parent.table_join->getMixedJoinExpression() != nullptr;
if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), prefer_use_maps_all, fill_callback))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness);
}
@ -1220,11 +1244,12 @@ void HashJoin::reuseJoinedData(const HashJoin & join)
if (flag_per_row)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin with ORs is not supported");
bool prefer_use_maps_all = join.table_join->getMixedJoinExpression() != nullptr;
for (auto & map : data->maps)
{
joinDispatch(kind, strictness, map, [this](auto kind_, auto strictness_, auto & map_)
joinDispatch(kind, strictness, map, prefer_use_maps_all, [this](auto kind_, auto strictness_, auto & map_)
{
used_flags->reinit<kind_, strictness_>(map_.getBufferSizeInCells(data->type) + 1);
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map_)>, MapsAll>>(map_.getBufferSizeInCells(data->type) + 1);
});
}
}
@ -1304,7 +1329,9 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona
additional_filter_expression->dumpActions());
}
bool is_supported = (strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind));
bool is_supported = ((strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind)))
|| ((strictness == JoinStrictness::Semi || strictness == JoinStrictness::Any || strictness == JoinStrictness::Anti)
&& (isLeft(kind) || isRight(kind))) || (strictness == JoinStrictness::Any && (isInner(kind)));
if (!is_supported)
{
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,

View File

@ -12,15 +12,8 @@
#include <Poco/Logger.h>
#include <Common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_JOIN_KEYS;
extern const int LOGICAL_ERROR;
}
/// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN.
template <typename HashMap, typename KeyGetter>
struct Inserter
@ -64,7 +57,6 @@ struct Inserter
}
};
/// MapsTemplate is one of MapsOne, MapsAll and MapsAsof
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
class HashJoinMethods
@ -81,27 +73,7 @@ public:
ConstNullMapPtr null_map,
UInt8ColumnDataPtr join_mask,
Arena & pool,
bool & is_inserted)
{
switch (type)
{
case HashJoin::Type::EMPTY:
[[fallthrough]];
case HashJoin::Type::CROSS:
/// Do nothing. We will only save block, and it is enough
is_inserted = true;
return 0;
#define M(TYPE) \
case HashJoin::Type::TYPE: \
return insertFromBlockImplTypeCase<typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>(\
join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \
break;
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
}
}
bool & is_inserted);
using MapsTemplateVector = std::vector<const MapsTemplate *>;
@ -110,280 +82,36 @@ public:
Block & block,
const Block & block_with_columns_to_add,
const MapsTemplateVector & maps_,
bool is_join_get = false)
{
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
std::vector<JoinOnKeyColumns> join_on_keys;
const auto & onexprs = join.table_join->getClauses();
for (size_t i = 0; i < onexprs.size(); ++i)
{
const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right;
join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]);
}
size_t existing_columns = block.columns();
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
* Because if they are constants, then in the "not joined" rows, they may have different values
* - default values, which can differ from the values of these constants.
*/
if constexpr (join_features.right || join_features.full)
{
materializeBlockInplace(block);
}
/** For LEFT/INNER JOIN, the saved blocks do not contain keys.
* For FULL/RIGHT JOIN, the saved blocks contain keys;
* but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped.
* For ASOF, the last column is used as the ASOF column
*/
AddedColumns<!join_features.is_any_join> added_columns(
block,
block_with_columns_to_add,
join.savedBlockSample(),
join,
std::move(join_on_keys),
join.table_join->getMixedJoinExpression(),
join_features.is_asof_join,
is_join_get);
bool has_required_right_keys = (join.required_right_keys.columns() != 0);
added_columns.need_filter = join_features.need_filter || has_required_right_keys;
added_columns.max_joined_block_rows = join.max_joined_block_rows;
if (!added_columns.max_joined_block_rows)
added_columns.max_joined_block_rows = std::numeric_limits<size_t>::max();
else
added_columns.reserve(join_features.need_replication);
size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags);
/// Do not hold memory for join_on_keys anymore
added_columns.join_on_keys.clear();
Block remaining_block = sliceBlock(block, num_joined);
added_columns.buildOutput();
for (size_t i = 0; i < added_columns.size(); ++i)
block.insert(added_columns.moveColumn(i));
std::vector<size_t> right_keys_to_replicate [[maybe_unused]];
if constexpr (join_features.need_filter)
{
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
for (size_t i = 0; i < existing_columns; ++i)
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1);
/// Add join key columns from right block if needed using value from left table because of equality
for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
{
const auto & right_key = join.required_right_keys.getByPosition(i);
/// asof column is already in block.
if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back())
continue;
const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column);
block.insert(std::move(right_col));
}
}
else if (has_required_right_keys)
{
/// Add join key columns from right block if needed.
for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
{
const auto & right_key = join.required_right_keys.getByPosition(i);
auto right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
/// asof column is already in block.
if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back())
continue;
const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter);
block.insert(std::move(right_col));
if constexpr (join_features.need_replication)
right_keys_to_replicate.push_back(block.getPositionByName(right_col_name));
}
}
if constexpr (join_features.need_replication)
{
std::unique_ptr<IColumn::Offsets> & offsets_to_replicate = added_columns.offsets_to_replicate;
/// If ALL ... JOIN - we replicate all the columns except the new ones.
for (size_t i = 0; i < existing_columns; ++i)
{
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate);
}
/// Replicate additional right keys
for (size_t pos : right_keys_to_replicate)
{
block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
}
}
return remaining_block;
}
bool is_join_get = false);
private:
template <typename KeyGetter, bool is_asof_join>
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes)
{
if constexpr (is_asof_join)
{
auto key_column_copy = key_columns;
auto key_size_copy = key_sizes;
key_column_copy.pop_back();
key_size_copy.pop_back();
return KeyGetter(key_column_copy, key_size_copy, nullptr);
}
else
return KeyGetter(key_columns, key_sizes, nullptr);
}
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes);
template <typename KeyGetter, typename HashMap>
static size_t NO_INLINE insertFromBlockImplTypeCase(
static size_t insertFromBlockImplTypeCase(
HashJoin & join, HashMap & map, size_t rows, const ColumnRawPtrs & key_columns,
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted)
{
[[maybe_unused]] constexpr bool mapped_one = std::is_same_v<typename HashMap::mapped_type, RowRef>;
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
const IColumn * asof_column [[maybe_unused]] = nullptr;
if constexpr (is_asof_join)
asof_column = key_columns.back();
auto key_getter = createKeyGetter<KeyGetter, is_asof_join>(key_columns, key_sizes);
/// For ALL and ASOF join always insert values
is_inserted = !mapped_one || is_asof_join;
for (size_t i = 0; i < rows; ++i)
{
if (null_map && (*null_map)[i])
{
/// nulls are not inserted into hash table,
/// keep them for RIGHT and FULL joins
is_inserted = true;
continue;
}
/// Check condition for right table from ON section
if (join_mask && !(*join_mask)[i])
continue;
if constexpr (is_asof_join)
Inserter<HashMap, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column);
else if constexpr (mapped_one)
is_inserted |= Inserter<HashMap, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool);
else
Inserter<HashMap, KeyGetter>::insertAll(join, map, key_getter, stored_block, i, pool);
}
return map.getBufferSizeInCells();
}
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted);
template <typename AddedColumns>
static size_t switchJoinRightColumns(
const std::vector<const MapsTemplate *> & mapv,
AddedColumns & added_columns,
HashJoin::Type type,
JoinStuff::JoinUsedFlags & used_flags)
{
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
switch (type)
{
case HashJoin::Type::EMPTY: {
if constexpr (!is_asof_join)
{
using KeyGetter = KeyGetterEmpty<typename MapsTemplate::MappedType>;
std::vector<KeyGetter> key_getter_vector;
key_getter_vector.emplace_back();
using MapTypeVal = typename KeyGetter::MappedType;
std::vector<const MapTypeVal *> a_map_type_vector;
a_map_type_vector.emplace_back();
return joinRightColumnsSwitchNullability<KeyGetter>(
std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags);
}
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys. Type: {}", type);
}
#define M(TYPE) \
case HashJoin::Type::TYPE: \
{ \
using MapTypeVal = const typename std::remove_reference_t<decltype(MapsTemplate::TYPE)>::element_type; \
using KeyGetter = typename KeyGetterForType<HashJoin::Type::TYPE, MapTypeVal>::Type; \
std::vector<const MapTypeVal *> a_map_type_vector(mapv.size()); \
std::vector<KeyGetter> key_getter_vector; \
for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \
{ \
const auto & join_on_key = added_columns.join_on_keys[d]; \
a_map_type_vector[d] = mapv[d]->TYPE.get(); \
key_getter_vector.push_back(std::move(createKeyGetter<KeyGetter, is_asof_join>(join_on_key.key_columns, join_on_key.key_sizes))); \
} \
return joinRightColumnsSwitchNullability<KeyGetter>( \
std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \
}
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
default:
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type);
}
}
JoinStuff::JoinUsedFlags & used_flags);
template <typename KeyGetter, typename Map, typename AddedColumns>
static size_t joinRightColumnsSwitchNullability(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags)
{
if (added_columns.need_filter)
{
return joinRightColumnsSwitchMultipleDisjuncts<KeyGetter, Map, true>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
else
{
return joinRightColumnsSwitchMultipleDisjuncts<KeyGetter, Map, false>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
}
JoinStuff::JoinUsedFlags & used_flags);
template <typename KeyGetter, typename Map, bool need_filter, typename AddedColumns>
static size_t joinRightColumnsSwitchMultipleDisjuncts(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags)
{
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
if constexpr (join_features.is_all_join)
{
if (added_columns.additional_filter_expression)
{
bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1;
return joinRightColumnsWithAddtitionalFilter<KeyGetter, Map, join_features.need_replication>(
std::forward<std::vector<KeyGetter>>(key_getter_vector),
mapv,
added_columns,
used_flags,
need_filter,
join_features.need_flags,
join_features.add_missing,
mark_per_row_used);
}
}
if (added_columns.additional_filter_expression)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN");
return mapv.size() > 1 ? joinRightColumns<KeyGetter, Map, need_filter, true>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags)
: joinRightColumns<KeyGetter, Map, need_filter, false>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
JoinStuff::JoinUsedFlags & used_flags);
/// Joins right table columns which indexes are present in right_indexes using specified map.
/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
@ -392,464 +120,30 @@ private:
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags)
{
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
size_t rows = added_columns.rows_to_add;
if constexpr (need_filter)
added_columns.filter = IColumn::Filter(rows, 0);
Arena pool;
if constexpr (join_features.need_replication)
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
IColumn::Offset current_offset = 0;
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
size_t i = 0;
for (; i < rows; ++i)
{
if constexpr (join_features.need_replication)
{
if (unlikely(current_offset >= max_joined_block_rows))
{
added_columns.offsets_to_replicate->resize_assume_reserved(i);
added_columns.filter.resize_assume_reserved(i);
break;
}
}
bool right_row_found = false;
KnownRowsHolder<flag_per_row> known_rows;
for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx)
{
const auto & join_keys = added_columns.join_on_keys[onexpr_idx];
if (join_keys.null_map && (*join_keys.null_map)[i])
continue;
bool row_acceptable = !join_keys.isRowFiltered(i);
using FindResult = typename KeyGetter::FindResult;
auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult();
if (find_result.isFound())
{
right_row_found = true;
auto & mapped = find_result.getMapped();
if constexpr (join_features.is_asof_join)
{
const IColumn & left_asof_key = added_columns.leftAsofKey();
auto row_ref = mapped->findAsof(left_asof_key, i);
if (row_ref.block)
{
setUsed<need_filter>(added_columns.filter, i);
if constexpr (flag_per_row)
used_flags.template setUsed<join_features.need_flags, flag_per_row>(row_ref.block, row_ref.row_num, 0);
else
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing);
}
else
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
}
else if constexpr (join_features.is_all_join)
{
setUsed<need_filter>(added_columns.filter, i);
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
addFoundRowAll<Map, join_features.add_missing>(mapped, added_columns, current_offset, known_rows, used_flags_opt);
}
else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right)
{
/// Use first appeared left key + it needs left columns replication
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
if (used_once)
{
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
setUsed<need_filter>(added_columns.filter, i);
addFoundRowAll<Map, join_features.add_missing>(
mapped, added_columns, current_offset, known_rows, used_flags_opt);
}
}
else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner)
{
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
/// Use first appeared left key only
if (used_once)
{
setUsed<need_filter>(added_columns.filter, i);
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
}
break;
}
else if constexpr (join_features.is_any_join && join_features.full)
{
/// TODO
}
else if constexpr (join_features.is_anti_join)
{
if constexpr (join_features.right && join_features.need_flags)
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
}
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
{
setUsed<need_filter>(added_columns.filter, i);
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
if (join_features.is_any_or_semi_join)
{
break;
}
}
}
}
if (!right_row_found)
{
if constexpr (join_features.is_anti_join && join_features.left)
setUsed<need_filter>(added_columns.filter, i);
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
}
if constexpr (join_features.need_replication)
{
(*added_columns.offsets_to_replicate)[i] = current_offset;
}
}
added_columns.applyLazyDefaults();
return i;
}
JoinStuff::JoinUsedFlags & used_flags);
template <bool need_filter>
static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]])
{
if constexpr (need_filter)
filter[pos] = 1;
}
static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]);
template <typename AddedColumns>
static ColumnPtr buildAdditionalFilter(
size_t left_start_row,
const std::vector<RowRef> & selected_rows,
const std::vector<size_t> & row_replicate_offset,
AddedColumns & added_columns)
{
ColumnPtr result_column;
do
{
if (selected_rows.empty())
{
result_column = ColumnUInt8::create();
break;
}
const Block & sample_right_block = *selected_rows.begin()->block;
if (!sample_right_block || !added_columns.additional_filter_expression)
{
auto filter = ColumnUInt8::create();
filter->insertMany(1, selected_rows.size());
result_column = std::move(filter);
break;
}
auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes();
if (required_cols.empty())
{
Block block;
added_columns.additional_filter_expression->execute(block);
result_column = block.getByPosition(0).column->cloneResized(selected_rows.size());
break;
}
NameSet required_column_names;
for (auto & col : required_cols)
required_column_names.insert(col.name);
Block executed_block;
size_t right_col_pos = 0;
for (const auto & col : sample_right_block.getColumnsWithTypeAndName())
{
if (required_column_names.contains(col.name))
{
auto new_col = col.column->cloneEmpty();
for (const auto & selected_row : selected_rows)
{
const auto & src_col = selected_row.block->getByPosition(right_col_pos);
new_col->insertFrom(*src_col.column, selected_row.row_num);
}
executed_block.insert({std::move(new_col), col.type, col.name});
}
right_col_pos += 1;
}
if (!executed_block)
{
result_column = ColumnUInt8::create();
break;
}
for (const auto & col_name : required_column_names)
{
const auto * src_col = added_columns.left_block.findByName(col_name);
if (!src_col)
continue;
auto new_col = src_col->column->cloneEmpty();
size_t prev_left_offset = 0;
for (size_t i = 1; i < row_replicate_offset.size(); ++i)
{
const size_t & left_offset = row_replicate_offset[i];
size_t rows = left_offset - prev_left_offset;
if (rows)
new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows);
prev_left_offset = left_offset;
}
executed_block.insert({std::move(new_col), src_col->type, col_name});
}
if (!executed_block)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"required columns: [{}], but not found any in left/right table. right table: {}, left table: {}",
required_cols.toString(),
sample_right_block.dumpNames(),
added_columns.left_block.dumpNames());
}
for (const auto & col : executed_block.getColumnsWithTypeAndName())
if (!col.column || !col.type)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure());
added_columns.additional_filter_expression->execute(executed_block);
result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst();
executed_block.clear();
} while (false);
result_column = result_column->convertToFullIfNeeded();
if (result_column->isNullable())
{
/// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros
/// Trying to avoid copying data, since we are the only owner of the column.
ColumnPtr mask_column = assert_cast<const ColumnNullable &>(*result_column).getNullMapColumnPtr();
MutableColumnPtr mutable_column;
{
ColumnPtr nested_column = assert_cast<const ColumnNullable &>(*result_column).getNestedColumnPtr();
result_column.reset();
mutable_column = IColumn::mutate(std::move(nested_column));
}
auto & column_data = assert_cast<ColumnUInt8 &>(*mutable_column).getData();
const auto & mask_column_data = assert_cast<const ColumnUInt8 &>(*mask_column).getData();
for (size_t i = 0; i < column_data.size(); ++i)
{
if (mask_column_data[i])
column_data[i] = 0;
}
return mutable_column;
}
return result_column;
}
AddedColumns & added_columns);
/// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression.
template <typename KeyGetter, typename Map, bool need_replication, typename AddedColumns>
template <typename KeyGetter, typename Map, typename AddedColumns>
static size_t joinRightColumnsWithAddtitionalFilter(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]],
bool need_filter [[maybe_unused]],
bool need_flags [[maybe_unused]],
bool add_missing [[maybe_unused]],
bool flag_per_row [[maybe_unused]])
{
size_t left_block_rows = added_columns.rows_to_add;
if (need_filter)
added_columns.filter = IColumn::Filter(left_block_rows, 0);
std::unique_ptr<Arena> pool;
if constexpr (need_replication)
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(left_block_rows);
std::vector<size_t> row_replicate_offset;
row_replicate_offset.reserve(left_block_rows);
using FindResult = typename KeyGetter::FindResult;
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
size_t left_row_iter = 0;
PreSelectedRows selected_rows;
selected_rows.reserve(left_block_rows);
std::vector<FindResult> find_results;
find_results.reserve(left_block_rows);
bool exceeded_max_block_rows = false;
IColumn::Offset total_added_rows = 0;
IColumn::Offset current_added_rows = 0;
auto collect_keys_matched_rows_refs = [&]()
{
pool = std::make_unique<Arena>();
find_results.clear();
row_replicate_offset.clear();
row_replicate_offset.push_back(0);
current_added_rows = 0;
selected_rows.clear();
for (; left_row_iter < left_block_rows; ++left_row_iter)
{
if constexpr (need_replication)
{
if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows))
{
break;
}
}
KnownRowsHolder<true> all_flag_known_rows;
KnownRowsHolder<false> single_flag_know_rows;
for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx)
{
const auto & join_keys = added_columns.join_on_keys[join_clause_idx];
if (join_keys.null_map && (*join_keys.null_map)[left_row_iter])
continue;
bool row_acceptable = !join_keys.isRowFiltered(left_row_iter);
auto find_result = row_acceptable
? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool)
: FindResult();
if (find_result.isFound())
{
auto & mapped = find_result.getMapped();
find_results.push_back(find_result);
if (flag_per_row)
addFoundRowAll<Map, false, true>(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr);
else
addFoundRowAll<Map, false, false>(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr);
}
}
row_replicate_offset.push_back(current_added_rows);
}
};
auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col)
{
const PaddedPODArray<UInt8> & filter_flags = assert_cast<const ColumnUInt8 &>(*filter_col).getData();
size_t prev_replicated_row = 0;
auto selected_right_row_it = selected_rows.begin();
size_t find_result_index = 0;
for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i)
{
bool any_matched = false;
/// For all right join, flag_per_row is true, we need mark used flags for each row.
if (flag_per_row)
{
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
{
if (filter_flags[replicated_row])
{
any_matched = true;
added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing);
total_added_rows += 1;
if (need_flags)
used_flags.template setUsed<true, true>(selected_right_row_it->block, selected_right_row_it->row_num, 0);
}
++selected_right_row_it;
}
}
else
{
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
{
if (filter_flags[replicated_row])
{
any_matched = true;
added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing);
total_added_rows += 1;
}
++selected_right_row_it;
}
}
if (!any_matched)
{
if (add_missing)
addNotFoundRow<true, need_replication>(added_columns, total_added_rows);
else
addNotFoundRow<false, need_replication>(added_columns, total_added_rows);
}
else
{
if (!flag_per_row && need_flags)
used_flags.template setUsed<true, false>(find_results[find_result_index]);
if (need_filter)
setUsed<true>(added_columns.filter, left_start_row + i - 1);
if (add_missing)
added_columns.applyLazyDefaults();
}
find_result_index += (prev_replicated_row != row_replicate_offset[i]);
if constexpr (need_replication)
{
(*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows;
}
prev_replicated_row = row_replicate_offset[i];
}
};
while (left_row_iter < left_block_rows && !exceeded_max_block_rows)
{
auto left_start_row = left_row_iter;
collect_keys_matched_rows_refs();
if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, "
"left_start_row: {}",
selected_rows.size(),
current_added_rows,
row_replicate_offset.size(),
left_row_iter,
left_start_row);
}
auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns);
copy_final_matched_rows(left_start_row, filter_col);
if constexpr (need_replication)
{
// Add a check for current_added_rows to avoid run the filter expression on too small size batch.
if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024)
exceeded_max_block_rows = true;
}
}
if constexpr (need_replication)
{
added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter);
added_columns.filter.resize_assume_reserved(left_row_iter);
}
added_columns.applyLazyDefaults();
return left_row_iter;
}
bool flag_per_row [[maybe_unused]]);
/// Cut first num_rows rows from block in place and returns block with remaining rows
static Block sliceBlock(Block & block, size_t num_rows)
{
size_t total_rows = block.rows();
if (num_rows >= total_rows)
return {};
size_t remaining_rows = total_rows - num_rows;
Block remaining_block = block.cloneEmpty();
for (size_t i = 0; i < block.columns(); ++i)
{
auto & col = block.getByPosition(i);
remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows);
col.column = col.column->cut(0, num_rows);
}
return remaining_block;
}
static Block sliceBlock(Block & block, size_t num_rows);
/** Since we do not store right key columns,
* this function is used to copy left key columns to right key columns.
@ -864,70 +158,22 @@ private:
const DataTypePtr & right_key_type,
const String & renamed_right_column,
const ColumnWithTypeAndName & left_column,
const IColumn::Filter * null_map_filter = nullptr)
{
ColumnWithTypeAndName right_column = left_column;
right_column.name = renamed_right_column;
const IColumn::Filter * null_map_filter = nullptr);
if (null_map_filter)
right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter);
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable);
bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type);
if (null_map_filter)
correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter);
else
correctNullabilityInplace(right_column, should_be_nullable);
if (!right_column.type->equals(*right_key_type))
{
right_column.column = castColumnAccurate(right_column, right_key_type);
right_column.type = right_key_type;
}
right_column.column = right_column.column->convertToFullColumnIfConst();
return right_column;
}
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable)
{
if (nullable)
{
JoinCommon::convertColumnToNullable(column);
}
else
{
/// We have to replace values masked by NULLs with defaults.
if (column.column)
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&*column.column))
column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true);
JoinCommon::removeColumnNullability(column);
}
}
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map)
{
if (nullable)
{
JoinCommon::convertColumnToNullable(column);
if (column.type->isNullable() && !negative_null_map.empty())
{
MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column));
assert_cast<ColumnNullable &>(*mutable_column).applyNegatedNullMap(negative_null_map);
column.column = std::move(mutable_column);
}
}
else
JoinCommon::removeColumnNullability(column);
}
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map);
};
/// Instantiate template class ahead in different .cpp files to avoid `too large translation unit`.
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::RightAny, HashJoin::MapsOne>;
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Any, HashJoin::MapsOne>;
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Any, HashJoin::MapsAll>;
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::All, HashJoin::MapsAll>;
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Semi, HashJoin::MapsOne>;
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Semi, HashJoin::MapsAll>;
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Anti, HashJoin::MapsOne>;
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Anti, HashJoin::MapsAll>;
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Asof, HashJoin::MapsAsof>;
extern template class HashJoinMethods<JoinKind::Right, JoinStrictness::RightAny, HashJoin::MapsOne>;
@ -939,6 +185,7 @@ extern template class HashJoinMethods<JoinKind::Right, JoinStrictness::Asof, Has
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::RightAny, HashJoin::MapsOne>;
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Any, HashJoin::MapsOne>;
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Any, HashJoin::MapsAll>;
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::All, HashJoin::MapsAll>;
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Semi, HashJoin::MapsOne>;
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Anti, HashJoin::MapsOne>;

View File

@ -0,0 +1,936 @@
#pragma once
#include <Interpreters/HashJoin/HashJoinMethods.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_JOIN_KEYS;
extern const int LOGICAL_ERROR;
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImpl(
HashJoin & join,
HashJoin::Type type,
MapsTemplate & maps,
size_t rows,
const ColumnRawPtrs & key_columns,
const Sizes & key_sizes,
Block * stored_block,
ConstNullMapPtr null_map,
UInt8ColumnDataPtr join_mask,
Arena & pool,
bool & is_inserted)
{
switch (type)
{
case HashJoin::Type::EMPTY:
[[fallthrough]];
case HashJoin::Type::CROSS:
/// Do nothing. We will only save block, and it is enough
is_inserted = true;
return 0;
#define M(TYPE) \
case HashJoin::Type::TYPE: \
return insertFromBlockImplTypeCase< \
typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>( \
join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \
break;
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
}
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
const HashJoin & join, Block & block, const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get)
{
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
std::vector<JoinOnKeyColumns> join_on_keys;
const auto & onexprs = join.table_join->getClauses();
for (size_t i = 0; i < onexprs.size(); ++i)
{
const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right;
join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]);
}
size_t existing_columns = block.columns();
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
* Because if they are constants, then in the "not joined" rows, they may have different values
* - default values, which can differ from the values of these constants.
*/
if constexpr (join_features.right || join_features.full)
{
materializeBlockInplace(block);
}
/** For LEFT/INNER JOIN, the saved blocks do not contain keys.
* For FULL/RIGHT JOIN, the saved blocks contain keys;
* but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped.
* For ASOF, the last column is used as the ASOF column
*/
AddedColumns<!join_features.is_any_join> added_columns(
block,
block_with_columns_to_add,
join.savedBlockSample(),
join,
std::move(join_on_keys),
join.table_join->getMixedJoinExpression(),
join_features.is_asof_join,
is_join_get);
bool has_required_right_keys = (join.required_right_keys.columns() != 0);
added_columns.need_filter = join_features.need_filter || has_required_right_keys;
added_columns.max_joined_block_rows = join.max_joined_block_rows;
if (!added_columns.max_joined_block_rows)
added_columns.max_joined_block_rows = std::numeric_limits<size_t>::max();
else
added_columns.reserve(join_features.need_replication);
size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags);
/// Do not hold memory for join_on_keys anymore
added_columns.join_on_keys.clear();
Block remaining_block = sliceBlock(block, num_joined);
added_columns.buildOutput();
for (size_t i = 0; i < added_columns.size(); ++i)
block.insert(added_columns.moveColumn(i));
std::vector<size_t> right_keys_to_replicate [[maybe_unused]];
if constexpr (join_features.need_filter)
{
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
for (size_t i = 0; i < existing_columns; ++i)
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1);
/// Add join key columns from right block if needed using value from left table because of equality
for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
{
const auto & right_key = join.required_right_keys.getByPosition(i);
/// asof column is already in block.
if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back())
continue;
const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column);
block.insert(std::move(right_col));
}
}
else if (has_required_right_keys)
{
/// Add join key columns from right block if needed.
for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
{
const auto & right_key = join.required_right_keys.getByPosition(i);
auto right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
/// asof column is already in block.
if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back())
continue;
const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter);
block.insert(std::move(right_col));
if constexpr (join_features.need_replication)
right_keys_to_replicate.push_back(block.getPositionByName(right_col_name));
}
}
if constexpr (join_features.need_replication)
{
std::unique_ptr<IColumn::Offsets> & offsets_to_replicate = added_columns.offsets_to_replicate;
/// If ALL ... JOIN - we replicate all the columns except the new ones.
for (size_t i = 0; i < existing_columns; ++i)
{
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate);
}
/// Replicate additional right keys
for (size_t pos : right_keys_to_replicate)
{
block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
}
}
return remaining_block;
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename KeyGetter, bool is_asof_join>
KeyGetter HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes)
{
if constexpr (is_asof_join)
{
auto key_column_copy = key_columns;
auto key_size_copy = key_sizes;
key_column_copy.pop_back();
key_size_copy.pop_back();
return KeyGetter(key_column_copy, key_size_copy, nullptr);
}
else
return KeyGetter(key_columns, key_sizes, nullptr);
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename KeyGetter, typename HashMap>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImplTypeCase(
HashJoin & join,
HashMap & map,
size_t rows,
const ColumnRawPtrs & key_columns,
const Sizes & key_sizes,
Block * stored_block,
ConstNullMapPtr null_map,
UInt8ColumnDataPtr join_mask,
Arena & pool,
bool & is_inserted)
{
[[maybe_unused]] constexpr bool mapped_one = std::is_same_v<typename HashMap::mapped_type, RowRef>;
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
const IColumn * asof_column [[maybe_unused]] = nullptr;
if constexpr (is_asof_join)
asof_column = key_columns.back();
auto key_getter = createKeyGetter<KeyGetter, is_asof_join>(key_columns, key_sizes);
/// For ALL and ASOF join always insert values
is_inserted = !mapped_one || is_asof_join;
for (size_t i = 0; i < rows; ++i)
{
if (null_map && (*null_map)[i])
{
/// nulls are not inserted into hash table,
/// keep them for RIGHT and FULL joins
is_inserted = true;
continue;
}
/// Check condition for right table from ON section
if (join_mask && !(*join_mask)[i])
continue;
if constexpr (is_asof_join)
Inserter<HashMap, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column);
else if constexpr (mapped_one)
is_inserted |= Inserter<HashMap, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool);
else
Inserter<HashMap, KeyGetter>::insertAll(join, map, key_getter, stored_block, i, pool);
}
return map.getBufferSizeInCells();
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename AddedColumns>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::switchJoinRightColumns(
const std::vector<const MapsTemplate *> & mapv,
AddedColumns & added_columns,
HashJoin::Type type,
JoinStuff::JoinUsedFlags & used_flags)
{
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
switch (type)
{
case HashJoin::Type::EMPTY: {
if constexpr (!is_asof_join)
{
using KeyGetter = KeyGetterEmpty<typename MapsTemplate::MappedType>;
std::vector<KeyGetter> key_getter_vector;
key_getter_vector.emplace_back();
using MapTypeVal = typename KeyGetter::MappedType;
std::vector<const MapTypeVal *> a_map_type_vector;
a_map_type_vector.emplace_back();
return joinRightColumnsSwitchNullability<KeyGetter>(
std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags);
}
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys. Type: {}", type);
}
#define M(TYPE) \
case HashJoin::Type::TYPE: { \
using MapTypeVal = const typename std::remove_reference_t<decltype(MapsTemplate::TYPE)>::element_type; \
using KeyGetter = typename KeyGetterForType<HashJoin::Type::TYPE, MapTypeVal>::Type; \
std::vector<const MapTypeVal *> a_map_type_vector(mapv.size()); \
std::vector<KeyGetter> key_getter_vector; \
for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \
{ \
const auto & join_on_key = added_columns.join_on_keys[d]; \
a_map_type_vector[d] = mapv[d]->TYPE.get(); \
key_getter_vector.push_back( \
std::move(createKeyGetter<KeyGetter, is_asof_join>(join_on_key.key_columns, join_on_key.key_sizes))); \
} \
return joinRightColumnsSwitchNullability<KeyGetter>(std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \
}
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
default:
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type);
}
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename KeyGetter, typename Map, typename AddedColumns>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumnsSwitchNullability(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags)
{
if (added_columns.need_filter)
{
return joinRightColumnsSwitchMultipleDisjuncts<KeyGetter, Map, true>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
else
{
return joinRightColumnsSwitchMultipleDisjuncts<KeyGetter, Map, false>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename KeyGetter, typename Map, bool need_filter, typename AddedColumns>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumnsSwitchMultipleDisjuncts(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags)
{
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
if constexpr (join_features.is_maps_all)
{
if (added_columns.additional_filter_expression)
{
bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1;
return joinRightColumnsWithAddtitionalFilter<KeyGetter, Map>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags, need_filter, mark_per_row_used);
}
}
if (added_columns.additional_filter_expression)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN");
return mapv.size() > 1 ? joinRightColumns<KeyGetter, Map, need_filter, true>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags)
: joinRightColumns<KeyGetter, Map, need_filter, false>(
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
/// Joins right table columns which indexes are present in right_indexes using specified map.
/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename KeyGetter, typename Map, bool need_filter, bool flag_per_row, typename AddedColumns>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumns(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags)
{
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
size_t rows = added_columns.rows_to_add;
if constexpr (need_filter)
added_columns.filter = IColumn::Filter(rows, 0);
Arena pool;
if constexpr (join_features.need_replication)
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
IColumn::Offset current_offset = 0;
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
size_t i = 0;
for (; i < rows; ++i)
{
if constexpr (join_features.need_replication)
{
if (unlikely(current_offset >= max_joined_block_rows))
{
added_columns.offsets_to_replicate->resize_assume_reserved(i);
added_columns.filter.resize_assume_reserved(i);
break;
}
}
bool right_row_found = false;
KnownRowsHolder<flag_per_row> known_rows;
for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx)
{
const auto & join_keys = added_columns.join_on_keys[onexpr_idx];
if (join_keys.null_map && (*join_keys.null_map)[i])
continue;
bool row_acceptable = !join_keys.isRowFiltered(i);
using FindResult = typename KeyGetter::FindResult;
auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult();
if (find_result.isFound())
{
right_row_found = true;
auto & mapped = find_result.getMapped();
if constexpr (join_features.is_asof_join)
{
const IColumn & left_asof_key = added_columns.leftAsofKey();
auto row_ref = mapped->findAsof(left_asof_key, i);
if (row_ref.block)
{
setUsed<need_filter>(added_columns.filter, i);
if constexpr (flag_per_row)
used_flags.template setUsed<join_features.need_flags, flag_per_row>(row_ref.block, row_ref.row_num, 0);
else
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing);
}
else
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
}
else if constexpr (join_features.is_all_join)
{
setUsed<need_filter>(added_columns.filter, i);
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
addFoundRowAll<Map, join_features.add_missing>(mapped, added_columns, current_offset, known_rows, used_flags_opt);
}
else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right)
{
/// Use first appeared left key + it needs left columns replication
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
if (used_once)
{
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
setUsed<need_filter>(added_columns.filter, i);
addFoundRowAll<Map, join_features.add_missing>(mapped, added_columns, current_offset, known_rows, used_flags_opt);
}
}
else if constexpr (join_features.is_any_join && join_features.inner)
{
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
/// Use first appeared left key only
if (used_once)
{
setUsed<need_filter>(added_columns.filter, i);
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
}
break;
}
else if constexpr (join_features.is_any_join && join_features.full)
{
/// TODO
}
else if constexpr (join_features.is_anti_join)
{
if constexpr (join_features.right && join_features.need_flags)
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
}
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
{
setUsed<need_filter>(added_columns.filter, i);
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
if (join_features.is_any_or_semi_join)
{
break;
}
}
}
}
if (!right_row_found)
{
if constexpr (join_features.is_anti_join && join_features.left)
setUsed<need_filter>(added_columns.filter, i);
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
}
if constexpr (join_features.need_replication)
{
(*added_columns.offsets_to_replicate)[i] = current_offset;
}
}
added_columns.applyLazyDefaults();
return i;
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <bool need_filter>
void HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]])
{
if constexpr (need_filter)
filter[pos] = 1;
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename AddedColumns>
ColumnPtr HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::buildAdditionalFilter(
size_t left_start_row,
const std::vector<RowRef> & selected_rows,
const std::vector<size_t> & row_replicate_offset,
AddedColumns & added_columns)
{
ColumnPtr result_column;
do
{
if (selected_rows.empty())
{
result_column = ColumnUInt8::create();
break;
}
const Block & sample_right_block = *selected_rows.begin()->block;
if (!sample_right_block || !added_columns.additional_filter_expression)
{
auto filter = ColumnUInt8::create();
filter->insertMany(1, selected_rows.size());
result_column = std::move(filter);
break;
}
auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes();
if (required_cols.empty())
{
Block block;
added_columns.additional_filter_expression->execute(block);
result_column = block.getByPosition(0).column->cloneResized(selected_rows.size());
break;
}
NameSet required_column_names;
for (auto & col : required_cols)
required_column_names.insert(col.name);
Block executed_block;
size_t right_col_pos = 0;
for (const auto & col : sample_right_block.getColumnsWithTypeAndName())
{
if (required_column_names.contains(col.name))
{
auto new_col = col.column->cloneEmpty();
for (const auto & selected_row : selected_rows)
{
const auto & src_col = selected_row.block->getByPosition(right_col_pos);
new_col->insertFrom(*src_col.column, selected_row.row_num);
}
executed_block.insert({std::move(new_col), col.type, col.name});
}
right_col_pos += 1;
}
if (!executed_block)
{
result_column = ColumnUInt8::create();
break;
}
for (const auto & col_name : required_column_names)
{
const auto * src_col = added_columns.left_block.findByName(col_name);
if (!src_col)
continue;
auto new_col = src_col->column->cloneEmpty();
size_t prev_left_offset = 0;
for (size_t i = 1; i < row_replicate_offset.size(); ++i)
{
const size_t & left_offset = row_replicate_offset[i];
size_t rows = left_offset - prev_left_offset;
if (rows)
new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows);
prev_left_offset = left_offset;
}
executed_block.insert({std::move(new_col), src_col->type, col_name});
}
if (!executed_block)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"required columns: [{}], but not found any in left/right table. right table: {}, left table: {}",
required_cols.toString(),
sample_right_block.dumpNames(),
added_columns.left_block.dumpNames());
}
for (const auto & col : executed_block.getColumnsWithTypeAndName())
if (!col.column || !col.type)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure());
added_columns.additional_filter_expression->execute(executed_block);
result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst();
executed_block.clear();
} while (false);
result_column = result_column->convertToFullIfNeeded();
if (result_column->isNullable())
{
/// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros
/// Trying to avoid copying data, since we are the only owner of the column.
ColumnPtr mask_column = assert_cast<const ColumnNullable &>(*result_column).getNullMapColumnPtr();
MutableColumnPtr mutable_column;
{
ColumnPtr nested_column = assert_cast<const ColumnNullable &>(*result_column).getNestedColumnPtr();
result_column.reset();
mutable_column = IColumn::mutate(std::move(nested_column));
}
auto & column_data = assert_cast<ColumnUInt8 &>(*mutable_column).getData();
const auto & mask_column_data = assert_cast<const ColumnUInt8 &>(*mask_column).getData();
for (size_t i = 0; i < column_data.size(); ++i)
{
if (mask_column_data[i])
column_data[i] = 0;
}
return mutable_column;
}
return result_column;
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename KeyGetter, typename Map, typename AddedColumns>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumnsWithAddtitionalFilter(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]],
bool need_filter [[maybe_unused]],
bool flag_per_row [[maybe_unused]])
{
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
size_t left_block_rows = added_columns.rows_to_add;
if (need_filter)
added_columns.filter = IColumn::Filter(left_block_rows, 0);
std::unique_ptr<Arena> pool;
if constexpr (join_features.need_replication)
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(left_block_rows);
std::vector<size_t> row_replicate_offset;
row_replicate_offset.reserve(left_block_rows);
using FindResult = typename KeyGetter::FindResult;
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
size_t left_row_iter = 0;
PreSelectedRows selected_rows;
selected_rows.reserve(left_block_rows);
std::vector<FindResult> find_results;
find_results.reserve(left_block_rows);
bool exceeded_max_block_rows = false;
IColumn::Offset total_added_rows = 0;
IColumn::Offset current_added_rows = 0;
auto collect_keys_matched_rows_refs = [&]()
{
pool = std::make_unique<Arena>();
find_results.clear();
row_replicate_offset.clear();
row_replicate_offset.push_back(0);
current_added_rows = 0;
selected_rows.clear();
for (; left_row_iter < left_block_rows; ++left_row_iter)
{
if constexpr (join_features.need_replication)
{
if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows))
{
break;
}
}
KnownRowsHolder<true> all_flag_known_rows;
KnownRowsHolder<false> single_flag_know_rows;
for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx)
{
const auto & join_keys = added_columns.join_on_keys[join_clause_idx];
if (join_keys.null_map && (*join_keys.null_map)[left_row_iter])
continue;
bool row_acceptable = !join_keys.isRowFiltered(left_row_iter);
auto find_result = row_acceptable
? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool)
: FindResult();
if (find_result.isFound())
{
auto & mapped = find_result.getMapped();
find_results.push_back(find_result);
/// We don't add missing in addFoundRowAll here. we will add it after filter is applied.
/// it's different from `joinRightColumns`.
if (flag_per_row)
addFoundRowAll<Map, false, true>(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr);
else
addFoundRowAll<Map, false, false>(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr);
}
}
row_replicate_offset.push_back(current_added_rows);
}
};
auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col)
{
const PaddedPODArray<UInt8> & filter_flags = assert_cast<const ColumnUInt8 &>(*filter_col).getData();
size_t prev_replicated_row = 0;
auto selected_right_row_it = selected_rows.begin();
size_t find_result_index = 0;
for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i)
{
bool any_matched = false;
/// right/full join or multiple disjuncts, we need to mark used flags for each row.
if (flag_per_row)
{
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
{
if (filter_flags[replicated_row])
{
if constexpr (join_features.is_semi_join || join_features.is_any_join)
{
/// For LEFT/INNER SEMI/ANY JOIN, we need to add only first appeared row from left,
if constexpr (join_features.left || join_features.inner)
{
if (!any_matched)
{
// For inner join, we need mark each right row'flag, because we only use each right row once.
auto used_once = used_flags.template setUsedOnce<join_features.need_flags, true>(
selected_right_row_it->block, selected_right_row_it->row_num, 0);
if (used_once)
{
any_matched = true;
total_added_rows += 1;
added_columns.appendFromBlock(
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
}
}
}
else
{
auto used_once = used_flags.template setUsedOnce<join_features.need_flags, true>(
selected_right_row_it->block, selected_right_row_it->row_num, 0);
if (used_once)
{
any_matched = true;
total_added_rows += 1;
added_columns.appendFromBlock(
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
}
}
}
else if constexpr (join_features.is_anti_join)
{
any_matched = true;
if constexpr (join_features.right && join_features.need_flags)
used_flags.template setUsed<true, true>(selected_right_row_it->block, selected_right_row_it->row_num, 0);
}
else
{
any_matched = true;
total_added_rows += 1;
added_columns.appendFromBlock(
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
used_flags.template setUsed<join_features.need_flags, true>(
selected_right_row_it->block, selected_right_row_it->row_num, 0);
}
}
++selected_right_row_it;
}
}
else
{
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
{
if constexpr (join_features.is_anti_join)
{
any_matched |= filter_flags[replicated_row];
}
else if constexpr (join_features.need_replication)
{
if (filter_flags[replicated_row])
{
any_matched = true;
added_columns.appendFromBlock(
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
total_added_rows += 1;
}
++selected_right_row_it;
}
else
{
if (filter_flags[replicated_row])
{
any_matched = true;
added_columns.appendFromBlock(
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
total_added_rows += 1;
selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row;
break;
}
else
++selected_right_row_it;
}
}
}
if constexpr (join_features.is_anti_join)
{
if (!any_matched)
{
if constexpr (join_features.left)
if (need_filter)
setUsed<true>(added_columns.filter, left_start_row + i - 1);
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, total_added_rows);
}
}
else
{
if (!any_matched)
{
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, total_added_rows);
}
else
{
if (!flag_per_row)
used_flags.template setUsed<join_features.need_flags, false>(find_results[find_result_index]);
if (need_filter)
setUsed<true>(added_columns.filter, left_start_row + i - 1);
if constexpr (join_features.add_missing)
added_columns.applyLazyDefaults();
}
}
find_result_index += (prev_replicated_row != row_replicate_offset[i]);
if constexpr (join_features.need_replication)
{
(*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows;
}
prev_replicated_row = row_replicate_offset[i];
}
};
while (left_row_iter < left_block_rows && !exceeded_max_block_rows)
{
auto left_start_row = left_row_iter;
collect_keys_matched_rows_refs();
if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, "
"left_start_row: {}",
selected_rows.size(),
current_added_rows,
row_replicate_offset.size(),
left_row_iter,
left_start_row);
}
auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns);
copy_final_matched_rows(left_start_row, filter_col);
if constexpr (join_features.need_replication)
{
// Add a check for current_added_rows to avoid run the filter expression on too small size batch.
if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024)
exceeded_max_block_rows = true;
}
}
if constexpr (join_features.need_replication)
{
added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter);
added_columns.filter.resize_assume_reserved(left_row_iter);
}
added_columns.applyLazyDefaults();
return left_row_iter;
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::sliceBlock(Block & block, size_t num_rows)
{
size_t total_rows = block.rows();
if (num_rows >= total_rows)
return {};
size_t remaining_rows = total_rows - num_rows;
Block remaining_block = block.cloneEmpty();
for (size_t i = 0; i < block.columns(); ++i)
{
auto & col = block.getByPosition(i);
remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows);
col.column = col.column->cut(0, num_rows);
}
return remaining_block;
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
ColumnWithTypeAndName HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::copyLeftKeyColumnToRight(
const DataTypePtr & right_key_type,
const String & renamed_right_column,
const ColumnWithTypeAndName & left_column,
const IColumn::Filter * null_map_filter)
{
ColumnWithTypeAndName right_column = left_column;
right_column.name = renamed_right_column;
if (null_map_filter)
right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter);
bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type);
if (null_map_filter)
correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter);
else
correctNullabilityInplace(right_column, should_be_nullable);
if (!right_column.type->equals(*right_key_type))
{
right_column.column = castColumnAccurate(right_column, right_key_type);
right_column.type = right_key_type;
}
right_column.column = right_column.column->convertToFullColumnIfConst();
return right_column;
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
void HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable)
{
if (nullable)
{
JoinCommon::convertColumnToNullable(column);
}
else
{
/// We have to replace values masked by NULLs with defaults.
if (column.column)
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&*column.column))
column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true);
JoinCommon::removeColumnNullability(column);
}
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
void HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::correctNullabilityInplace(
ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map)
{
if (nullable)
{
JoinCommon::convertColumnToNullable(column);
if (column.type->isNullable() && !negative_null_map.empty())
{
MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column));
assert_cast<ColumnNullable &>(*mutable_column).applyNegatedNullMap(negative_null_map);
column.column = std::move(mutable_column);
}
}
else
JoinCommon::removeColumnNullability(column);
}
}

View File

@ -1,10 +1,11 @@
#include <Interpreters/HashJoin/HashJoinMethods.h>
#include <Interpreters/HashJoin/HashJoinMethodsImpl.h>
namespace DB
{
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::RightAny, HashJoin::MapsOne>;
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Any, HashJoin::MapsOne>;
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Any, HashJoin::MapsAll>;
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::All, HashJoin::MapsAll>;
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Semi, HashJoin::MapsOne>;
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Anti, HashJoin::MapsOne>;

View File

@ -3,15 +3,15 @@
#include <Interpreters/joinDispatch.h>
namespace DB
{
template <JoinKind KIND, JoinStrictness STRICTNESS>
template <JoinKind KIND, JoinStrictness STRICTNESS, typename Map>
struct JoinFeatures
{
static constexpr bool is_any_join = STRICTNESS == JoinStrictness::Any;
static constexpr bool is_any_or_semi_join = STRICTNESS == JoinStrictness::Any || STRICTNESS == JoinStrictness::RightAny || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Left);
static constexpr bool is_all_join = STRICTNESS == JoinStrictness::All;
static constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
static constexpr bool is_semi_join = STRICTNESS == JoinStrictness::Semi;
static constexpr bool is_anti_join = STRICTNESS == JoinStrictness::Anti;
static constexpr bool is_any_or_semi_join = is_any_join || STRICTNESS == JoinStrictness::RightAny || (is_semi_join && KIND == JoinKind::Left);
static constexpr bool left = KIND == JoinKind::Left;
static constexpr bool right = KIND == JoinKind::Right;
@ -22,7 +22,8 @@ struct JoinFeatures
static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left));
static constexpr bool add_missing = (left || full) && !is_semi_join;
static constexpr bool need_flags = MapGetter<KIND, STRICTNESS>::flagged;
static constexpr bool need_flags = MapGetter<KIND, STRICTNESS, std::is_same_v<std::decay_t<Map>, HashJoin::MapsAll>>::flagged;
static constexpr bool is_maps_all = std::is_same_v<std::decay_t<Map>, HashJoin::MapsAll>;
};
}

View File

@ -26,10 +26,10 @@ public:
/// Update size for vector with flags.
/// Calling this method invalidates existing flags.
/// It can be called several times, but all of them should happen before using this structure.
template <JoinKind KIND, JoinStrictness STRICTNESS>
template <JoinKind KIND, JoinStrictness STRICTNESS, bool prefer_use_maps_all>
void reinit(size_t size)
{
if constexpr (MapGetter<KIND, STRICTNESS>::flagged)
if constexpr (MapGetter<KIND, STRICTNESS, prefer_use_maps_all>::flagged)
{
assert(flags[nullptr].size() <= size);
need_flags = true;
@ -43,10 +43,10 @@ public:
}
}
template <JoinKind KIND, JoinStrictness STRICTNESS>
template <JoinKind KIND, JoinStrictness STRICTNESS, bool prefer_use_maps_all>
void reinit(const Block * block_ptr)
{
if constexpr (MapGetter<KIND, STRICTNESS>::flagged)
if constexpr (MapGetter<KIND, STRICTNESS, prefer_use_maps_all>::flagged)
{
assert(flags[block_ptr].size() <= block_ptr->rows());
need_flags = true;
@ -148,6 +148,31 @@ public:
}
}
template <bool use_flags, bool flag_per_row>
bool setUsedOnce(const Block * block, size_t row_num, size_t offset)
{
if constexpr (!use_flags)
return true;
if constexpr (flag_per_row)
{
/// fast check to prevent heavy CAS with seq_cst order
if (flags[block][row_num].load(std::memory_order_relaxed))
return false;
bool expected = false;
return flags[block][row_num].compare_exchange_strong(expected, true);
}
else
{
/// fast check to prevent heavy CAS with seq_cst order
if (flags[nullptr][offset].load(std::memory_order_relaxed))
return false;
bool expected = false;
return flags[nullptr][offset].compare_exchange_strong(expected, true);
}
}
};
}

View File

@ -1,11 +1,14 @@
#include <Interpreters/HashJoin/HashJoinMethods.h>
#include <Interpreters/HashJoin/HashJoinMethodsImpl.h>
namespace DB
{
template class HashJoinMethods<JoinKind::Left, JoinStrictness::RightAny, HashJoin::MapsOne>;
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Any, HashJoin::MapsOne>;
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Any, HashJoin::MapsAll>;
template class HashJoinMethods<JoinKind::Left, JoinStrictness::All, HashJoin::MapsAll>;
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Semi, HashJoin::MapsOne>;
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Semi, HashJoin::MapsAll>;
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Anti, HashJoin::MapsOne>;
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Anti, HashJoin::MapsAll>;
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Asof, HashJoin::MapsAsof>;
}

View File

@ -1,4 +1,4 @@
#include <Interpreters/HashJoin/HashJoinMethods.h>
#include <Interpreters/HashJoin/HashJoinMethodsImpl.h>
namespace DB
{

View File

@ -146,7 +146,6 @@ ColumnDependencies getAllColumnDependencies(
bool isStorageTouchedByMutations(
MergeTreeData & storage,
MergeTreeData::DataPartPtr source_part,
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MutationCommand> & commands,
@ -155,7 +154,9 @@ bool isStorageTouchedByMutations(
if (commands.empty())
return false;
auto storage_from_part = std::make_shared<StorageFromMergeTreeDataPart>(source_part);
bool all_commands_can_be_skipped = true;
for (const auto & command : commands)
{
if (command.type == MutationCommand::APPLY_DELETED_MASK)
@ -170,7 +171,7 @@ bool isStorageTouchedByMutations(
if (command.partition)
{
const String partition_id = storage.getPartitionIDFromQuery(command.partition, context);
const String partition_id = storage_from_part->getPartitionIDFromQuery(command.partition, context);
if (partition_id == source_part->info.partition_id)
all_commands_can_be_skipped = false;
}
@ -184,20 +185,18 @@ bool isStorageTouchedByMutations(
if (all_commands_can_be_skipped)
return false;
auto storage_from_part = std::make_shared<StorageFromMergeTreeDataPart>(source_part);
std::optional<InterpreterSelectQuery> interpreter_select_query;
BlockIO io;
if (context->getSettingsRef().allow_experimental_analyzer)
{
auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context);
auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage_from_part, context);
InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits());
io = interpreter.execute();
}
else
{
ASTPtr select_query = prepareQueryAffectedAST(commands, storage.shared_from_this(), context);
ASTPtr select_query = prepareQueryAffectedAST(commands, storage_from_part, context);
/// Interpreter must be alive, when we use result of execute() method.
/// For some reason it may copy context and give it into ExpressionTransform
/// after that we will use context from destroyed stack frame in our stream.

View File

@ -19,7 +19,6 @@ using QueryPipelineBuilderPtr = std::unique_ptr<QueryPipelineBuilder>;
/// Return false if the data isn't going to be changed by mutations.
bool isStorageTouchedByMutations(
MergeTreeData & storage,
MergeTreeData::DataPartPtr source_part,
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MutationCommand> & commands,

View File

@ -214,6 +214,10 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
}
else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String)
{
/// Bool is not represented in which_type, so we need to type it separately
if (isInt64OrUInt64orBoolFieldType(src.getType()) && type.getName() == "Bool")
return bool(src.safeGet<bool>());
if (which_type.isUInt8()) return convertNumericType<UInt8>(src, type);
if (which_type.isUInt16()) return convertNumericType<UInt16>(src, type);
if (which_type.isUInt32()) return convertNumericType<UInt32>(src, type);

View File

@ -12,38 +12,53 @@
namespace DB
{
template <JoinKind kind, JoinStrictness join_strictness>
/// HashJoin::MapsOne is more efficient, it only store one row for each key in the map. It is recommended to use it whenever possible.
/// When only need to match only one row from right table, use HashJoin::MapsOne. For example, LEFT ANY/SEMI/ANTI.
///
/// HashJoin::MapsAll will store all rows for each key in the map. It is used when need to match multiple rows from right table.
/// For example, LEFT ALL, INNER ALL, RIGHT ALL/ANY.
///
/// prefer_use_maps_all is true when there is mixed inequal condition in the join condition. For example, `t1.a = t2.a AND t1.b > t2.b`.
/// In this case, we need to use HashJoin::MapsAll to store all rows for each key in the map. We will select all matched rows from the map
/// and filter them by `t1.b > t2.b`.
///
/// flagged indicates whether we need to store flags for each row whether it has been used in the join. See JoinUsedFlags.h.
template <JoinKind kind, JoinStrictness join_strictness, bool prefer_use_maps_all>
struct MapGetter;
template <> struct MapGetter<JoinKind::Left, JoinStrictness::RightAny> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::RightAny> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Right, JoinStrictness::RightAny> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Full, JoinStrictness::RightAny> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Left, JoinStrictness::RightAny, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Inner, JoinStrictness::RightAny, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::RightAny, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::RightAny, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Any> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Any> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Right, JoinStrictness::Any> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Full, JoinStrictness::Any> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Any, false> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Any, true> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Any, true> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Any, false> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::Any, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::Any, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Left, JoinStrictness::All> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::All> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Right, JoinStrictness::All> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Full, JoinStrictness::All> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Left, JoinStrictness::All, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Inner, JoinStrictness::All, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::All, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::All, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
/// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation.
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Semi> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Semi> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Right, JoinStrictness::Semi> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Full, JoinStrictness::Semi> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Semi, false> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Semi, true> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Inner, JoinStrictness::Semi, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::Semi, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::Semi, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
/// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation.
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Anti> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Anti> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Right, JoinStrictness::Anti> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <> struct MapGetter<JoinKind::Full, JoinStrictness::Anti> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
/// Only ANTI LEFT and ANTI RIGHT are valid. INNER and FULL are here for templates instantiation.
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Anti, false> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Anti, true> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Inner, JoinStrictness::Anti, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::Anti, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::Anti, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
template <JoinKind kind>
struct MapGetter<kind, JoinStrictness::Asof> { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; };
template <JoinKind kind, bool prefer_use_maps_all>
struct MapGetter<kind, JoinStrictness::Asof, prefer_use_maps_all> { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; };
static constexpr std::array<JoinStrictness, 6> STRICTNESSES = {
JoinStrictness::RightAny,
@ -62,7 +77,7 @@ static constexpr std::array<JoinKind, 4> KINDS = {
};
/// Init specified join map
inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin::MapsVariant & maps)
inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin::MapsVariant & maps, bool prefer_use_maps_all = false)
{
return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij)
{
@ -70,7 +85,10 @@ inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin:
constexpr auto j = ij % STRICTNESSES.size();
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
{
maps = typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map();
if (prefer_use_maps_all)
maps = typename MapGetter<KINDS[i], STRICTNESSES[j], true>::Map();
else
maps = typename MapGetter<KINDS[i], STRICTNESSES[j], false>::Map();
return true;
}
return false;
@ -79,7 +97,7 @@ inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin:
/// Call function on specified join map
template <typename MapsVariant, typename Func>
inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & maps, Func && func)
inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & maps, bool prefer_use_maps_all, Func && func)
{
return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij)
{
@ -89,10 +107,16 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant &
constexpr auto j = ij % STRICTNESSES.size();
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
{
func(
std::integral_constant<JoinKind, KINDS[i]>(),
std::integral_constant<JoinStrictness, STRICTNESSES[j]>(),
std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps));
if (prefer_use_maps_all)
func(
std::integral_constant<JoinKind, KINDS[i]>(),
std::integral_constant<JoinStrictness, STRICTNESSES[j]>(),
std::get<typename MapGetter<KINDS[i], STRICTNESSES[j], true>::Map>(maps));
else
func(
std::integral_constant<JoinKind, KINDS[i]>(),
std::integral_constant<JoinStrictness, STRICTNESSES[j]>(),
std::get<typename MapGetter<KINDS[i], STRICTNESSES[j], false>::Map>(maps));
return true;
}
return false;
@ -101,7 +125,7 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant &
/// Call function on specified join map
template <typename MapsVariant, typename Func>
inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector<const MapsVariant *> & mapsv, Func && func)
inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector<const MapsVariant *> & mapsv, bool prefer_use_maps_all, Func && func)
{
return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij)
{
@ -111,17 +135,31 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector<c
constexpr auto j = ij % STRICTNESSES.size();
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
{
using MapType = typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map;
std::vector<const MapType *> v;
v.reserve(mapsv.size());
for (const auto & el : mapsv)
v.push_back(&std::get<MapType>(*el));
if (prefer_use_maps_all)
{
using MapType = typename MapGetter<KINDS[i], STRICTNESSES[j], true>::Map;
std::vector<const MapType *> v;
v.reserve(mapsv.size());
for (const auto & el : mapsv)
v.push_back(&std::get<MapType>(*el));
func(
std::integral_constant<JoinKind, KINDS[i]>(),
std::integral_constant<JoinStrictness, STRICTNESSES[j]>(),
v
/*std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps)*/);
func(
std::integral_constant<JoinKind, KINDS[i]>(), std::integral_constant<JoinStrictness, STRICTNESSES[j]>(), v
/*std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps)*/);
}
else
{
using MapType = typename MapGetter<KINDS[i], STRICTNESSES[j], false>::Map;
std::vector<const MapType *> v;
v.reserve(mapsv.size());
for (const auto & el : mapsv)
v.push_back(&std::get<MapType>(*el));
func(
std::integral_constant<JoinKind, KINDS[i]>(), std::integral_constant<JoinStrictness, STRICTNESSES[j]>(), v
/*std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps)*/);
}
return true;
}
return false;

View File

@ -1,5 +1,4 @@
#include <Parsers/FieldFromAST.h>
#include <Disks/getOrCreateDiskFromAST.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>

View File

@ -69,9 +69,10 @@ void IOutputFormat::work()
if (finished && !finalized)
{
if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit())
if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedStep())
setRowsBeforeLimit(rows_before_limit_counter->get());
if (rows_before_aggregation_counter && rows_before_aggregation_counter->hasAppliedStep())
setRowsBeforeAggregation(rows_before_aggregation_counter->get());
finalize();
if (auto_flush)
flush();

View File

@ -1,9 +1,9 @@
#pragma once
#include <string>
#include <Processors/IProcessor.h>
#include <Processors/RowsBeforeLimitCounter.h>
#include <IO/Progress.h>
#include <Processors/IProcessor.h>
#include <Processors/RowsBeforeStepCounter.h>
#include <Common/Stopwatch.h>
namespace DB
@ -36,14 +36,20 @@ public:
void setAutoFlush() { auto_flush = true; }
/// Value for rows_before_limit_at_least field.
virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {}
virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) { }
/// Counter to calculate rows_before_limit_at_least in processors pipeline.
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_counter.swap(counter); }
void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_counter.swap(counter); }
/// Value for rows_before_aggregation field.
virtual void setRowsBeforeAggregation(size_t /*rows_before_aggregation*/) { }
/// Counter to calculate rows_before_aggregation in processors pipeline.
void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation_counter.swap(counter); }
/// Notify about progress. Method could be called from different threads.
/// Passed value are delta, that must be summarized.
virtual void onProgress(const Progress & /*progress*/) {}
virtual void onProgress(const Progress & /*progress*/) { }
/// Content-Type to set when sending HTTP response.
virtual std::string getContentType() const { return "text/plain; charset=UTF-8"; }
@ -151,6 +157,8 @@ protected:
Progress progress;
bool applied_limit = false;
size_t rows_before_limit = 0;
bool applied_aggregation = false;
size_t rows_before_aggregation = 0;
Chunk totals;
Chunk extremes;
};
@ -184,7 +192,8 @@ protected:
bool need_write_prefix = true;
bool need_write_suffix = true;
RowsBeforeLimitCounterPtr rows_before_limit_counter;
RowsBeforeStepCounterPtr rows_before_limit_counter;
RowsBeforeStepCounterPtr rows_before_aggregation_counter;
Statistics statistics;
private:

View File

@ -81,6 +81,8 @@ void JSONColumnsWithMetadataBlockOutputFormat::finalizeImpl()
rows,
statistics.rows_before_limit,
statistics.applied_limit,
statistics.rows_before_aggregation,
statistics.applied_aggregation,
statistics.watch,
statistics.progress,
format_settings.write_statistics,

View File

@ -44,6 +44,11 @@ public:
String getName() const override { return "JSONCompactColumnsBlockOutputFormat"; }
void setRowsBeforeLimit(size_t rows_before_limit_) override { statistics.rows_before_limit = rows_before_limit_; statistics.applied_limit = true; }
void setRowsBeforeAggregation(size_t rows_before_aggregation_) override
{
statistics.rows_before_aggregation = rows_before_aggregation_;
statistics.applied_aggregation = true;
}
void onProgress(const Progress & progress_) override { statistics.progress.incrementPiecewiseAtomically(progress_); }
protected:

View File

@ -116,6 +116,8 @@ void JSONRowOutputFormat::finalizeImpl()
row_count,
statistics.rows_before_limit,
statistics.applied_limit,
statistics.rows_before_aggregation,
statistics.applied_aggregation,
statistics.watch,
statistics.progress,
settings.write_statistics && exception_message.empty(),

View File

@ -35,6 +35,11 @@ public:
statistics.applied_limit = true;
statistics.rows_before_limit = rows_before_limit_;
}
void setRowsBeforeAggregation(size_t rows_before_aggregation_) override
{
statistics.applied_aggregation = true;
statistics.rows_before_aggregation = rows_before_aggregation_;
}
protected:
void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override;

View File

@ -313,6 +313,12 @@ private:
statistics.rows_before_limit = rows_before_limit;
statistics.applied_limit = true;
}
void setRowsBeforeAggregation(size_t rows_before_aggregation) override
{
std::lock_guard lock(statistics_mutex);
statistics.rows_before_aggregation = rows_before_aggregation;
statistics.applied_aggregation = true;
}
};
}

View File

@ -42,9 +42,11 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ
case static_cast<size_t>(ResultsetPart::TimeElapsed):
case static_cast<size_t>(ResultsetPart::RowsRead):
case static_cast<size_t>(ResultsetPart::BytesRead):
case static_cast<size_t>(ResultsetPart::RowsBeforeAggregation):
if (format.escaping_rules[i] == EscapingRule::None)
format.throwInvalidFormat("Serialization type for output part rows, rows_before_limit, time, "
"rows_read or bytes_read is not specified", i);
format.throwInvalidFormat(
"Serialization type for output part rows, rows, time, "
"rows_read or bytes_read is not specified", i);
break;
default:
format.throwInvalidFormat("Invalid output part", i);
@ -88,6 +90,8 @@ TemplateBlockOutputFormat::ResultsetPart TemplateBlockOutputFormat::stringToResu
return ResultsetPart::RowsRead;
else if (part == "bytes_read")
return ResultsetPart::BytesRead;
else if (part == "rows_before_aggregation")
return ResultsetPart::RowsBeforeAggregation;
else
throw Exception(ErrorCodes::SYNTAX_ERROR, "Unknown output part {}", part);
}
@ -173,6 +177,11 @@ void TemplateBlockOutputFormat::finalizeImpl()
case ResultsetPart::BytesRead:
writeValue<size_t, DataTypeUInt64>(statistics.progress.read_bytes.load(), format.escaping_rules[i]);
break;
case ResultsetPart::RowsBeforeAggregation:
if (!statistics.applied_aggregation)
format.throwInvalidFormat("Cannot print rows_before_aggregation for this request", i);
writeValue<size_t, DataTypeUInt64>(statistics.rows_before_aggregation, format.escaping_rules[i]);
break;
default:
break;
}

View File

@ -21,6 +21,11 @@ public:
String getName() const override { return "TemplateBlockOutputFormat"; }
void setRowsBeforeLimit(size_t rows_before_limit_) override { statistics.rows_before_limit = rows_before_limit_; statistics.applied_limit = true; }
void setRowsBeforeAggregation(size_t rows_before_aggregation_) override
{
statistics.rows_before_aggregation = rows_before_aggregation_;
statistics.applied_aggregation = true;
}
void onProgress(const Progress & progress_) override { statistics.progress.incrementPiecewiseAtomically(progress_); }
enum class ResultsetPart : size_t
@ -33,7 +38,8 @@ public:
RowsBeforeLimit,
TimeElapsed,
RowsRead,
BytesRead
BytesRead,
RowsBeforeAggregation
};
static ResultsetPart stringToResultsetPart(const String & part);

View File

@ -191,6 +191,7 @@ void XMLRowOutputFormat::finalizeImpl()
writeRowsBeforeLimitAtLeast();
writeRowsBeforeAggregationAtLeast();
if (!exception_message.empty())
writeException();
@ -219,6 +220,16 @@ void XMLRowOutputFormat::writeRowsBeforeLimitAtLeast()
}
}
void XMLRowOutputFormat::writeRowsBeforeAggregationAtLeast()
{
if (statistics.applied_aggregation)
{
writeCString("\t<rows_before_aggregation>", *ostr);
writeIntText(statistics.rows_before_aggregation, *ostr);
writeCString("</rows_before_aggregation>\n", *ostr);
}
}
void XMLRowOutputFormat::writeStatistics()
{
writeCString("\t<statistics>\n", *ostr);

View File

@ -48,6 +48,11 @@ private:
statistics.rows_before_limit = rows_before_limit_;
}
void setRowsBeforeAggregation(size_t rows_before_aggregation_) override
{
statistics.applied_aggregation = true;
statistics.rows_before_aggregation = rows_before_aggregation_;
}
void onRowsReadBeforeUpdate() override { row_count = getRowsReadBefore(); }
void onProgress(const Progress & value) override;
@ -56,6 +61,7 @@ private:
void writeExtremesElement(const char * title, const Columns & columns, size_t row_num);
void writeRowsBeforeLimitAtLeast();
void writeRowsBeforeAggregationAtLeast();
void writeStatistics();
void writeException();

View File

@ -45,4 +45,8 @@ void LazyOutputFormat::setRowsBeforeLimit(size_t rows_before_limit)
info.setRowsBeforeLimit(rows_before_limit);
}
void LazyOutputFormat::setRowsBeforeAggregation(size_t rows_before_aggregation)
{
info.setRowsBeforeAggregation(rows_before_aggregation);
}
}

View File

@ -28,6 +28,7 @@ public:
ProfileInfo & getProfileInfo() { return info; }
void setRowsBeforeLimit(size_t rows_before_limit) override;
void setRowsBeforeAggregation(size_t rows_before_aggregation) override;
void onCancel() noexcept override
{

View File

@ -42,5 +42,8 @@ void PullingOutputFormat::setRowsBeforeLimit(size_t rows_before_limit)
{
info.setRowsBeforeLimit(rows_before_limit);
}
void PullingOutputFormat::setRowsBeforeAggregation(size_t rows_before_aggregation)
{
info.setRowsBeforeAggregation(rows_before_aggregation);
}
}

View File

@ -22,6 +22,7 @@ public:
ProfileInfo & getProfileInfo() { return info; }
void setRowsBeforeLimit(size_t rows_before_limit) override;
void setRowsBeforeAggregation(size_t rows_before_aggregation) override;
bool expectMaterializedColumns() const override { return false; }

View File

@ -21,8 +21,8 @@ class IQueryPlanStep;
struct StorageLimits;
using StorageLimitsList = std::list<StorageLimits>;
class RowsBeforeLimitCounter;
using RowsBeforeLimitCounterPtr = std::shared_ptr<RowsBeforeLimitCounter>;
class RowsBeforeStepCounter;
using RowsBeforeStepCounterPtr = std::shared_ptr<RowsBeforeStepCounter>;
class IProcessor;
using ProcessorPtr = std::shared_ptr<IProcessor>;
@ -377,7 +377,11 @@ public:
/// Set rows_before_limit counter for current processor.
/// This counter is used to calculate the number of rows right before any filtration of LimitTransform.
virtual void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr /* counter */) {}
virtual void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr /* counter */) { }
/// Set rows_before_aggregation counter for current processor.
/// This counter is used to calculate the number of rows right before AggregatingTransform.
virtual void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr /* counter */) { }
protected:
virtual void onCancel() noexcept {}

View File

@ -1,8 +1,8 @@
#pragma once
#include <Processors/IProcessor.h>
#include <Processors/RowsBeforeLimitCounter.h>
#include <Core/SortDescription.h>
#include <Processors/IProcessor.h>
#include <Processors/RowsBeforeStepCounter.h>
namespace DB
{
@ -30,7 +30,7 @@ private:
std::vector<size_t> sort_column_positions;
UInt64 rows_read = 0; /// including the last read block
RowsBeforeLimitCounterPtr rows_before_limit_at_least;
RowsBeforeStepCounterPtr rows_before_limit_at_least;
/// State of port's pair.
/// Chunks from different port pairs are not mixed for better cache locality.
@ -71,7 +71,7 @@ public:
InputPort & getInputPort() { return inputs.front(); }
OutputPort & getOutputPort() { return outputs.front(); }
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); }
void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_at_least.swap(counter); }
void setInputPortHasCounter(size_t pos) { ports_data[pos].input_port_has_counter = true; }
};

View File

@ -1,8 +1,8 @@
#pragma once
#include <Processors/IProcessor.h>
#include <Processors/RowsBeforeLimitCounter.h>
#include <Core/SortDescription.h>
#include <Processors/IProcessor.h>
#include <Processors/RowsBeforeStepCounter.h>
namespace DB
{
@ -16,7 +16,7 @@ private:
UInt64 offset;
UInt64 rows_read = 0; /// including the last read block
RowsBeforeLimitCounterPtr rows_before_limit_at_least;
RowsBeforeStepCounterPtr rows_before_limit_at_least;
/// State of port's pair.
/// Chunks from different port pairs are not mixed for better cache locality.
@ -45,7 +45,7 @@ public:
InputPort & getInputPort() { return inputs.front(); }
OutputPort & getOutputPort() { return outputs.front(); }
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); }
void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit_at_least.swap(counter); }
};
}

View File

@ -1,36 +0,0 @@
#pragma once
#include <atomic>
#include <memory>
namespace DB
{
/// This class helps to calculate rows_before_limit_at_least.
class RowsBeforeLimitCounter
{
public:
void add(uint64_t rows)
{
setAppliedLimit();
rows_before_limit.fetch_add(rows, std::memory_order_release);
}
void set(uint64_t rows)
{
setAppliedLimit();
rows_before_limit.store(rows, std::memory_order_release);
}
uint64_t get() const { return rows_before_limit.load(std::memory_order_acquire); }
void setAppliedLimit() { has_applied_limit.store(true, std::memory_order_release); }
bool hasAppliedLimit() const { return has_applied_limit.load(std::memory_order_acquire); }
private:
std::atomic<uint64_t> rows_before_limit = 0;
std::atomic_bool has_applied_limit = false;
};
using RowsBeforeLimitCounterPtr = std::shared_ptr<RowsBeforeLimitCounter>;
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <atomic>
#include <memory>
namespace DB
{
/// This class helps to calculate rows_before_limit_at_least and rows_before_aggregation.
class RowsBeforeStepCounter
{
public:
void add(uint64_t rows)
{
setAppliedStep();
rows_before_step.fetch_add(rows, std::memory_order_release);
}
void set(uint64_t rows)
{
setAppliedStep();
rows_before_step.store(rows, std::memory_order_release);
}
uint64_t get() const { return rows_before_step.load(std::memory_order_acquire); }
void setAppliedStep() { has_applied_step.store(true, std::memory_order_release); }
bool hasAppliedStep() const { return has_applied_step.load(std::memory_order_acquire); }
private:
std::atomic<uint64_t> rows_before_step = 0;
std::atomic_bool has_applied_step = false;
};
using RowsBeforeStepCounterPtr = std::shared_ptr<RowsBeforeStepCounter>;
}

View File

@ -139,6 +139,12 @@ void DelayedSource::work()
processor->setRowsBeforeLimitCounter(rows_before_limit);
}
if (rows_before_aggregation)
{
for (auto & processor : processors)
processor->setRowsBeforeAggregationCounter(rows_before_aggregation);
}
synchronizePorts(totals_output, totals, header, processors);
synchronizePorts(extremes_output, extremes, header, processors);
}

View File

@ -30,13 +30,15 @@ public:
OutputPort * getTotalsPort() { return totals; }
OutputPort * getExtremesPort() { return extremes; }
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); }
void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit.swap(counter); }
void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); }
private:
QueryPlanResourceHolder resources;
Creator creator;
Processors processors;
RowsBeforeLimitCounterPtr rows_before_limit;
RowsBeforeStepCounterPtr rows_before_limit;
RowsBeforeStepCounterPtr rows_before_aggregation;
/// Outputs for DelayedSource.
OutputPort * main = nullptr;

View File

@ -37,16 +37,23 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation
progress(value.read_rows, value.read_bytes);
});
query_executor->setProfileInfoCallback([this](const ProfileInfo & info)
{
if (rows_before_limit)
query_executor->setProfileInfoCallback(
[this](const ProfileInfo & info)
{
if (info.hasAppliedLimit())
rows_before_limit->add(info.getRowsBeforeLimit());
else
manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit
}
});
if (rows_before_limit)
{
if (info.hasAppliedLimit())
rows_before_limit->add(info.getRowsBeforeLimit());
else
manually_add_rows_before_limit_counter = true; /// Remote subquery doesn't contain a limit
}
if (rows_before_aggregation)
{
if (info.hasAppliedAggregation())
rows_before_aggregation->add(info.getRowsBeforeAggregation());
}
});
}
RemoteSource::~RemoteSource() = default;
@ -184,7 +191,6 @@ std::optional<Chunk> RemoteSource::tryGenerate()
{
if (manually_add_rows_before_limit_counter)
rows_before_limit->add(rows);
query_executor->finish();
return {};
}

View File

@ -1,10 +1,10 @@
#pragma once
#include <Processors/ISource.h>
#include <Processors/RowsBeforeLimitCounter.h>
#include <Processors/RowsBeforeStepCounter.h>
#include <QueryPipeline/Pipe.h>
#include <Core/UUID.h>
#include <Core/UUID.h>
namespace DB
{
@ -25,7 +25,8 @@ public:
void work() override;
String getName() const override { return "Remote"; }
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit.swap(counter); }
void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { rows_before_limit.swap(counter); }
void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); }
/// Stop reading from stream if output port is finished.
void onUpdatePorts() override;
@ -46,7 +47,8 @@ private:
bool executor_finished = false;
bool add_aggregation_info = false;
RemoteQueryExecutorPtr query_executor;
RowsBeforeLimitCounterPtr rows_before_limit;
RowsBeforeStepCounterPtr rows_before_limit;
RowsBeforeStepCounterPtr rows_before_aggregation;
const bool async_read;
const bool async_query_sending;

View File

@ -81,6 +81,8 @@ void AggregatingInOrderTransform::consume(Chunk chunk)
is_consume_started = true;
}
if (rows_before_aggregation)
rows_before_aggregation->add(rows);
src_rows += rows;
src_bytes += chunk.bytes();

View File

@ -45,6 +45,7 @@ public:
void work() override;
void consume(Chunk chunk);
void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); }
private:
void generate();
@ -86,6 +87,8 @@ private:
Chunk current_chunk;
Chunk to_push_chunk;
RowsBeforeStepCounterPtr rows_before_aggregation;
LoggerPtr log = getLogger("AggregatingInOrderTransform");
};

View File

@ -8,7 +8,6 @@
#include <Core/ProtocolDefines.h>
#include <Common/logger_useful.h>
#include <Common/formatReadable.h>
#include <Processors/Transforms/SquashingTransform.h>
@ -676,7 +675,8 @@ void AggregatingTransform::consume(Chunk chunk)
LOG_TRACE(log, "Aggregating");
is_consume_started = true;
}
if (rows_before_aggregation)
rows_before_aggregation->add(num_rows);
src_rows += num_rows;
src_bytes += chunk.bytes();

View File

@ -4,11 +4,13 @@
#include <Interpreters/Aggregator.h>
#include <Processors/Chunk.h>
#include <Processors/IAccumulatingTransform.h>
#include <Common/Stopwatch.h>
#include <Common/setThreadName.h>
#include <Common/scope_guard_safe.h>
#include <Processors/RowsBeforeStepCounter.h>
#include <Common/CurrentMetrics.h>
#include <Common/CurrentThread.h>
#include <Common/Stopwatch.h>
#include <Common/scope_guard_safe.h>
#include <Common/setThreadName.h>
namespace CurrentMetrics
{
@ -168,6 +170,7 @@ public:
Status prepare() override;
void work() override;
Processors expandPipeline() override;
void setRowsBeforeAggregationCounter(RowsBeforeStepCounterPtr counter) override { rows_before_aggregation.swap(counter); }
protected:
void consume(Chunk chunk);
@ -211,6 +214,8 @@ private:
bool is_consume_started = false;
RowsBeforeStepCounterPtr rows_before_aggregation;
void initGenerate();
};

View File

@ -1,7 +1,7 @@
#pragma once
#include <Processors/ISimpleTransform.h>
#include <Processors/RowsBeforeLimitCounter.h>
#include <Core/SortDescription.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/RowsBeforeStepCounter.h>
#include <Common/PODArray.h>
namespace DB
@ -20,7 +20,7 @@ public:
String getName() const override { return "PartialSortingTransform"; }
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { read_rows.swap(counter); }
void setRowsBeforeLimitCounter(RowsBeforeStepCounterPtr counter) override { read_rows.swap(counter); }
protected:
void transform(Chunk & chunk) override;
@ -29,7 +29,7 @@ private:
const SortDescription description;
SortDescriptionWithPositions description_with_positions;
const UInt64 limit;
RowsBeforeLimitCounterPtr read_rows;
RowsBeforeStepCounterPtr read_rows;
Columns sort_description_threshold_columns;

View File

@ -1,14 +1,14 @@
#include <QueryPipeline/ProfileInfo.h>
#include <Core/Block.h>
#include <Core/ProtocolDefines.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Core/Block.h>
namespace DB
{
void ProfileInfo::read(ReadBuffer & in)
void ProfileInfo::read(ReadBuffer & in, UInt64 server_revision)
{
readVarUInt(rows, in);
readVarUInt(blocks, in);
@ -16,10 +16,15 @@ void ProfileInfo::read(ReadBuffer & in)
readBinary(applied_limit, in);
readVarUInt(rows_before_limit, in);
readBinary(calculated_rows_before_limit, in);
if (server_revision >= DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION)
{
readBinary(applied_aggregation, in);
readVarUInt(rows_before_aggregation, in);
}
}
void ProfileInfo::write(WriteBuffer & out) const
void ProfileInfo::write(WriteBuffer & out, UInt64 client_revision) const
{
writeVarUInt(rows, out);
writeVarUInt(blocks, out);
@ -27,6 +32,11 @@ void ProfileInfo::write(WriteBuffer & out) const
writeBinary(hasAppliedLimit(), out);
writeVarUInt(getRowsBeforeLimit(), out);
writeBinary(calculated_rows_before_limit, out);
if (client_revision >= DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION)
{
writeBinary(hasAppliedAggregation(), out);
writeVarUInt(getRowsBeforeAggregation(), out);
}
}
@ -41,6 +51,8 @@ void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info)
applied_limit = rhs.applied_limit;
rows_before_limit = rhs.rows_before_limit;
calculated_rows_before_limit = rhs.calculated_rows_before_limit;
applied_aggregation = rhs.applied_aggregation;
rows_before_aggregation = rhs.rows_before_aggregation;
}
@ -57,6 +69,17 @@ bool ProfileInfo::hasAppliedLimit() const
return applied_limit;
}
size_t ProfileInfo::getRowsBeforeAggregation() const
{
return rows_before_aggregation;
}
bool ProfileInfo::hasAppliedAggregation() const
{
return applied_aggregation;
}
void ProfileInfo::update(Block & block)
{

View File

@ -32,13 +32,16 @@ struct ProfileInfo
size_t getRowsBeforeLimit() const;
bool hasAppliedLimit() const;
size_t getRowsBeforeAggregation() const;
bool hasAppliedAggregation() const;
void update(Block & block);
void update(size_t num_rows, size_t num_bytes);
/// Binary serialization and deserialization of main fields.
/// Writes only main fields i.e. fields that required by internal transmission protocol.
void read(ReadBuffer & in);
void write(WriteBuffer & out) const;
void read(ReadBuffer & in, UInt64 server_revision);
void write(WriteBuffer & out, UInt64 client_revision) const;
/// Sets main fields from other object (see methods above).
/// If skip_block_size_info if true, then rows, bytes and block fields are ignored.
@ -51,11 +54,21 @@ struct ProfileInfo
rows_before_limit = rows_before_limit_;
}
/// Only for Processors.
void setRowsBeforeAggregation(size_t rows_before_aggregation_)
{
applied_aggregation = true;
rows_before_aggregation = rows_before_aggregation_;
}
private:
/// For these fields we make accessors, because they must be calculated beforehand.
mutable bool applied_limit = false; /// Whether LIMIT was applied
mutable size_t rows_before_limit = 0;
mutable bool calculated_rows_before_limit = false; /// Whether the field rows_before_limit was calculated
mutable bool calculated_rows_before_limit = false; /// Whether the field rows was calculated
mutable bool applied_aggregation = false; /// Whether GROUP BY was applied
mutable size_t rows_before_aggregation = 0;
};
}

View File

@ -1,15 +1,14 @@
#include <QueryPipeline/QueryPipeline.h>
#include <queue>
#include <QueryPipeline/Chain.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/IProcessor.h>
#include <Processors/LimitTransform.h>
#include <Core/Settings.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/ExpressionActions.h>
#include <QueryPipeline/ReadProgressCallback.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/printPipeline.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/IProcessor.h>
#include <Processors/ISource.h>
#include <Processors/LimitTransform.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/Sinks/EmptySink.h>
#include <Processors/Sinks/NullSink.h>
#include <Processors/Sinks/SinkToStorage.h>
@ -17,15 +16,19 @@
#include <Processors/Sources/NullSource.h>
#include <Processors/Sources/RemoteSource.h>
#include <Processors/Sources/SourceFromChunks.h>
#include <Processors/ISource.h>
#include <Processors/Transforms/AggregatingInOrderTransform.h>
#include <Processors/Transforms/AggregatingTransform.h>
#include <Processors/Transforms/CountingTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Transforms/PartialSortingTransform.h>
#include <Processors/Transforms/StreamInQueryCacheTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/TotalsHavingTransform.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <QueryPipeline/Chain.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/ReadProgressCallback.h>
#include <QueryPipeline/printPipeline.h>
namespace DB
@ -139,7 +142,7 @@ static void checkCompleted(Processors & processors)
static void initRowsBeforeLimit(IOutputFormat * output_format)
{
RowsBeforeLimitCounterPtr rows_before_limit_at_least;
RowsBeforeStepCounterPtr rows_before_limit_at_least;
std::vector<IProcessor *> processors;
std::map<LimitTransform *, std::vector<size_t>> limit_candidates;
std::unordered_set<IProcessor *> visited;
@ -261,7 +264,7 @@ static void initRowsBeforeLimit(IOutputFormat * output_format)
if (!processors.empty())
{
rows_before_limit_at_least = std::make_shared<RowsBeforeLimitCounter>();
rows_before_limit_at_least = std::make_shared<RowsBeforeStepCounter>();
for (auto & processor : processors)
processor->setRowsBeforeLimitCounter(rows_before_limit_at_least);
@ -273,7 +276,28 @@ static void initRowsBeforeLimit(IOutputFormat * output_format)
output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least);
}
}
static void initRowsBeforeAggregation(std::shared_ptr<Processors> processors, IOutputFormat * output_format)
{
bool has_aggregation = false;
if (!processors->empty())
{
RowsBeforeStepCounterPtr rows_before_aggregation = std::make_shared<RowsBeforeStepCounter>();
for (const auto & processor : *processors)
{
if (typeid_cast<AggregatingTransform *>(processor.get()) || typeid_cast<AggregatingInOrderTransform *>(processor.get()))
{
processor->setRowsBeforeAggregationCounter(rows_before_aggregation);
has_aggregation = true;
}
if (typeid_cast<RemoteSource *>(processor.get()) || typeid_cast<DelayedSource *>(processor.get()))
processor->setRowsBeforeAggregationCounter(rows_before_aggregation);
}
if (has_aggregation)
rows_before_aggregation->add(0);
output_format->setRowsBeforeAggregationCounter(rows_before_aggregation);
}
}
QueryPipeline::QueryPipeline(
QueryPlanResourceHolder resources_,
@ -521,6 +545,14 @@ void QueryPipeline::complete(std::shared_ptr<IOutputFormat> format)
extremes = nullptr;
initRowsBeforeLimit(format.get());
for (const auto & context : resources.interpreter_context)
{
if (context->getSettingsRef().rows_before_aggregation)
{
initRowsBeforeAggregation(processors, format.get());
break;
}
}
output_format = format.get();
processors->emplace_back(std::move(format));

View File

@ -15,7 +15,7 @@
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/RowsBeforeLimitCounter.h>
#include <Processors/RowsBeforeStepCounter.h>
#include <Processors/Sources/RemoteSource.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Processors/Transforms/CreatingSetsTransform.h>

View File

@ -1577,6 +1577,8 @@ namespace
stats.set_allocated_bytes(info.bytes);
stats.set_applied_limit(info.hasAppliedLimit());
stats.set_rows_before_limit(info.getRowsBeforeLimit());
stats.set_applied_aggregation(info.hasAppliedAggregation());
stats.set_rows_before_aggregation(info.getRowsBeforeAggregation());
}
void Call::addLogsToResult()

View File

@ -1237,7 +1237,7 @@ void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest re
void TCPHandler::sendProfileInfo(const ProfileInfo & info)
{
writeVarUInt(Protocol::Server::ProfileInfo, *out);
info.write(*out);
info.write(*out, client_tcp_protocol_version);
out->next();
}

View File

@ -180,6 +180,8 @@ message Stats {
uint64 allocated_bytes = 3;
bool applied_limit = 4;
uint64 rows_before_limit = 5;
bool applied_aggregation = 6;
uint64 rows_before_aggregation = 7;
}
message Exception {

View File

@ -3359,6 +3359,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"ALTER MODIFY REFRESH is not supported by MergeTree engines family");
if (command.type == AlterCommand::MODIFY_SQL_SECURITY)
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"ALTER MODIFY SQL SECURITY is not supported by MergeTree engines family");
if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
@ -6268,10 +6272,13 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr
}
else
{
String partition_id = getPartitionIDFromQuery(partition, local_context);
String partition_id;
bool all = partition->as<ASTPartition>()->all;
if (!all)
partition_id = getPartitionIDFromQuery(partition, local_context);
DetachedPartsInfo detached_parts = getDetachedParts();
for (const auto & part_info : detached_parts)
if (part_info.valid_name && part_info.partition_id == partition_id
if (part_info.valid_name && (all || part_info.partition_id == partition_id)
&& part_info.prefix != "attaching" && part_info.prefix != "deleting")
renamed_parts.addPart(part_info.dir_name, "deleting_" + part_info.dir_name, part_info.disk);
}

View File

@ -1,6 +1,6 @@
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Disks/getOrCreateDiskFromAST.h>
#include <Disks/DiskFomAST.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTFunction.h>
@ -59,15 +59,19 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte
CustomType custom;
if (name == "disk")
{
ASTPtr value_as_custom_ast = nullptr;
if (value.tryGet<CustomType>(custom) && 0 == strcmp(custom.getTypeName(), "AST"))
value_as_custom_ast = dynamic_cast<const FieldFromASTImpl &>(custom.getImpl()).ast;
if (value_as_custom_ast && isDiskFunction(value_as_custom_ast))
{
auto ast = dynamic_cast<const FieldFromASTImpl &>(custom.getImpl()).ast;
if (ast && isDiskFunction(ast))
{
auto disk_name = getOrCreateDiskFromDiskAST(ast, context, is_attach);
LOG_TRACE(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name);
value = disk_name;
}
auto disk_name = DiskFomAST::createCustomDisk(value_as_custom_ast, context, is_attach);
LOG_DEBUG(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name);
value = disk_name;
}
else
{
DiskFomAST::ensureDiskIsNotCustom(value.safeGet<String>(), context);
}
if (has("storage_policy"))

View File

@ -2134,7 +2134,7 @@ bool MutateTask::prepare()
ctx->commands_for_part.emplace_back(command);
if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations(
*ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading))
ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading))
{
NameSet files_to_copy_instead_of_hardlinks;
auto settings_ptr = ctx->data->getSettings();

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