Merge remote-tracking branch 'origin/master' into LWDRebuildProj

This commit is contained in:
jsc0218 2024-08-10 14:41:03 +00:00
commit 70727c604a
162 changed files with 4392 additions and 1457 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

@ -232,15 +232,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,9 +264,20 @@ 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 \
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

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

@ -11,6 +11,8 @@ Returns a WKT (Well Known Text) geometric object from various [Geo Data Types](.
- 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 `LINESTRING` is returned for a LineString.
- WKT geometric object `MULTILINESTRING` is returned for a MultiLineString.
**Examples**
@ -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.

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

@ -200,8 +200,6 @@ void ClientApplicationBase::init(int argc, char ** argv)
("pager", po::value<std::string>(), "Pipe all output into this command (less or similar)")
("max_memory_usage_in_client", po::value<std::string>(), "Set memory limit in client/local server")
("fuzzer-args", po::value<std::string>(), "Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer.")
("client_logs_file", po::value<std::string>(), "Path to a file for writing client logs. Currently we only have fatal logs (when the client crashes)")
;

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);
if (out_stream)
{
*out_stream << std::endl;
WriteBufferFromOStream ast_buf(*out_stream, 4096);
formatAST(*ast, ast_buf, false /*highlight*/);
ast_buf.finalize();
std::cout << std::endl << std::endl;
*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

@ -1242,6 +1242,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

@ -3,6 +3,7 @@
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Core/Settings.h>
#include <Interpreters/parseColumnsListForTableFunction.h>
@ -35,7 +36,7 @@ FunctionBasePtr createFunctionBaseCast(
class CastOverloadResolverImpl : public IFunctionOverloadResolver
{
public:
const char * getNameImpl() const
static const char * getNameImpl(CastType cast_type, bool internal)
{
if (cast_type == CastType::accurate)
return "accurateCast";
@ -49,7 +50,7 @@ public:
String getName() const override
{
return getNameImpl();
return getNameImpl(cast_type, internal);
}
size_t getNumberOfArguments() const override { return 2; }
@ -79,10 +80,22 @@ public:
}
}
static FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional<CastDiagnostic> diagnostic)
{
if (cast_type == CastType::accurateOrNull && !isVariant(to))
to = makeNullable(to);
ColumnsWithTypeAndName arguments;
arguments.emplace_back(std::move(from));
arguments.emplace_back().type = std::make_unique<DataTypeString>();
return createFunctionBaseCast(nullptr, getNameImpl(cast_type, true), arguments, to, diagnostic, cast_type);
}
protected:
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
{
return createFunctionBaseCast(context, getNameImpl(), arguments, return_type, diagnostic, cast_type);
return createFunctionBaseCast(context, getNameImpl(cast_type, internal), arguments, return_type, diagnostic, cast_type);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
@ -130,9 +143,9 @@ private:
};
FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional<CastDiagnostic> diagnostic)
FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional<CastDiagnostic> diagnostic)
{
return CastOverloadResolverImpl::create(ContextPtr{}, type, true, diagnostic);
return CastOverloadResolverImpl::createInternalCast(std::move(from), std::move(to), cast_type, std::move(diagnostic));
}
REGISTER_FUNCTION(CastOverloadResolvers)

View File

@ -3,6 +3,7 @@
#include <memory>
#include <optional>
#include <Interpreters/Context_fwd.h>
#include <Core/ColumnWithTypeAndName.h>
namespace DB
@ -11,6 +12,9 @@ namespace DB
class IFunctionOverloadResolver;
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
class IFunctionBase;
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
enum class CastType : uint8_t
{
nonAccurate,
@ -24,6 +28,6 @@ struct CastDiagnostic
std::string column_to;
};
FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, std::optional<CastDiagnostic> diagnostic);
FunctionBasePtr createInternalCast(ColumnWithTypeAndName from, DataTypePtr to, CastType cast_type, std::optional<CastDiagnostic> diagnostic);
}

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

@ -54,8 +54,7 @@ namespace
}
};
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {});
auto func_cast = func_builder_cast->build(cast_args);
auto func_cast = createInternalCast(arguments[0], result_type, CastType::nonAccurate, {});
return func_cast->execute(cast_args, result_type, arguments[0].column->size());
}
};

View File

@ -301,11 +301,11 @@ const ActionsDAG::Node & ActionsDAG::addCast(const Node & node_to_cast, const Da
column.column = DataTypeString().createColumnConst(0, cast_type_constant_value);
column.type = std::make_shared<DataTypeString>();
const auto * cast_type_constant_node = &addColumn(std::move(column));
const auto * cast_type_constant_node = &addColumn(column);
ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node};
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::nonAccurate, {});
auto func_base_cast = createInternalCast(ColumnWithTypeAndName{node_to_cast.result_type, node_to_cast.result_name}, cast_type, CastType::nonAccurate, {});
return addFunction(func_builder_cast, std::move(children), result_name);
return addFunction(func_base_cast, std::move(children), result_name);
}
const ActionsDAG::Node & ActionsDAG::addFunctionImpl(
@ -1547,11 +1547,11 @@ ActionsDAG ActionsDAG::makeConvertingActions(
const auto * left_arg = dst_node;
CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name};
FunctionOverloadResolverPtr func_builder_cast
= createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic));
ColumnWithTypeAndName left_column{nullptr, dst_node->result_type, {}};
auto func_base_cast = createInternalCast(std::move(left_column), res_elem.type, CastType::nonAccurate, std::move(diagnostic));
NodeRawConstPtrs children = { left_arg, right_arg };
dst_node = &actions_dag.addFunction(func_builder_cast, std::move(children), {});
dst_node = &actions_dag.addFunction(func_base_cast, std::move(children), {});
}
if (dst_node->column && isColumnConst(*dst_node->column) && !(res_elem.column && isColumnConst(*res_elem.column)))

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(
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

@ -26,11 +26,9 @@ static ColumnPtr castColumn(CastType cast_type, const ColumnWithTypeAndName & ar
""
}
};
auto get_cast_func = [cast_type, &arguments]
auto get_cast_func = [from = arg, to = type, cast_type]
{
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(cast_type, {});
return func_builder_cast->build(arguments);
return createInternalCast(from, to, cast_type, {});
};
FunctionBasePtr func_cast = cache ? cache->getOrSet(cast_type, from_name, to_name, std::move(get_cast_func)) : get_cast_func();

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])
{
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]>::Map>(maps));
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;
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::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
@ -39,7 +39,13 @@ public:
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.
@ -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,8 +42,10 @@ 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, "
format.throwInvalidFormat(
"Serialization type for output part rows, rows, time, "
"rows_read or bytes_read is not specified", i);
break;
default:
@ -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

@ -255,20 +255,13 @@ static void appendAggregateFunctions(
const auto * node = input;
if (node->result_name != aggregate.column_name)
{
if (DataTypeAggregateFunction::strictEquals(type, node->result_type))
{
node = &proj_dag.addAlias(*node, aggregate.column_name);
}
else
{
if (!DataTypeAggregateFunction::strictEquals(type, node->result_type))
/// Cast to aggregate types specified in query if it's not
/// strictly the same as the one specified in projection. This
/// is required to generate correct results during finalization.
node = &proj_dag.addCast(*node, type, aggregate.column_name);
}
}
else if (node->result_name != aggregate.column_name)
node = &proj_dag.addAlias(*node, aggregate.column_name);
proj_dag_outputs.push_back(node);
}

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,7 +37,8 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation
progress(value.read_rows, value.read_bytes);
});
query_executor->setProfileInfoCallback([this](const ProfileInfo & info)
query_executor->setProfileInfoCallback(
[this](const ProfileInfo & info)
{
if (rows_before_limit)
{
@ -46,6 +47,12 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation
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());
}
});
}
@ -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

@ -2337,22 +2337,9 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction
argument_types[2]->getName());
}
const auto from_name = argument_types[2]->getName();
const auto to_name = argument_types[0]->getName();
ColumnsWithTypeAndName arguments
auto get_cast_func = [from = argument_types[2], to = argument_types[0]]
{
{ argument_types[2], "" },
{
DataTypeString().createColumnConst(0, to_name),
std::make_shared<DataTypeString>(),
""
}
};
auto get_cast_func = [&arguments]
{
FunctionOverloadResolverPtr func_builder_cast = createInternalCastOverloadResolver(CastType::accurate, {});
return func_builder_cast->build(arguments);
return createInternalCast({from, {}}, to, CastType::accurate, {});
};
func_cast = get_cast_func();

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;
};
}

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