Merge branch 'master' into json-with-progress

This commit is contained in:
Alexey Milovidov 2024-11-18 03:50:09 +01:00
commit 851fea80d0
116 changed files with 2565 additions and 921 deletions

View File

@ -122,7 +122,7 @@ Default value: `0`.
### s3queue_polling_min_timeout_ms {#polling_min_timeout_ms} ### s3queue_polling_min_timeout_ms {#polling_min_timeout_ms}
Minimal timeout before next polling (in milliseconds). Specifies the minimum time, in milliseconds, that ClickHouse waits before making the next polling attempt.
Possible values: Possible values:
@ -132,7 +132,7 @@ Default value: `1000`.
### s3queue_polling_max_timeout_ms {#polling_max_timeout_ms} ### s3queue_polling_max_timeout_ms {#polling_max_timeout_ms}
Maximum timeout before next polling (in milliseconds). Defines the maximum time, in milliseconds, that ClickHouse waits before initiating the next polling attempt.
Possible values: Possible values:
@ -142,7 +142,7 @@ Default value: `10000`.
### s3queue_polling_backoff_ms {#polling_backoff_ms} ### s3queue_polling_backoff_ms {#polling_backoff_ms}
Polling backoff (in milliseconds). Determines the additional wait time added to the previous polling interval when no new files are found. The next poll occurs after the sum of the previous interval and this backoff value, or the maximum interval, whichever is lower.
Possible values: Possible values:

View File

@ -10,6 +10,11 @@ The engine inherits from [MergeTree](../../../engines/table-engines/mergetree-fa
You can use `AggregatingMergeTree` tables for incremental data aggregation, including for aggregated materialized views. You can use `AggregatingMergeTree` tables for incremental data aggregation, including for aggregated materialized views.
You can see an example of how to use the AggregatingMergeTree and Aggregate functions in the below video:
<div class='vimeo-container'>
<iframe width="1030" height="579" src="https://www.youtube.com/embed/pryhI4F_zqQ" title="Aggregation States in ClickHouse" frameborder="0" allow="accelerometer; autoplay; clipboard-write; encrypted-media; gyroscope; picture-in-picture; web-share" referrerpolicy="strict-origin-when-cross-origin" allowfullscreen></iframe>
</div>
The engine processes all columns with the following types: The engine processes all columns with the following types:
## [AggregateFunction](../../../sql-reference/data-types/aggregatefunction.md) ## [AggregateFunction](../../../sql-reference/data-types/aggregatefunction.md)

View File

@ -211,7 +211,7 @@ Number of threads in the server of the replicas communication protocol (without
The difference in time the thread for calculation of the asynchronous metrics was scheduled to wake up and the time it was in fact, woken up. A proxy-indicator of overall system latency and responsiveness. The difference in time the thread for calculation of the asynchronous metrics was scheduled to wake up and the time it was in fact, woken up. A proxy-indicator of overall system latency and responsiveness.
### LoadAverage_*N* ### LoadAverage*N*
The whole system load, averaged with exponential smoothing over 1 minute. The load represents the number of threads across all the processes (the scheduling entities of the OS kernel), that are currently running by CPU or waiting for IO, or ready to run but not being scheduled at this point of time. This number includes all the processes, not only clickhouse-server. The number can be greater than the number of CPU cores, if the system is overloaded, and many processes are ready to run but waiting for CPU or IO. The whole system load, averaged with exponential smoothing over 1 minute. The load represents the number of threads across all the processes (the scheduling entities of the OS kernel), that are currently running by CPU or waiting for IO, or ready to run but not being scheduled at this point of time. This number includes all the processes, not only clickhouse-server. The number can be greater than the number of CPU cores, if the system is overloaded, and many processes are ready to run but waiting for CPU or IO.

View File

@ -75,7 +75,7 @@ FROM t_null_big
└────────────────────┴─────────────────────┘ └────────────────────┴─────────────────────┘
``` ```
Also you can use [Tuple](/docs/en/sql-reference/data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. Also you can use [Tuple](/docs/en/sql-reference/data-types/tuple.md) to work around NULL skipping behavior. A `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value.
```sql ```sql
SELECT SELECT
@ -110,7 +110,7 @@ GROUP BY v
└──────┴─────────┴──────────┘ └──────┴─────────┴──────────┘
``` ```
And here is an example of of first_value with `RESPECT NULLS` where we can see that NULL inputs are respected and it will return the first value read, whether it's NULL or not: And here is an example of first_value with `RESPECT NULLS` where we can see that NULL inputs are respected and it will return the first value read, whether it's NULL or not:
```sql ```sql
SELECT SELECT

View File

@ -5,7 +5,15 @@ sidebar_position: 102
# any # any
Selects the first encountered value of a column, ignoring any `NULL` values. Selects the first encountered value of a column.
:::warning
As a query can be executed in arbitrary order, the result of this function is non-deterministic.
If you need an arbitrary but deterministic result, use functions [`min`](../reference/min.md) or [`max`](../reference/max.md).
:::
By default, the function never returns NULL, i.e. ignores NULL values in the input column.
However, if the function is used with the `RESPECT NULLS` modifier, it returns the first value reads no matter if NULL or not.
**Syntax** **Syntax**
@ -13,46 +21,51 @@ Selects the first encountered value of a column, ignoring any `NULL` values.
any(column) [RESPECT NULLS] any(column) [RESPECT NULLS]
``` ```
Aliases: `any_value`, [`first_value`](../reference/first_value.md). Aliases `any(column)` (without `RESPECT NULLS`)
- `any_value`
- [`first_value`](../reference/first_value.md).
Alias for `any(column) RESPECT NULLS`
- `anyRespectNulls`, `any_respect_nulls`
- `firstValueRespectNulls`, `first_value_respect_nulls`
- `anyValueRespectNulls`, `any_value_respect_nulls`
**Parameters** **Parameters**
- `column`: The column name. - `column`: The column name.
**Returned value** **Returned value**
:::note The first value encountered.
Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not.
:::
:::note :::note
The return type of the function is the same as the input, except for LowCardinality which is discarded. This means that given no rows as input it will return the default value of that type (0 for integers, or Null for a Nullable() column). You might use the `-OrNull` [combinator](../../../sql-reference/aggregate-functions/combinators.md) ) to modify this behaviour. The return type of the function is the same as the input, except for LowCardinality which is discarded.
::: This means that given no rows as input it will return the default value of that type (0 for integers, or Null for a Nullable() column).
You might use the `-OrNull` [combinator](../../../sql-reference/aggregate-functions/combinators.md) ) to modify this behaviour.
:::warning
The query can be executed in any order and even in a different order each time, so the result of this function is indeterminate.
To get a determinate result, you can use the [`min`](../reference/min.md) or [`max`](../reference/max.md) function instead of `any`.
::: :::
**Implementation details** **Implementation details**
In some cases, you can rely on the order of execution. This applies to cases when `SELECT` comes from a subquery that uses `ORDER BY`. In some cases, you can rely on the order of execution.
This applies to cases when `SELECT` comes from a subquery that uses `ORDER BY`.
When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. To get behavior like in MySQL, you can put the other columns in the `any` aggregate function. When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions.
In other words, each column selected from the table must be used either in keys or inside aggregate functions.
To get behavior like in MySQL, you can put the other columns in the `any` aggregate function.
**Example** **Example**
Query: Query:
```sql ```sql
CREATE TABLE any_nulls (city Nullable(String)) ENGINE=Log; CREATE TABLE tab (city Nullable(String)) ENGINE=Memory;
INSERT INTO any_nulls (city) VALUES (NULL), ('Amsterdam'), ('New York'), ('Tokyo'), ('Valencia'), (NULL); INSERT INTO tab (city) VALUES (NULL), ('Amsterdam'), ('New York'), ('Tokyo'), ('Valencia'), (NULL);
SELECT any(city) FROM any_nulls; SELECT any(city), anyRespectNulls(city) FROM tab;
``` ```
```response ```response
┌─any(city)─┐ ┌─any(city)─┬─anyRespectNulls(city)─
│ Amsterdam │ │ Amsterdam │ ᴺᵁᴸᴸ │
└───────────┘ └───────────┴───────────────────────
``` ```

View File

@ -5,7 +5,15 @@ sidebar_position: 105
# anyLast # anyLast
Selects the last value encountered, ignoring any `NULL` values by default. The result is just as indeterminate as for the [any](../../../sql-reference/aggregate-functions/reference/any.md) function. Selects the last encountered value of a column.
:::warning
As a query can be executed in arbitrary order, the result of this function is non-deterministic.
If you need an arbitrary but deterministic result, use functions [`min`](../reference/min.md) or [`max`](../reference/max.md).
:::
By default, the function never returns NULL, i.e. ignores NULL values in the input column.
However, if the function is used with the `RESPECT NULLS` modifier, it returns the first value reads no matter if NULL or not.
**Syntax** **Syntax**
@ -13,12 +21,15 @@ Selects the last value encountered, ignoring any `NULL` values by default. The r
anyLast(column) [RESPECT NULLS] anyLast(column) [RESPECT NULLS]
``` ```
**Parameters** Alias `anyLast(column)` (without `RESPECT NULLS`)
- `column`: The column name. - [`last_value`](../reference/last_value.md).
:::note Aliases for `anyLast(column) RESPECT NULLS`
Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the last value passed, regardless of whether it is `NULL` or not. - `anyLastRespectNulls`, `anyLast_respect_nulls`
::: - `lastValueRespectNulls`, `last_value_respect_nulls`
**Parameters**
- `column`: The column name.
**Returned value** **Returned value**
@ -29,15 +40,15 @@ Supports the `RESPECT NULLS` modifier after the function name. Using this modifi
Query: Query:
```sql ```sql
CREATE TABLE any_last_nulls (city Nullable(String)) ENGINE=Log; CREATE TABLE tab (city Nullable(String)) ENGINE=Memory;
INSERT INTO any_last_nulls (city) VALUES ('Amsterdam'),(NULL),('New York'),('Tokyo'),('Valencia'),(NULL); INSERT INTO tab (city) VALUES ('Amsterdam'),(NULL),('New York'),('Tokyo'),('Valencia'),(NULL);
SELECT anyLast(city) FROM any_last_nulls; SELECT anyLast(city), anyLastRespectNulls(city) FROM tab;
``` ```
```response ```response
┌─anyLast(city)─┐ ┌─anyLast(city)─┬─anyLastRespectNulls(city)─
│ Valencia │ │ Valencia │ ᴺᵁᴸᴸ │
└───────────────┘ └───────────────┴───────────────────────────
``` ```

View File

@ -6791,7 +6791,7 @@ parseDateTime(str[, format[, timezone]])
**Returned value(s)** **Returned value(s)**
Returns DateTime values parsed from input string according to a MySQL style format string. Return a [DateTime](../data-types/datetime.md) value parsed from the input string according to a MySQL-style format string.
**Supported format specifiers** **Supported format specifiers**
@ -6840,7 +6840,7 @@ parseDateTimeInJodaSyntax(str[, format[, timezone]])
**Returned value(s)** **Returned value(s)**
Returns DateTime values parsed from input string according to a Joda style format. Return a [DateTime](../data-types/datetime.md) value parsed from the input string according to a Joda-style format string.
**Supported format specifiers** **Supported format specifiers**
@ -6885,7 +6885,8 @@ parseDateTime64(str[, format[, timezone]])
**Returned value(s)** **Returned value(s)**
Returns [DateTime64](../data-types/datetime64.md) type values parsed from input string according to a MySQL style format string. Return a [DateTime64](../data-types/datetime64.md) value parsed from the input string according to a MySQL-style format string.
The precision of the returned value is 6.
## parseDateTime64OrZero ## parseDateTime64OrZero
@ -6913,7 +6914,8 @@ parseDateTime64InJodaSyntax(str[, format[, timezone]])
**Returned value(s)** **Returned value(s)**
Returns [DateTime64](../data-types/datetime64.md) type values parsed from input string according to a joda style format string. Return a [DateTime64](../data-types/datetime64.md) value parsed from the input string according to a Joda-style format string.
The precision of the returned value equal to the number of `S` placeholders in the format string (but at most 6).
## parseDateTime64InJodaSyntaxOrZero ## parseDateTime64InJodaSyntaxOrZero

View File

@ -15,7 +15,7 @@ first_value (column_name) [[RESPECT NULLS] | [IGNORE NULLS]]
OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column]
[ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name])
FROM table_name FROM table_name
WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) WINDOW window_name as ([PARTITION BY grouping_column] [ORDER BY sorting_column])
``` ```
Alias: `any`. Alias: `any`.
@ -23,6 +23,8 @@ Alias: `any`.
:::note :::note
Using the optional modifier `RESPECT NULLS` after `first_value(column_name)` will ensure that `NULL` arguments are not skipped. Using the optional modifier `RESPECT NULLS` after `first_value(column_name)` will ensure that `NULL` arguments are not skipped.
See [NULL processing](../aggregate-functions/index.md/#null-processing) for more information. See [NULL processing](../aggregate-functions/index.md/#null-processing) for more information.
Alias: `firstValueRespectNulls`
::: :::
For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax).
@ -48,7 +50,7 @@ CREATE TABLE salaries
) )
Engine = Memory; Engine = Memory;
INSERT INTO salaries FORMAT Values INSERT INTO salaries FORMAT VALUES
('Port Elizabeth Barbarians', 'Gary Chen', 196000, 'F'), ('Port Elizabeth Barbarians', 'Gary Chen', 196000, 'F'),
('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'),
('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'), ('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'),

View File

@ -23,6 +23,8 @@ Alias: `anyLast`.
:::note :::note
Using the optional modifier `RESPECT NULLS` after `first_value(column_name)` will ensure that `NULL` arguments are not skipped. Using the optional modifier `RESPECT NULLS` after `first_value(column_name)` will ensure that `NULL` arguments are not skipped.
See [NULL processing](../aggregate-functions/index.md/#null-processing) for more information. See [NULL processing](../aggregate-functions/index.md/#null-processing) for more information.
Alias: `lastValueRespectNulls`
::: :::
For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax).
@ -33,7 +35,7 @@ For more detail on window function syntax see: [Window Functions - Syntax](./ind
**Example** **Example**
In this example the `last_value` function is used to find the highest paid footballer from a fictional dataset of salaries of Premier League football players. In this example the `last_value` function is used to find the lowest paid footballer from a fictional dataset of salaries of Premier League football players.
Query: Query:
@ -48,7 +50,7 @@ CREATE TABLE salaries
) )
Engine = Memory; Engine = Memory;
INSERT INTO salaries FORMAT Values INSERT INTO salaries FORMAT VALUES
('Port Elizabeth Barbarians', 'Gary Chen', 196000, 'F'), ('Port Elizabeth Barbarians', 'Gary Chen', 196000, 'F'),
('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'),
('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'), ('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'),

View File

@ -221,11 +221,16 @@ void registerAggregateFunctionsAnyRespectNulls(AggregateFunctionFactory & factor
= {.returns_default_when_only_null = false, .is_order_dependent = true, .is_window_function = true}; = {.returns_default_when_only_null = false, .is_order_dependent = true, .is_window_function = true};
factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls}); factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls});
factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("anyRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive);
factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("firstValueRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive);
factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("anyValueRespectNulls", "any_respect_nulls", AggregateFunctionFactory::Case::Sensitive);
factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls}); factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls});
factory.registerAlias("anyLastRespectNulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Sensitive);
factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
factory.registerAlias("lastValueRespectNulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Sensitive);
/// Must happen after registering any and anyLast /// Must happen after registering any and anyLast
factory.registerNullsActionTransformation("any", "any_respect_nulls"); factory.registerNullsActionTransformation("any", "any_respect_nulls");

View File

@ -685,13 +685,13 @@ void BackupCoordinationStageSync::cancelQueryIfError()
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
if (!state.host_with_error) if (state.host_with_error)
return; exception = state.hosts.at(*state.host_with_error).exception;
exception = state.hosts.at(*state.host_with_error).exception;
} }
chassert(exception); if (!exception)
return;
process_list_element->cancelQuery(false, exception); process_list_element->cancelQuery(false, exception);
state_changed.notify_all(); state_changed.notify_all();
} }
@ -741,6 +741,11 @@ void BackupCoordinationStageSync::cancelQueryIfDisconnectedTooLong()
if (!exception) if (!exception)
return; return;
/// In this function we only pass the new `exception` (about that the connection was lost) to `process_list_element`.
/// We don't try to create the 'error' node here (because this function is called from watchingThread() and
/// we don't want the watching thread to try waiting here for retries or a reconnection).
/// Also we don't set the `state.host_with_error` field here because `state.host_with_error` can only be set
/// AFTER creating the 'error' node (see the comment for `State`).
process_list_element->cancelQuery(false, exception); process_list_element->cancelQuery(false, exception);
state_changed.notify_all(); state_changed.notify_all();
} }
@ -870,6 +875,9 @@ bool BackupCoordinationStageSync::checkIfHostsReachStage(const Strings & hosts,
continue; continue;
} }
if (state.host_with_error)
std::rethrow_exception(state.hosts.at(*state.host_with_error).exception);
if (host_info.finished) if (host_info.finished)
throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE,
"{} finished without coming to stage {}", getHostDesc(host), stage_to_wait); "{} finished without coming to stage {}", getHostDesc(host), stage_to_wait);
@ -1150,6 +1158,9 @@ bool BackupCoordinationStageSync::checkIfOtherHostsFinish(
if ((host == current_host) || host_info.finished) if ((host == current_host) || host_info.finished)
continue; continue;
if (throw_if_error && state.host_with_error)
std::rethrow_exception(state.hosts.at(*state.host_with_error).exception);
String reason_text = reason.empty() ? "" : (" " + reason); String reason_text = reason.empty() ? "" : (" " + reason);
String host_status; String host_status;

View File

@ -197,6 +197,9 @@ private:
}; };
/// Information about all the host participating in the current BACKUP or RESTORE operation. /// Information about all the host participating in the current BACKUP or RESTORE operation.
/// This information is read from ZooKeeper.
/// To simplify the programming logic `state` can only be updated AFTER changing corresponding nodes in ZooKeeper
/// (for example, first we create the 'error' node, and only after that we set or read from ZK the `state.host_with_error` field).
struct State struct State
{ {
std::map<String /* host */, HostInfo> hosts; /// std::map because we need to compare states std::map<String /* host */, HostInfo> hosts; /// std::map because we need to compare states

View File

@ -52,6 +52,7 @@ private:
explicit ColumnVector(const size_t n) : data(n) {} explicit ColumnVector(const size_t n) : data(n) {}
ColumnVector(const size_t n, const ValueType x) : data(n, x) {} ColumnVector(const size_t n, const ValueType x) : data(n, x) {}
ColumnVector(const ColumnVector & src) : data(src.data.begin(), src.data.end()) {} ColumnVector(const ColumnVector & src) : data(src.data.begin(), src.data.end()) {}
ColumnVector(Container::const_iterator begin, Container::const_iterator end) : data(begin, end) { }
/// Sugar constructor. /// Sugar constructor.
ColumnVector(std::initializer_list<T> il) : data{il} {} ColumnVector(std::initializer_list<T> il) : data{il} {}

View File

@ -10,6 +10,7 @@
#include <fcntl.h> #include <fcntl.h>
#include <algorithm> #include <algorithm>
namespace DB namespace DB
{ {

View File

@ -150,6 +150,9 @@ Squash blocks passed to the external table to a specified size in bytes, if bloc
)", 0) \ )", 0) \
DECLARE(UInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, R"( DECLARE(UInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, R"(
Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited. Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited.
)", 0) \
DECLARE(UInt64, min_joined_block_size_bytes, 524288, R"(
Minimum block size for JOIN result (if join algorithm supports it). 0 means unlimited.
)", 0) \ )", 0) \
DECLARE(UInt64, max_insert_threads, 0, R"( DECLARE(UInt64, max_insert_threads, 0, R"(
The maximum number of threads to execute the `INSERT SELECT` query. The maximum number of threads to execute the `INSERT SELECT` query.

View File

@ -80,6 +80,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."}, {"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."},
{"query_plan_merge_filters", false, true, "Allow to merge filters in the query plan. This is required to properly support filter-push-down with a new analyzer."}, {"query_plan_merge_filters", false, true, "Allow to merge filters in the query plan. This is required to properly support filter-push-down with a new analyzer."},
{"parallel_replicas_local_plan", false, true, "Use local plan for local replica in a query with parallel replicas"}, {"parallel_replicas_local_plan", false, true, "Use local plan for local replica in a query with parallel replicas"},
{"min_joined_block_size_bytes", 524288, 524288, "New setting."},
{"allow_experimental_bfloat16_type", false, false, "Add new experimental BFloat16 type"}, {"allow_experimental_bfloat16_type", false, false, "Add new experimental BFloat16 type"},
{"filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit", 1, 1, "Rename of setting skip_download_if_exceeds_query_cache_limit"}, {"filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit", 1, 1, "Rename of setting skip_download_if_exceeds_query_cache_limit"},
{"filesystem_cache_prefer_bigger_buffer_size", true, true, "New setting"}, {"filesystem_cache_prefer_bigger_buffer_size", true, true, "New setting"},

View File

@ -64,6 +64,7 @@ constexpr time_t MAX_DATETIME_DAY_NUM = 49710; // 2106-02-07
/// This factor transformation will say that the function is monotone everywhere. /// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform struct ZeroTransform
{ {
static constexpr auto name = "Zero";
static UInt16 execute(Int64, const DateLUTImpl &) { return 0; } static UInt16 execute(Int64, const DateLUTImpl &) { return 0; }
static UInt16 execute(UInt32, const DateLUTImpl &) { return 0; } static UInt16 execute(UInt32, const DateLUTImpl &) { return 0; }
static UInt16 execute(Int32, const DateLUTImpl &) { return 0; } static UInt16 execute(Int32, const DateLUTImpl &) { return 0; }

View File

@ -56,6 +56,21 @@ public:
: is_not_monotonic; : is_not_monotonic;
} }
if (checkAndGetDataType<DataTypeDateTime64>(&type))
{
const auto & left_date_time = left.safeGet<DateTime64>();
TransformDateTime64<typename Transform::FactorTransform> transformer_left(left_date_time.getScale());
const auto & right_date_time = right.safeGet<DateTime64>();
TransformDateTime64<typename Transform::FactorTransform> transformer_right(right_date_time.getScale());
return transformer_left.execute(left_date_time.getValue(), date_lut)
== transformer_right.execute(right_date_time.getValue(), date_lut)
? is_monotonic
: is_not_monotonic;
}
return Transform::FactorTransform::execute(UInt32(left.safeGet<UInt64>()), date_lut) return Transform::FactorTransform::execute(UInt32(left.safeGet<UInt64>()), date_lut)
== Transform::FactorTransform::execute(UInt32(right.safeGet<UInt64>()), date_lut) == Transform::FactorTransform::execute(UInt32(right.safeGet<UInt64>()), date_lut)
? is_monotonic ? is_monotonic

View File

@ -457,16 +457,16 @@ namespace
return {}; return {};
} }
[[nodiscard]] void setScale(UInt32 scale_, ParseSyntax parse_syntax_)
VoidOrError setScale(UInt8 scale_, ParseSyntax parse_syntax_)
{ {
/// Because the scale argument for parseDateTime*() is constant, always throw an exception (don't allow continuing to the
/// next row like in other set* functions)
if (parse_syntax_ == ParseSyntax::MySQL && scale_ != 6) if (parse_syntax_ == ParseSyntax::MySQL && scale_ != 6)
RETURN_ERROR(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for scale must be 6 for MySQL parse syntax", std::to_string(scale_)) throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Precision {} is invalid (must be 6)", scale);
else if (parse_syntax_ == ParseSyntax::Joda && scale_ > 6) else if (parse_syntax_ == ParseSyntax::Joda && scale_ > 6)
RETURN_ERROR(ErrorCodes::CANNOT_PARSE_DATETIME, "Value {} for scale must be in the range [0, 6] for Joda syntax", std::to_string(scale_)) throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Precision {} is invalid (must be [0, 6])", scale);
scale = scale_; scale = scale_;
return {};
} }
/// For debug /// For debug
@ -611,7 +611,6 @@ namespace
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
bool isVariadic() const override { return true; } bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; } size_t getNumberOfArguments() const override { return 0; }
@ -637,13 +636,13 @@ namespace
data_type = std::make_shared<DataTypeDateTime64>(6, time_zone_name); data_type = std::make_shared<DataTypeDateTime64>(6, time_zone_name);
else else
{ {
/// The precision of the return type is the number of 'S' placeholders.
String format = getFormat(arguments); String format = getFormat(arguments);
std::vector<Instruction> instructions = parseFormat(format); std::vector<Instruction> instructions = parseFormat(format);
/// How many 'S' characters does the format string contain? size_t s_count = 0;
UInt32 s_count = 0;
for (const auto & instruction : instructions) for (const auto & instruction : instructions)
{ {
const String fragment = instruction.getFragment(); const String & fragment = instruction.getFragment();
for (char c : fragment) for (char c : fragment)
{ {
if (c == 'S') if (c == 'S')
@ -654,7 +653,6 @@ namespace
if (s_count > 0) if (s_count > 0)
break; break;
} }
/// Use s_count as DateTime64's scale.
data_type = std::make_shared<DataTypeDateTime64>(s_count, time_zone_name); data_type = std::make_shared<DataTypeDateTime64>(s_count, time_zone_name);
} }
} }
@ -715,25 +713,18 @@ namespace
const String format = getFormat(arguments); const String format = getFormat(arguments);
const std::vector<Instruction> instructions = parseFormat(format); const std::vector<Instruction> instructions = parseFormat(format);
const auto & time_zone = getTimeZone(arguments); const auto & time_zone = getTimeZone(arguments);
/// Make datetime fit in a cache line. alignas(64) DateTime<error_handling> datetime; /// Make datetime fit in a cache line.
alignas(64) DateTime<error_handling> datetime;
for (size_t i = 0; i < input_rows_count; ++i) for (size_t i = 0; i < input_rows_count; ++i)
{ {
datetime.reset(); datetime.reset();
if constexpr (return_type == ReturnType::DateTime64)
datetime.setScale(scale, parse_syntax);
StringRef str_ref = col_str->getDataAt(i); StringRef str_ref = col_str->getDataAt(i);
Pos cur = str_ref.data; Pos cur = str_ref.data;
Pos end = str_ref.data + str_ref.size; Pos end = str_ref.data + str_ref.size;
bool error = false; bool error = false;
if constexpr (return_type == ReturnType::DateTime64)
{
if (auto result = datetime.setScale(static_cast<UInt8>(scale), parse_syntax); !result.has_value())
{
const ErrorCodeAndMessage & err = result.error();
throw Exception(err.error_code, "Invalid scale value: {}, {}", std::to_string(scale), err.error_message);
}
}
for (const auto & instruction : instructions) for (const auto & instruction : instructions)
{ {
if (auto result = instruction.perform(cur, end, datetime); result.has_value()) if (auto result = instruction.perform(cur, end, datetime); result.has_value())

View File

@ -3387,6 +3387,8 @@ UInt64 calculateCacheKey(const DB::ASTPtr & select_query)
SipHash hash; SipHash hash;
hash.update(select.tables()->getTreeHash(/*ignore_aliases=*/true)); hash.update(select.tables()->getTreeHash(/*ignore_aliases=*/true));
if (const auto prewhere = select.prewhere())
hash.update(prewhere->getTreeHash(/*ignore_aliases=*/true));
if (const auto where = select.where()) if (const auto where = select.where())
hash.update(where->getTreeHash(/*ignore_aliases=*/true)); hash.update(where->getTreeHash(/*ignore_aliases=*/true));
if (const auto group_by = select.groupBy()) if (const auto group_by = select.groupBy())

View File

@ -5,16 +5,21 @@
#include <Core/Names.h> #include <Core/Names.h>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
#include <DataTypes/DataTypeLowCardinality.h> #include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/Serializations/ISerialization.h>
#include <Interpreters/ConcurrentHashJoin.h> #include <Interpreters/ConcurrentHashJoin.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/HashJoin/ScatteredBlock.h>
#include <Interpreters/PreparedSets.h> #include <Interpreters/PreparedSets.h>
#include <Interpreters/TableJoin.h> #include <Interpreters/TableJoin.h>
#include <Interpreters/createBlockSelector.h> #include <Interpreters/createBlockSelector.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/DumpASTNode.h> #include <Parsers/DumpASTNode.h>
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Storages/SelectQueryInfo.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
@ -24,6 +29,12 @@
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <algorithm>
#include <numeric>
#include <vector>
using namespace DB;
namespace ProfileEvents namespace ProfileEvents
{ {
extern const Event HashJoinPreallocatedElementsInHashTables; extern const Event HashJoinPreallocatedElementsInHashTables;
@ -116,9 +127,7 @@ ConcurrentHashJoin::ConcurrentHashJoin(
auto inner_hash_join = std::make_shared<InternalHashJoin>(); auto inner_hash_join = std::make_shared<InternalHashJoin>();
inner_hash_join->data = std::make_unique<HashJoin>( inner_hash_join->data = std::make_unique<HashJoin>(
table_join_, right_sample_block, any_take_last_row_, reserve_size, fmt::format("concurrent{}", idx)); table_join_, right_sample_block, any_take_last_row_, reserve_size, fmt::format("concurrent{}", idx));
/// Non zero `max_joined_block_rows` allows to process block partially and return not processed part. inner_hash_join->data->setMaxJoinedBlockRows(table_join->maxJoinedBlockRows());
/// TODO: It's not handled properly in ConcurrentHashJoin case, so we set it to 0 to disable this feature.
inner_hash_join->data->setMaxJoinedBlockRows(0);
hash_joins[idx] = std::move(inner_hash_join); hash_joins[idx] = std::move(inner_hash_join);
}); });
} }
@ -165,10 +174,13 @@ ConcurrentHashJoin::~ConcurrentHashJoin()
} }
} }
bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block, bool check_limits) bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block_, bool check_limits)
{ {
Blocks dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block); /// We materialize columns here to avoid materializing them multiple times on different threads
/// (inside different `hash_join`-s) because the block will be shared.
Block right_block = hash_joins[0]->data->materializeColumnsFromRightBlock(right_block_);
auto dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, std::move(right_block));
size_t blocks_left = 0; size_t blocks_left = 0;
for (const auto & block : dispatched_blocks) for (const auto & block : dispatched_blocks)
{ {
@ -211,19 +223,52 @@ bool ConcurrentHashJoin::addBlockToJoin(const Block & right_block, bool check_li
void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr<ExtraBlock> & /*not_processed*/) void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr<ExtraBlock> & /*not_processed*/)
{ {
Blocks dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block); Blocks res;
ExtraScatteredBlocks extra_blocks;
joinBlock(block, extra_blocks, res);
chassert(!extra_blocks.rows());
block = concatenateBlocks(res);
}
void ConcurrentHashJoin::joinBlock(Block & block, ExtraScatteredBlocks & extra_blocks, std::vector<Block> & res)
{
ScatteredBlocks dispatched_blocks;
auto & remaining_blocks = extra_blocks.remaining_blocks;
if (extra_blocks.rows())
{
dispatched_blocks.swap(remaining_blocks);
}
else
{
hash_joins[0]->data->materializeColumnsFromLeftBlock(block);
dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, std::move(block));
}
block = {}; block = {};
/// Just in case, should be no-op always
remaining_blocks.resize(slots);
chassert(res.empty());
res.clear();
res.reserve(dispatched_blocks.size());
for (size_t i = 0; i < dispatched_blocks.size(); ++i) for (size_t i = 0; i < dispatched_blocks.size(); ++i)
{ {
std::shared_ptr<ExtraBlock> none_extra_block; std::shared_ptr<ExtraBlock> none_extra_block;
auto & hash_join = hash_joins[i]; auto & hash_join = hash_joins[i];
auto & dispatched_block = dispatched_blocks[i]; auto & dispatched_block = dispatched_blocks[i];
hash_join->data->joinBlock(dispatched_block, none_extra_block); if (dispatched_block && (i == 0 || dispatched_block.rows()))
hash_join->data->joinBlock(dispatched_block, remaining_blocks[i]);
if (none_extra_block && !none_extra_block->empty()) if (none_extra_block && !none_extra_block->empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty"); throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty");
} }
for (size_t i = 0; i < dispatched_blocks.size(); ++i)
block = concatenateBlocks(dispatched_blocks); {
auto & dispatched_block = dispatched_blocks[i];
if (dispatched_block && (i == 0 || dispatched_block.rows()))
res.emplace_back(std::move(dispatched_block).getSourceBlock());
}
} }
void ConcurrentHashJoin::checkTypesOfKeys(const Block & block) const void ConcurrentHashJoin::checkTypesOfKeys(const Block & block) const
@ -302,10 +347,9 @@ static ALWAYS_INLINE IColumn::Selector hashToSelector(const WeakHash32 & hash, s
return selector; return selector;
} }
IColumn::Selector ConcurrentHashJoin::selectDispatchBlock(const Strings & key_columns_names, const Block & from_block) IColumn::Selector selectDispatchBlock(size_t num_shards, const Strings & key_columns_names, const Block & from_block)
{ {
size_t num_rows = from_block.rows(); size_t num_rows = from_block.rows();
size_t num_shards = hash_joins.size();
WeakHash32 hash(num_rows); WeakHash32 hash(num_rows);
for (const auto & key_name : key_columns_names) for (const auto & key_name : key_columns_names)
@ -317,40 +361,101 @@ IColumn::Selector ConcurrentHashJoin::selectDispatchBlock(const Strings & key_co
return hashToSelector(hash, num_shards); return hashToSelector(hash, num_shards);
} }
Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block) ScatteredBlocks scatterBlocksByCopying(size_t num_shards, const IColumn::Selector & selector, const Block & from_block)
{ {
/// TODO: use JoinCommon::scatterBlockByHash Blocks blocks(num_shards);
size_t num_shards = hash_joins.size();
size_t num_cols = from_block.columns();
IColumn::Selector selector = selectDispatchBlock(key_columns_names, from_block);
Blocks result(num_shards);
for (size_t i = 0; i < num_shards; ++i) for (size_t i = 0; i < num_shards; ++i)
result[i] = from_block.cloneEmpty(); blocks[i] = from_block.cloneEmpty();
for (size_t i = 0; i < num_cols; ++i) for (size_t i = 0; i < from_block.columns(); ++i)
{ {
auto dispatched_columns = from_block.getByPosition(i).column->scatter(num_shards, selector); auto dispatched_columns = from_block.getByPosition(i).column->scatter(num_shards, selector);
assert(result.size() == dispatched_columns.size()); chassert(blocks.size() == dispatched_columns.size());
for (size_t block_index = 0; block_index < num_shards; ++block_index) for (size_t block_index = 0; block_index < num_shards; ++block_index)
{ {
result[block_index].getByPosition(i).column = std::move(dispatched_columns[block_index]); blocks[block_index].getByPosition(i).column = std::move(dispatched_columns[block_index]);
} }
} }
ScatteredBlocks result;
result.reserve(num_shards);
for (size_t i = 0; i < num_shards; ++i)
result.emplace_back(std::move(blocks[i]));
return result; return result;
} }
UInt64 calculateCacheKey(std::shared_ptr<TableJoin> & table_join, const QueryTreeNodePtr & right_table_expression) ScatteredBlocks scatterBlocksWithSelector(size_t num_shards, const IColumn::Selector & selector, const Block & from_block)
{ {
std::vector<ScatteredBlock::IndexesPtr> selectors(num_shards);
for (size_t i = 0; i < num_shards; ++i)
{
selectors[i] = ScatteredBlock::Indexes::create();
selectors[i]->reserve(selector.size() / num_shards + 1);
}
for (size_t i = 0; i < selector.size(); ++i)
{
const size_t shard = selector[i];
selectors[shard]->getData().push_back(i);
}
ScatteredBlocks result;
result.reserve(num_shards);
for (size_t i = 0; i < num_shards; ++i)
result.emplace_back(from_block, std::move(selectors[i]));
return result;
}
ScatteredBlocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, Block && from_block)
{
size_t num_shards = hash_joins.size();
if (num_shards == 1)
{
ScatteredBlocks res;
res.emplace_back(std::move(from_block));
return res;
}
IColumn::Selector selector = selectDispatchBlock(num_shards, key_columns_names, from_block);
/// With zero-copy approach we won't copy the source columns, but will create a new one with indices.
/// This is not beneficial when the whole set of columns is e.g. a single small column.
constexpr auto threshold = sizeof(IColumn::Selector::value_type);
const auto & data_types = from_block.getDataTypes();
const bool use_zero_copy_approach
= std::accumulate(
data_types.begin(),
data_types.end(),
0u,
[](size_t sum, const DataTypePtr & type)
{ return sum + (type->haveMaximumSizeOfValue() ? type->getMaximumSizeOfValueInMemory() : threshold + 1); })
> threshold;
return use_zero_copy_approach ? scatterBlocksWithSelector(num_shards, selector, from_block)
: scatterBlocksByCopying(num_shards, selector, from_block);
}
UInt64 calculateCacheKey(
std::shared_ptr<TableJoin> & table_join, const QueryTreeNodePtr & right_table_expression, const SelectQueryInfo & select_query_info)
{
const auto * select = select_query_info.query->as<DB::ASTSelectQuery>();
if (!select)
return 0;
IQueryTreeNode::HashState hash; IQueryTreeNode::HashState hash;
if (const auto prewhere = select->prewhere())
hash.update(prewhere->getTreeHash(/*ignore_aliases=*/true));
if (const auto where = select->where())
hash.update(where->getTreeHash(/*ignore_aliases=*/true));
chassert(right_table_expression); chassert(right_table_expression);
hash.update(right_table_expression->getTreeHash()); hash.update(right_table_expression->getTreeHash());
chassert(table_join && table_join->oneDisjunct()); chassert(table_join && table_join->oneDisjunct());
const auto keys const auto keys
= NameOrderedSet{table_join->getClauses().at(0).key_names_right.begin(), table_join->getClauses().at(0).key_names_right.end()}; = NameOrderedSet{table_join->getClauses().at(0).key_names_right.begin(), table_join->getClauses().at(0).key_names_right.end()};
for (const auto & name : keys) for (const auto & name : keys)
hash.update(name); hash.update(name);
return hash.get64(); return hash.get64();
} }
} }

View File

@ -1,13 +1,11 @@
#pragma once #pragma once
#include <condition_variable>
#include <memory> #include <memory>
#include <optional>
#include <Analyzer/IQueryTreeNode.h> #include <Analyzer/IQueryTreeNode.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/HashTablesStatistics.h>
#include <Interpreters/HashJoin/HashJoin.h> #include <Interpreters/HashJoin/HashJoin.h>
#include <Interpreters/HashTablesStatistics.h>
#include <Interpreters/IJoin.h> #include <Interpreters/IJoin.h>
#include <base/defines.h> #include <base/defines.h>
#include <base/types.h> #include <base/types.h>
@ -17,6 +15,8 @@
namespace DB namespace DB
{ {
struct SelectQueryInfo;
/** /**
* Can run addBlockToJoin() parallelly to speedup the join process. On test, it almose linear speedup by * Can run addBlockToJoin() parallelly to speedup the join process. On test, it almose linear speedup by
* the degree of parallelism. * the degree of parallelism.
@ -47,7 +47,7 @@ public:
std::string getName() const override { return "ConcurrentHashJoin"; } std::string getName() const override { return "ConcurrentHashJoin"; }
const TableJoin & getTableJoin() const override { return *table_join; } const TableJoin & getTableJoin() const override { return *table_join; }
bool addBlockToJoin(const Block & block, bool check_limits) override; bool addBlockToJoin(const Block & right_block_, bool check_limits) override;
void checkTypesOfKeys(const Block & block) const override; void checkTypesOfKeys(const Block & block) const override;
void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) override; void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) override;
void setTotals(const Block & block) override; void setTotals(const Block & block) override;
@ -57,6 +57,9 @@ public:
bool alwaysReturnsEmptySet() const override; bool alwaysReturnsEmptySet() const override;
bool supportParallelJoin() const override { return true; } bool supportParallelJoin() const override { return true; }
bool isScatteredJoin() const override { return true; }
void joinBlock(Block & block, ExtraScatteredBlocks & extra_blocks, std::vector<Block> & res) override;
IBlocksStreamPtr IBlocksStreamPtr
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override;
@ -78,9 +81,9 @@ private:
std::mutex totals_mutex; std::mutex totals_mutex;
Block totals; Block totals;
IColumn::Selector selectDispatchBlock(const Strings & key_columns_names, const Block & from_block); ScatteredBlocks dispatchBlock(const Strings & key_columns_names, Block && from_block);
Blocks dispatchBlock(const Strings & key_columns_names, const Block & from_block);
}; };
UInt64 calculateCacheKey(std::shared_ptr<TableJoin> & table_join, const QueryTreeNodePtr & right_table_expression); UInt64 calculateCacheKey(
std::shared_ptr<TableJoin> & table_join, const QueryTreeNodePtr & right_table_expression, const SelectQueryInfo & select_query_info);
} }

View File

@ -3,14 +3,16 @@
namespace DB namespace DB
{ {
JoinOnKeyColumns::JoinOnKeyColumns(const Block & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_) JoinOnKeyColumns::JoinOnKeyColumns(
: key_names(key_names_) const ScatteredBlock & block_, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_)
, materialized_keys_holder(JoinCommon::materializeColumns( : block(block_)
block, key_names)) /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. , key_names(key_names_)
/// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them.
, materialized_keys_holder(JoinCommon::materializeColumns(block.getSourceBlock(), key_names))
, key_columns(JoinCommon::getRawPointers(materialized_keys_holder)) , key_columns(JoinCommon::getRawPointers(materialized_keys_holder))
, null_map(nullptr) , null_map(nullptr)
, null_map_holder(extractNestedColumnsAndNullMap(key_columns, null_map)) , null_map_holder(extractNestedColumnsAndNullMap(key_columns, null_map))
, join_mask_column(JoinCommon::getColumnAsMask(block, cond_column_name)) , join_mask_column(JoinCommon::getColumnAsMask(block.getSourceBlock(), cond_column_name))
, key_sizes(key_sizes_) , key_sizes(key_sizes_)
{ {
} }

View File

@ -1,4 +1,6 @@
#pragma once #pragma once
#include <Core/Defines.h>
#include <Interpreters/HashJoin/HashJoin.h> #include <Interpreters/HashJoin/HashJoin.h>
#include <Interpreters/TableJoin.h> #include <Interpreters/TableJoin.h>
@ -14,6 +16,8 @@ using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
struct JoinOnKeyColumns struct JoinOnKeyColumns
{ {
const ScatteredBlock & block;
Names key_names; Names key_names;
Columns materialized_keys_holder; Columns materialized_keys_holder;
@ -27,9 +31,13 @@ struct JoinOnKeyColumns
Sizes key_sizes; Sizes key_sizes;
explicit JoinOnKeyColumns(const Block & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_); JoinOnKeyColumns(
const ScatteredBlock & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_);
bool isRowFiltered(size_t i) const { return join_mask_column.isRowFiltered(i); } bool isRowFiltered(size_t i) const
{
return join_mask_column.isRowFiltered(i);
}
}; };
template <bool lazy> template <bool lazy>
@ -54,7 +62,7 @@ public:
}; };
AddedColumns( AddedColumns(
const Block & left_block_, const ScatteredBlock & left_block_,
const Block & block_with_columns_to_add, const Block & block_with_columns_to_add,
const Block & saved_block_sample, const Block & saved_block_sample,
const HashJoin & join, const HashJoin & join,
@ -62,10 +70,11 @@ public:
ExpressionActionsPtr additional_filter_expression_, ExpressionActionsPtr additional_filter_expression_,
bool is_asof_join, bool is_asof_join,
bool is_join_get_) bool is_join_get_)
: left_block(left_block_) : src_block(left_block_)
, left_block(left_block_.getSourceBlock())
, join_on_keys(join_on_keys_) , join_on_keys(join_on_keys_)
, additional_filter_expression(additional_filter_expression_) , additional_filter_expression(additional_filter_expression_)
, rows_to_add(left_block.rows()) , rows_to_add(left_block_.rows())
, join_data_avg_perkey_rows(join.getJoinedData()->avgPerKeyRows()) , join_data_avg_perkey_rows(join.getJoinedData()->avgPerKeyRows())
, output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold()) , output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold())
, join_data_sorted(join.getJoinedData()->sorted) , join_data_sorted(join.getJoinedData()->sorted)
@ -139,6 +148,7 @@ public:
static constexpr bool isLazy() { return lazy; } static constexpr bool isLazy() { return lazy; }
const ScatteredBlock & src_block;
Block left_block; Block left_block;
std::vector<JoinOnKeyColumns> join_on_keys; std::vector<JoinOnKeyColumns> join_on_keys;
ExpressionActionsPtr additional_filter_expression; ExpressionActionsPtr additional_filter_expression;
@ -159,7 +169,7 @@ public:
return; return;
/// Do not allow big allocations when user set max_joined_block_rows to huge value /// Do not allow big allocations when user set max_joined_block_rows to huge value
size_t reserve_size = std::min<size_t>(max_joined_block_rows, DEFAULT_BLOCK_SIZE * 2); size_t reserve_size = std::min<size_t>(max_joined_block_rows, rows_to_add * 2);
if (need_replicate) if (need_replicate)
/// Reserve 10% more space for columns, because some rows can be repeated /// Reserve 10% more space for columns, because some rows can be repeated
@ -218,7 +228,7 @@ private:
void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name) void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name)
{ {
columns.push_back(src_column.column->cloneEmpty()); columns.push_back(src_column.column->cloneEmpty());
columns.back()->reserve(src_column.column->size()); columns.back()->reserve(rows_to_add);
type_name.emplace_back(src_column.type, src_column.name, qualified_name); type_name.emplace_back(src_column.type, src_column.name, qualified_name);
} }

View File

@ -13,24 +13,23 @@
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/HashJoin/HashJoin.h> #include <Interpreters/HashJoin/HashJoin.h>
#include <Interpreters/JoinUtils.h> #include <Interpreters/JoinUtils.h>
#include <Interpreters/TableJoin.h>
#include <Interpreters/joinDispatch.h>
#include <Interpreters/NullableUtils.h> #include <Interpreters/NullableUtils.h>
#include <Interpreters/RowRefs.h> #include <Interpreters/RowRefs.h>
#include <Interpreters/TableJoin.h>
#include <Interpreters/joinDispatch.h>
#include <Interpreters/TemporaryDataOnDisk.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h> #include <Common/assert_cast.h>
#include <Common/formatReadable.h> #include <Common/formatReadable.h>
#include <Interpreters/TemporaryDataOnDisk.h> #include <Common/typeid_cast.h>
#include <Interpreters/HashJoin/HashJoinMethods.h> #include <Interpreters/HashJoin/HashJoinMethods.h>
#include <Interpreters/HashJoin/JoinUsedFlags.h> #include <Interpreters/HashJoin/JoinUsedFlags.h>
@ -40,16 +39,16 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int INCOMPATIBLE_TYPE_OF_JOIN; extern const int INCOMPATIBLE_TYPE_OF_JOIN;
extern const int UNSUPPORTED_JOIN_KEYS; extern const int UNSUPPORTED_JOIN_KEYS;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int SYNTAX_ERROR; extern const int SYNTAX_ERROR;
extern const int SET_SIZE_LIMIT_EXCEEDED; extern const int SET_SIZE_LIMIT_EXCEEDED;
extern const int TYPE_MISMATCH; extern const int TYPE_MISMATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int INVALID_JOIN_ON_EXPRESSION; extern const int INVALID_JOIN_ON_EXPRESSION;
} }
namespace namespace
@ -72,6 +71,40 @@ Int64 getCurrentQueryMemoryUsage()
return 0; return 0;
} }
Block filterColumnsPresentInSampleBlock(const Block & block, const Block & sample_block)
{
Block filtered_block;
for (const auto & sample_column : sample_block.getColumnsWithTypeAndName())
filtered_block.insert(block.getByName(sample_column.name));
return filtered_block;
}
ScatteredBlock filterColumnsPresentInSampleBlock(const ScatteredBlock & block, const Block & sample_block)
{
return ScatteredBlock{filterColumnsPresentInSampleBlock(block.getSourceBlock(), sample_block)};
}
Block materializeColumnsFromRightBlock(Block block, const Block & sample_block, const Names &)
{
for (const auto & sample_column : sample_block.getColumnsWithTypeAndName())
{
auto & column = block.getByName(sample_column.name);
/// There's no optimization for right side const columns. Remove constness if any.
column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst());
if (column.column->lowCardinality() && !sample_column.column->lowCardinality())
{
column.column = column.column->convertToFullColumnIfLowCardinality();
column.type = removeLowCardinality(column.type);
}
if (sample_column.column->isNullable())
JoinCommon::convertColumnToNullable(column);
}
return block;
}
} }
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable) static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable)
@ -91,8 +124,12 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla
} }
} }
HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block_, HashJoin::HashJoin(
bool any_take_last_row_, size_t reserve_num_, const String & instance_id_) std::shared_ptr<TableJoin> table_join_,
const Block & right_sample_block_,
bool any_take_last_row_,
size_t reserve_num_,
const String & instance_id_)
: table_join(table_join_) : table_join(table_join_)
, kind(table_join->kind()) , kind(table_join->kind())
, strictness(table_join->strictness()) , strictness(table_join->strictness())
@ -107,8 +144,15 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
, instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "") , instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "")
, log(getLogger("HashJoin")) , log(getLogger("HashJoin"))
{ {
LOG_TRACE(log, "{}Keys: {}, datatype: {}, kind: {}, strictness: {}, right header: {}", LOG_TRACE(
instance_log_id, TableJoin::formatClauses(table_join->getClauses(), true), data->type, kind, strictness, right_sample_block.dumpStructure()); log,
"{}Keys: {}, datatype: {}, kind: {}, strictness: {}, right header: {}",
instance_log_id,
TableJoin::formatClauses(table_join->getClauses(), true),
data->type,
kind,
strictness,
right_sample_block.dumpStructure());
validateAdditionalFilterExpression(table_join->getMixedJoinExpression()); validateAdditionalFilterExpression(table_join->getMixedJoinExpression());
@ -252,8 +296,8 @@ HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_c
}; };
const auto * key_column = key_columns[0]; const auto * key_column = key_columns[0];
if (is_string_column(key_column) || if (is_string_column(key_column)
(isColumnConst(*key_column) && is_string_column(assert_cast<const ColumnConst *>(key_column)->getDataColumnPtr().get()))) || (isColumnConst(*key_column) && is_string_column(assert_cast<const ColumnConst *>(key_column)->getDataColumnPtr().get())))
return Type::key_string; return Type::key_string;
} }
@ -323,7 +367,8 @@ size_t HashJoin::getTotalRowCount() const
auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
for (const auto & map : data->maps) for (const auto & map : data->maps)
{ {
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](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); });
} }
} }
@ -338,16 +383,22 @@ void HashJoin::doDebugAsserts() const
debug_blocks_allocated_size += block.allocatedBytes(); debug_blocks_allocated_size += block.allocatedBytes();
if (data->blocks_allocated_size != debug_blocks_allocated_size) if (data->blocks_allocated_size != debug_blocks_allocated_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "data->blocks_allocated_size != debug_blocks_allocated_size ({} != {})", throw Exception(
data->blocks_allocated_size, debug_blocks_allocated_size); ErrorCodes::LOGICAL_ERROR,
"data->blocks_allocated_size != debug_blocks_allocated_size ({} != {})",
data->blocks_allocated_size,
debug_blocks_allocated_size);
size_t debug_blocks_nullmaps_allocated_size = 0; size_t debug_blocks_nullmaps_allocated_size = 0;
for (const auto & nullmap : data->blocks_nullmaps) for (const auto & nullmap : data->blocks_nullmaps)
debug_blocks_nullmaps_allocated_size += nullmap.second->allocatedBytes(); debug_blocks_nullmaps_allocated_size += nullmap.allocatedBytes();
if (data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size) if (data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size ({} != {})", throw Exception(
data->blocks_nullmaps_allocated_size, debug_blocks_nullmaps_allocated_size); ErrorCodes::LOGICAL_ERROR,
"data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size ({} != {})",
data->blocks_nullmaps_allocated_size,
debug_blocks_nullmaps_allocated_size);
#endif #endif
} }
@ -369,7 +420,12 @@ size_t HashJoin::getTotalByteCount() const
auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
for (const auto & map : data->maps) for (const auto & map : data->maps)
{ {
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](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; return res;
@ -386,11 +442,8 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample)
bool multiple_disjuncts = !table_join->oneDisjunct(); bool multiple_disjuncts = !table_join->oneDisjunct();
/// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any).
bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH)
table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || || isRightOrFull(kind) || multiple_disjuncts || table_join->getMixedJoinExpression();
isRightOrFull(kind) ||
multiple_disjuncts ||
table_join->getMixedJoinExpression();
if (save_key_columns) if (save_key_columns)
{ {
saved_block_sample = right_table_keys.cloneEmpty(); saved_block_sample = right_table_keys.cloneEmpty();
@ -411,29 +464,27 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample)
} }
} }
void HashJoin::materializeColumnsFromLeftBlock(Block & block) const
{
/** 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 (kind == JoinKind::Right || kind == JoinKind::Full)
{
materializeBlockInplace(block);
}
}
Block HashJoin::materializeColumnsFromRightBlock(Block block) const
{
return DB::materializeColumnsFromRightBlock(std::move(block), savedBlockSample(), table_join->getAllNames(JoinTableSide::Right));
}
Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block_sample_) Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block_sample_)
{ {
Block structured_block; Block prepared_block = DB::materializeColumnsFromRightBlock(block, saved_block_sample_, {});
for (const auto & sample_column : saved_block_sample_.getColumnsWithTypeAndName()) return filterColumnsPresentInSampleBlock(prepared_block, saved_block_sample_);
{
ColumnWithTypeAndName column = block.getByName(sample_column.name);
/// There's no optimization for right side const columns. Remove constness if any.
column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst());
if (column.column->lowCardinality() && !sample_column.column->lowCardinality())
{
column.column = column.column->convertToFullColumnIfLowCardinality();
column.type = removeLowCardinality(column.type);
}
if (sample_column.column->isNullable())
JoinCommon::convertColumnToNullable(column);
structured_block.insert(std::move(column));
}
return structured_block;
} }
Block HashJoin::prepareRightBlock(const Block & block) const Block HashJoin::prepareRightBlock(const Block & block) const
@ -441,15 +492,22 @@ Block HashJoin::prepareRightBlock(const Block & block) const
return prepareRightBlock(block, savedBlockSample()); return prepareRightBlock(block, savedBlockSample());
} }
bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) bool HashJoin::addBlockToJoin(const Block & source_block, bool check_limits)
{
auto materialized = materializeColumnsFromRightBlock(source_block);
auto scattered_block = ScatteredBlock{materialized};
return addBlockToJoin(scattered_block, check_limits);
}
bool HashJoin::addBlockToJoin(ScatteredBlock & source_block, bool check_limits)
{ {
if (!data) if (!data)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Join data was released"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Join data was released");
/// RowRef::SizeT is uint32_t (not size_t) for hash table Cell memory efficiency. /// RowRef::SizeT is uint32_t (not size_t) for hash table Cell memory efficiency.
/// It's possible to split bigger blocks and insert them by parts here. But it would be a dead code. /// It's possible to split bigger blocks and insert them by parts here. But it would be a dead code.
if (unlikely(source_block_.rows() > std::numeric_limits<RowRef::SizeT>::max())) if (unlikely(source_block.rows() > std::numeric_limits<RowRef::SizeT>::max()))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Too many rows in right table block for HashJoin: {}", source_block_.rows()); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Too many rows in right table block for HashJoin: {}", source_block.rows());
/** We do not allocate memory for stored blocks inside HashJoin, only for hash table. /** We do not allocate memory for stored blocks inside HashJoin, only for hash table.
* In case when we have all the blocks allocated before the first `addBlockToJoin` call, will already be quite high. * In case when we have all the blocks allocated before the first `addBlockToJoin` call, will already be quite high.
@ -458,7 +516,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
if (!memory_usage_before_adding_blocks) if (!memory_usage_before_adding_blocks)
memory_usage_before_adding_blocks = getCurrentQueryMemoryUsage(); memory_usage_before_adding_blocks = getCurrentQueryMemoryUsage();
Block source_block = source_block_;
if (strictness == JoinStrictness::Asof) if (strictness == JoinStrictness::Asof)
{ {
chassert(kind == JoinKind::Left || kind == JoinKind::Inner); chassert(kind == JoinKind::Left || kind == JoinKind::Inner);
@ -467,7 +524,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
/// We support only INNER/LEFT ASOF join, so rows with NULLs never return from the right joined table. /// We support only INNER/LEFT ASOF join, so rows with NULLs never return from the right joined table.
/// So filter them out here not to handle in implementation. /// So filter them out here not to handle in implementation.
const auto & asof_key_name = table_join->getOnlyClause().key_names_right.back(); const auto & asof_key_name = table_join->getOnlyClause().key_names_right.back();
auto & asof_column = source_block.getByName(asof_key_name); const auto & asof_column = source_block.getByName(asof_key_name);
if (asof_column.type->isNullable()) if (asof_column.type->isNullable())
{ {
@ -485,13 +542,12 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
for (size_t i = 0; i < asof_column_nullable.size(); ++i) for (size_t i = 0; i < asof_column_nullable.size(); ++i)
negative_null_map[i] = !asof_column_nullable[i]; negative_null_map[i] = !asof_column_nullable[i];
for (auto & column : source_block) source_block.filter(negative_null_map);
column.column = column.column->filter(negative_null_map, -1);
} }
} }
} }
size_t rows = source_block.rows(); const size_t rows = source_block.rows();
data->rows_to_join += rows; data->rows_to_join += rows;
const auto & right_key_names = table_join->getAllNames(JoinTableSide::Right); const auto & right_key_names = table_join->getAllNames(JoinTableSide::Right);
ColumnPtrMap all_key_columns(right_key_names.size()); ColumnPtrMap all_key_columns(right_key_names.size());
@ -501,7 +557,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
all_key_columns[column_name] = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); all_key_columns[column_name] = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality();
} }
Block block_to_save = prepareRightBlock(source_block); ScatteredBlock block_to_save = filterColumnsPresentInSampleBlock(source_block, savedBlockSample());
if (shrink_blocks) if (shrink_blocks)
block_to_save = block_to_save.shrinkToFit(); block_to_save = block_to_save.shrinkToFit();
@ -515,7 +571,8 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
if (!tmp_stream) if (!tmp_stream)
tmp_stream.emplace(right_sample_block, tmp_data.get()); tmp_stream.emplace(right_sample_block, tmp_data.get());
tmp_stream.value()->write(block_to_save); chassert(!source_block.wasScattered()); /// We don't run parallel_hash for cross join
tmp_stream.value()->write(block_to_save.getSourceBlock());
return true; return true;
} }
@ -527,7 +584,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
if (storage_join_lock) if (storage_join_lock)
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "addBlockToJoin called when HashJoin locked to prevent updates"); throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "addBlockToJoin called when HashJoin locked to prevent updates");
assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "joined block"); assertBlocksHaveEqualStructure(data->sample_block, block_to_save.getSourceBlock(), "joined block");
size_t min_bytes_to_compress = table_join->crossJoinMinBytesToCompress(); size_t min_bytes_to_compress = table_join->crossJoinMinBytesToCompress();
size_t min_rows_to_compress = table_join->crossJoinMinRowsToCompress(); size_t min_rows_to_compress = table_join->crossJoinMinRowsToCompress();
@ -536,6 +593,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
&& ((min_bytes_to_compress && getTotalByteCount() >= min_bytes_to_compress) && ((min_bytes_to_compress && getTotalByteCount() >= min_bytes_to_compress)
|| (min_rows_to_compress && getTotalRowCount() >= min_rows_to_compress))) || (min_rows_to_compress && getTotalRowCount() >= min_rows_to_compress)))
{ {
chassert(!source_block.wasScattered()); /// We don't run parallel_hash for cross join
block_to_save = block_to_save.compress(); block_to_save = block_to_save.compress();
have_compressed = true; have_compressed = true;
} }
@ -543,7 +601,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
doDebugAsserts(); doDebugAsserts();
data->blocks_allocated_size += block_to_save.allocatedBytes(); data->blocks_allocated_size += block_to_save.allocatedBytes();
data->blocks.emplace_back(std::move(block_to_save)); data->blocks.emplace_back(std::move(block_to_save));
Block * stored_block = &data->blocks.back(); const auto * stored_block = &data->blocks.back();
doDebugAsserts(); doDebugAsserts();
if (rows) if (rows)
@ -570,7 +628,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
save_nullmap |= (*null_map)[i]; save_nullmap |= (*null_map)[i];
} }
auto join_mask_col = JoinCommon::getColumnAsMask(source_block, onexprs[onexpr_idx].condColumnNames().second); auto join_mask_col = JoinCommon::getColumnAsMask(source_block.getSourceBlock(), onexprs[onexpr_idx].condColumnNames().second);
/// Save blocks that do not hold conditions in ON section /// Save blocks that do not hold conditions in ON section
ColumnUInt8::MutablePtr not_joined_map = nullptr; ColumnUInt8::MutablePtr not_joined_map = nullptr;
if (!flag_per_row && isRightOrFull(kind) && join_mask_col.hasData()) if (!flag_per_row && isRightOrFull(kind) && join_mask_col.hasData())
@ -595,39 +653,45 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
bool is_inserted = false; bool is_inserted = false;
if (kind != JoinKind::Cross) if (kind != JoinKind::Cross)
{ {
joinDispatch(kind, strictness, data->maps[onexpr_idx], prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & map) joinDispatch(
{ kind,
size_t size = HashJoinMethods<kind_, strictness_, std::decay_t<decltype(map)>>::insertFromBlockImpl( 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, *this,
data->type, data->type,
map, map,
rows,
key_columns, key_columns,
key_sizes[onexpr_idx], key_sizes[onexpr_idx],
stored_block, &stored_block->getSourceBlock(),
source_block.getSelector(),
null_map, null_map,
join_mask_col.getData(), join_mask_col.getData(),
data->pool, data->pool,
is_inserted); is_inserted);
if (flag_per_row) if (flag_per_row)
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map)>, MapsAll>>(stored_block); used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map)>, MapsAll>>(
else if (is_inserted) &stored_block->getSourceBlock());
/// Number of buckets + 1 value from zero storage else if (is_inserted)
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map)>, MapsAll>>(size + 1); /// Number of buckets + 1 value from zero storage
}); used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map)>, MapsAll>>(size + 1);
});
} }
if (!flag_per_row && save_nullmap && is_inserted) if (!flag_per_row && save_nullmap && is_inserted)
{ {
data->blocks_nullmaps_allocated_size += null_map_holder->allocatedBytes();
data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); data->blocks_nullmaps.emplace_back(stored_block, null_map_holder);
data->blocks_nullmaps_allocated_size += data->blocks_nullmaps.back().allocatedBytes();
} }
if (!flag_per_row && not_joined_map && is_inserted) if (!flag_per_row && not_joined_map && is_inserted)
{ {
data->blocks_nullmaps_allocated_size += not_joined_map->allocatedBytes();
data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map));
data->blocks_nullmaps_allocated_size += data->blocks_nullmaps.back().allocatedBytes();
} }
if (!flag_per_row && !is_inserted) if (!flag_per_row && !is_inserted)
@ -654,7 +718,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_optimize) void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_optimize)
{ {
Int64 current_memory_usage = getCurrentQueryMemoryUsage(); Int64 current_memory_usage = getCurrentQueryMemoryUsage();
Int64 query_memory_usage_delta = current_memory_usage - memory_usage_before_adding_blocks; Int64 query_memory_usage_delta = current_memory_usage - memory_usage_before_adding_blocks;
Int64 max_total_bytes_for_query = memory_usage_before_adding_blocks ? table_join->getMaxMemoryUsage() : 0; Int64 max_total_bytes_for_query = memory_usage_before_adding_blocks ? table_join->getMaxMemoryUsage() : 0;
@ -671,15 +734,19 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_
* is bigger than half of all memory available for query, * is bigger than half of all memory available for query,
* then shrink stored blocks to fit. * then shrink stored blocks to fit.
*/ */
shrink_blocks = (max_total_bytes_in_join && total_bytes_in_join > max_total_bytes_in_join / 2) || shrink_blocks = (max_total_bytes_in_join && total_bytes_in_join > max_total_bytes_in_join / 2)
(max_total_bytes_for_query && query_memory_usage_delta > max_total_bytes_for_query / 2); || (max_total_bytes_for_query && query_memory_usage_delta > max_total_bytes_for_query / 2);
if (!shrink_blocks) if (!shrink_blocks)
return; return;
} }
LOG_DEBUG(log, "Shrinking stored blocks, memory consumption is {} {} calculated by join, {} {} by memory tracker", LOG_DEBUG(
ReadableSize(total_bytes_in_join), max_total_bytes_in_join ? fmt::format("/ {}", ReadableSize(max_total_bytes_in_join)) : "", log,
ReadableSize(query_memory_usage_delta), max_total_bytes_for_query ? fmt::format("/ {}", ReadableSize(max_total_bytes_for_query)) : ""); "Shrinking stored blocks, memory consumption is {} {} calculated by join, {} {} by memory tracker",
ReadableSize(total_bytes_in_join),
max_total_bytes_in_join ? fmt::format("/ {}", ReadableSize(max_total_bytes_in_join)) : "",
ReadableSize(query_memory_usage_delta),
max_total_bytes_for_query ? fmt::format("/ {}", ReadableSize(max_total_bytes_for_query)) : "");
for (auto & stored_block : data->blocks) for (auto & stored_block : data->blocks)
{ {
@ -692,10 +759,13 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_
if (old_size >= new_size) if (old_size >= new_size)
{ {
if (data->blocks_allocated_size < old_size - new_size) if (data->blocks_allocated_size < old_size - new_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Blocks allocated size value is broken: " "Blocks allocated size value is broken: "
"blocks_allocated_size = {}, old_size = {}, new_size = {}", "blocks_allocated_size = {}, old_size = {}, new_size = {}",
data->blocks_allocated_size, old_size, new_size); data->blocks_allocated_size,
old_size,
new_size);
data->blocks_allocated_size -= old_size - new_size; data->blocks_allocated_size -= old_size - new_size;
} }
@ -710,9 +780,13 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_
Int64 new_current_memory_usage = getCurrentQueryMemoryUsage(); Int64 new_current_memory_usage = getCurrentQueryMemoryUsage();
LOG_DEBUG(log, "Shrunk stored blocks {} freed ({} by memory tracker), new memory consumption is {} ({} by memory tracker)", LOG_DEBUG(
ReadableSize(total_bytes_in_join - new_total_bytes_in_join), ReadableSize(current_memory_usage - new_current_memory_usage), log,
ReadableSize(new_total_bytes_in_join), ReadableSize(new_current_memory_usage)); "Shrunk stored blocks {} freed ({} by memory tracker), new memory consumption is {} ({} by memory tracker)",
ReadableSize(total_bytes_in_join - new_total_bytes_in_join),
ReadableSize(current_memory_usage - new_current_memory_usage),
ReadableSize(new_total_bytes_in_join),
ReadableSize(new_current_memory_usage));
total_bytes_in_join = new_total_bytes_in_join; total_bytes_in_join = new_total_bytes_in_join;
} }
@ -776,7 +850,7 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed)
} }
}; };
for (const Block & block_right : data->blocks) for (const auto & block_right : data->blocks)
{ {
++block_number; ++block_number;
if (block_number < start_right_block) if (block_number < start_right_block)
@ -784,9 +858,12 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed)
/// The following statement cannot be substituted with `process_right_block(!have_compressed ? block_right : block_right.decompress())` /// The following statement cannot be substituted with `process_right_block(!have_compressed ? block_right : block_right.decompress())`
/// because it will lead to copying of `block_right` even if its branch is taken (because common type of `block_right` and `block_right.decompress()` is `Block`). /// because it will lead to copying of `block_right` even if its branch is taken (because common type of `block_right` and `block_right.decompress()` is `Block`).
if (!have_compressed) if (!have_compressed)
process_right_block(block_right); process_right_block(block_right.getSourceBlock());
else else
process_right_block(block_right.decompress()); {
chassert(!block_right.wasScattered()); /// Compression only happens for cross join and scattering only for concurrent hash
process_right_block(block_right.getSourceBlock().decompress());
}
if (rows_added > max_joined_block_rows) if (rows_added > max_joined_block_rows)
{ {
@ -837,9 +914,11 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types,
{ {
size_t num_keys = data_types.size(); size_t num_keys = data_types.size();
if (right_table_keys.columns() != num_keys) if (right_table_keys.columns() != num_keys)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function joinGet{} doesn't match: passed, should be equal to {}", "Number of arguments for function joinGet{} doesn't match: passed, should be equal to {}",
toString(or_null ? "OrNull" : ""), toString(num_keys)); toString(or_null ? "OrNull" : ""),
toString(num_keys));
for (size_t i = 0; i < num_keys; ++i) for (size_t i = 0; i < num_keys; ++i)
{ {
@ -848,8 +927,13 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types,
auto left_type = removeNullable(recursiveRemoveLowCardinality(left_type_origin)); auto left_type = removeNullable(recursiveRemoveLowCardinality(left_type_origin));
auto right_type = removeNullable(recursiveRemoveLowCardinality(right_type_origin)); auto right_type = removeNullable(recursiveRemoveLowCardinality(right_type_origin));
if (!left_type->equals(*right_type)) if (!left_type->equals(*right_type))
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in joinGet key {}: " throw Exception(
"found type {}, while the needed type is {}", i, left_type->getName(), right_type->getName()); ErrorCodes::TYPE_MISMATCH,
"Type mismatch in joinGet key {}: "
"found type {}, while the needed type is {}",
i,
left_type->getName(),
right_type->getName());
} }
if (!sample_block_with_columns_to_add.has(column_name)) if (!sample_block_with_columns_to_add.has(column_name))
@ -865,8 +949,7 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types,
/// TODO: return array of values when strictness == JoinStrictness::All /// TODO: return array of values when strictness == JoinStrictness::All
ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const
{ {
bool is_valid = (strictness == JoinStrictness::Any || strictness == JoinStrictness::RightAny) bool is_valid = (strictness == JoinStrictness::Any || strictness == JoinStrictness::RightAny) && kind == JoinKind::Left;
&& kind == JoinKind::Left;
if (!is_valid) if (!is_valid)
throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "joinGet only supports StorageJoin of type Left Any"); throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "joinGet only supports StorageJoin of type Left Any");
const auto & key_names_right = table_join->getOnlyClause().key_names_right; const auto & key_names_right = table_join->getOnlyClause().key_names_right;
@ -880,12 +963,14 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block
keys.insert(std::move(key)); keys.insert(std::move(key));
} }
static_assert(!MapGetter<JoinKind::Left, JoinStrictness::Any, false>::flagged, static_assert(
"joinGet are not protected from hash table changes between block processing"); !MapGetter<JoinKind::Left, JoinStrictness::Any, false>::flagged,
"joinGet are not protected from hash table changes between block processing");
std::vector<const MapsOne *> maps_vector; std::vector<const MapsOne *> maps_vector;
maps_vector.push_back(&std::get<MapsOne>(data->maps[0])); maps_vector.push_back(&std::get<MapsOne>(data->maps[0]));
HashJoinMethods<JoinKind::Left, JoinStrictness::Any, MapsOne>::joinBlockImpl(*this, keys, block_with_columns_to_add, maps_vector, /* is_join_get = */ true); HashJoinMethods<JoinKind::Left, JoinStrictness::Any, MapsOne>::joinBlockImpl(
*this, keys, block_with_columns_to_add, maps_vector, /* is_join_get = */ true);
return keys.getByPosition(keys.columns() - 1); return keys.getByPosition(keys.columns() - 1);
} }
@ -906,8 +991,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
{ {
auto cond_column_name = onexpr.condColumnNames(); auto cond_column_name = onexpr.condColumnNames();
JoinCommon::checkTypesOfKeys( JoinCommon::checkTypesOfKeys(
block, onexpr.key_names_left, cond_column_name.first, block, onexpr.key_names_left, cond_column_name.first, right_sample_block, onexpr.key_names_right, cond_column_name.second);
right_sample_block, onexpr.key_names_right, cond_column_name.second);
} }
if (kind == JoinKind::Cross) if (kind == JoinKind::Cross)
@ -916,20 +1000,85 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
return; return;
} }
if (kind == JoinKind::Right || kind == JoinKind::Full) materializeColumnsFromLeftBlock(block);
{
materializeBlockInplace(block);
}
bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr; bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
{ {
std::vector<const std::decay_t<decltype(data->maps[0])> * > maps_vector; std::vector<const std::decay_t<decltype(data->maps[0])> *> maps_vector;
for (size_t i = 0; i < table_join->getClauses().size(); ++i) for (size_t i = 0; i < table_join->getClauses().size(); ++i)
maps_vector.push_back(&data->maps[i]); maps_vector.push_back(&data->maps[i]);
if (joinDispatch(kind, strictness, maps_vector, prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & maps_vector_) if (joinDispatch(
kind,
strictness,
maps_vector,
prefer_use_maps_all,
[&](auto kind_, auto strictness_, auto & maps_vector_)
{
Block remaining_block;
if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsAll *>>)
{
remaining_block = HashJoinMethods<kind_, strictness_, MapsAll>::joinBlockImpl(
*this, block, sample_block_with_columns_to_add, maps_vector_);
}
else if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsOne *>>)
{
remaining_block = HashJoinMethods<kind_, strictness_, MapsOne>::joinBlockImpl(
*this, block, sample_block_with_columns_to_add, maps_vector_);
}
else if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsAsof *>>)
{
remaining_block = HashJoinMethods<kind_, strictness_, MapsAsof>::joinBlockImpl(
*this, block, sample_block_with_columns_to_add, maps_vector_);
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown maps type");
}
if (remaining_block.rows())
not_processed = std::make_shared<ExtraBlock>(ExtraBlock{std::move(remaining_block)});
else
not_processed.reset();
}))
{
/// Joined
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind);
}
}
void HashJoin::joinBlock(ScatteredBlock & block, ScatteredBlock & remaining_block)
{
if (!data)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released");
chassert(kind == JoinKind::Left || kind == JoinKind::Inner);
for (const auto & onexpr : table_join->getClauses())
{
auto cond_column_name = onexpr.condColumnNames();
JoinCommon::checkTypesOfKeys(
block.getSourceBlock(),
onexpr.key_names_left,
cond_column_name.first,
right_sample_block,
onexpr.key_names_right,
cond_column_name.second);
}
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]);
bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
const bool joined = joinDispatch(
kind,
strictness,
maps_vector,
prefer_use_maps_all,
[&](auto kind_, auto strictness_, auto & maps_vector_)
{ {
Block remaining_block;
if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsAll *>>) if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsAll *>>)
{ {
remaining_block = HashJoinMethods<kind_, strictness_, MapsAll>::joinBlockImpl( remaining_block = HashJoinMethods<kind_, strictness_, MapsAll>::joinBlockImpl(
@ -949,17 +1098,9 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown maps type"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown maps type");
} }
if (remaining_block.rows()) });
not_processed = std::make_shared<ExtraBlock>(ExtraBlock{std::move(remaining_block)});
else chassert(joined);
not_processed.reset();
}))
{
/// Joined
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind);
}
} }
HashJoin::~HashJoin() HashJoin::~HashJoin()
@ -1023,10 +1164,7 @@ class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller
{ {
public: public:
NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_, bool flag_per_row_) NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_, bool flag_per_row_)
: parent(parent_) : parent(parent_), max_block_size(max_block_size_), flag_per_row(flag_per_row_), current_block_start(0)
, max_block_size(max_block_size_)
, flag_per_row(flag_per_row_)
, current_block_start(0)
{ {
if (parent.data == nullptr) if (parent.data == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released");
@ -1043,14 +1181,12 @@ public:
} }
else else
{ {
auto fill_callback = [&](auto, auto, auto & map) auto fill_callback = [&](auto, auto, auto & map) { rows_added = fillColumnsFromMap(map, columns_right); };
{
rows_added = fillColumnsFromMap(map, columns_right);
};
bool prefer_use_maps_all = parent.table_join->getMixedJoinExpression() != nullptr; 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)) 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); throw Exception(
ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness);
} }
if (!flag_per_row) if (!flag_per_row)
@ -1070,14 +1206,14 @@ private:
std::any position; std::any position;
std::optional<HashJoin::BlockNullmapList::const_iterator> nulls_position; std::optional<HashJoin::BlockNullmapList::const_iterator> nulls_position;
std::optional<BlocksList::const_iterator> used_position; std::optional<HashJoin::ScatteredBlocksList::const_iterator> used_position;
size_t fillColumnsFromData(const BlocksList & blocks, MutableColumns & columns_right) size_t fillColumnsFromData(const HashJoin::ScatteredBlocksList & blocks, MutableColumns & columns_right)
{ {
if (!position.has_value()) if (!position.has_value())
position = std::make_any<BlocksList::const_iterator>(blocks.begin()); position = std::make_any<HashJoin::ScatteredBlocksList::const_iterator>(blocks.begin());
auto & block_it = std::any_cast<BlocksList::const_iterator &>(position); auto & block_it = std::any_cast<HashJoin::ScatteredBlocksList::const_iterator &>(position);
auto end = blocks.end(); auto end = blocks.end();
size_t rows_added = 0; size_t rows_added = 0;
@ -1113,11 +1249,11 @@ private:
{ {
switch (parent.data->type) switch (parent.data->type)
{ {
#define M(TYPE) \ #define M(TYPE) \
case HashJoin::Type::TYPE: \ case HashJoin::Type::TYPE: \
return fillColumns(*maps.TYPE, columns_keys_and_right); return fillColumns(*maps.TYPE, columns_keys_and_right);
APPLY_FOR_JOIN_VARIANTS(M) APPLY_FOR_JOIN_VARIANTS(M)
#undef M #undef M
default: default:
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type); throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type);
} }
@ -1137,11 +1273,11 @@ private:
for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it) for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it)
{ {
const Block & mapped_block = *it; const auto & mapped_block = *it;
for (size_t row = 0; row < mapped_block.rows(); ++row) for (size_t row = 0; row < mapped_block.rows(); ++row)
{ {
if (!parent.isUsed(&mapped_block, row)) if (!parent.isUsed(&mapped_block.getSourceBlock(), row))
{ {
for (size_t colnum = 0; colnum < columns_keys_and_right.size(); ++colnum) for (size_t colnum = 0; colnum < columns_keys_and_right.size(); ++colnum)
{ {
@ -1194,10 +1330,10 @@ private:
for (auto & it = *nulls_position; it != end && rows_added < max_block_size; ++it) for (auto & it = *nulls_position; it != end && rows_added < max_block_size; ++it)
{ {
const auto * block = it->first; const auto * block = it->block;
ConstNullMapPtr nullmap = nullptr; ConstNullMapPtr nullmap = nullptr;
if (it->second) if (it->column)
nullmap = &assert_cast<const ColumnUInt8 &>(*it->second).getData(); nullmap = &assert_cast<const ColumnUInt8 &>(*it->column).getData();
for (size_t row = 0; row < block->rows(); ++row) for (size_t row = 0; row < block->rows(); ++row)
{ {
@ -1212,9 +1348,8 @@ private:
} }
}; };
IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, IBlocksStreamPtr
const Block & result_sample_block, HashJoin::getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const
UInt64 max_block_size) const
{ {
if (!JoinCommon::hasNonJoinedBlocks(*table_join)) if (!JoinCommon::hasNonJoinedBlocks(*table_join))
return {}; return {};
@ -1227,9 +1362,14 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block,
size_t expected_columns_count = left_columns_count + required_right_keys.columns() + sample_block_with_columns_to_add.columns(); size_t expected_columns_count = left_columns_count + required_right_keys.columns() + sample_block_with_columns_to_add.columns();
if (expected_columns_count != result_sample_block.columns()) if (expected_columns_count != result_sample_block.columns())
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected number of columns in result sample block: {} instead of {} ({} + {} + {})", throw Exception(
result_sample_block.columns(), expected_columns_count, ErrorCodes::LOGICAL_ERROR,
left_columns_count, required_right_keys.columns(), sample_block_with_columns_to_add.columns()); "Unexpected number of columns in result sample block: {} instead of {} ({} + {} + {})",
result_sample_block.columns(),
expected_columns_count,
left_columns_count,
required_right_keys.columns(),
sample_block_with_columns_to_add.columns());
} }
} }
@ -1250,22 +1390,37 @@ void HashJoin::reuseJoinedData(const HashJoin & join)
bool prefer_use_maps_all = join.table_join->getMixedJoinExpression() != nullptr; bool prefer_use_maps_all = join.table_join->getMixedJoinExpression() != nullptr;
for (auto & map : data->maps) for (auto & map : data->maps)
{ {
joinDispatch(kind, strictness, map, prefer_use_maps_all, [this](auto kind_, auto strictness_, auto & map_) joinDispatch(
{ kind,
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map_)>, MapsAll>>(map_.getBufferSizeInCells(data->type) + 1); strictness,
}); map,
prefer_use_maps_all,
[this](auto kind_, auto strictness_, auto & map_)
{
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map_)>, MapsAll>>(
map_.getBufferSizeInCells(data->type) + 1);
});
} }
} }
BlocksList HashJoin::releaseJoinedBlocks(bool restructure) BlocksList HashJoin::releaseJoinedBlocks(bool restructure [[maybe_unused]])
{ {
LOG_TRACE(log, "{}Join data is being released, {} bytes and {} rows in hash table", instance_log_id, getTotalByteCount(), getTotalRowCount()); LOG_TRACE(
log, "{}Join data is being released, {} bytes and {} rows in hash table", instance_log_id, getTotalByteCount(), getTotalRowCount());
BlocksList right_blocks = std::move(data->blocks); auto extract_source_blocks = [](ScatteredBlocksList && blocks)
{
BlocksList result;
for (auto & block : blocks)
result.emplace_back(std::move(block).getSourceBlock());
return result;
};
ScatteredBlocksList right_blocks = std::move(data->blocks);
if (!restructure) if (!restructure)
{ {
data.reset(); data.reset();
return right_blocks; return extract_source_blocks(std::move(right_blocks));
} }
data->maps.clear(); data->maps.clear();
@ -1279,7 +1434,7 @@ BlocksList HashJoin::releaseJoinedBlocks(bool restructure)
if (!right_blocks.empty()) if (!right_blocks.empty())
{ {
positions.reserve(right_sample_block.columns()); positions.reserve(right_sample_block.columns());
const Block & tmp_block = *right_blocks.begin(); const Block & tmp_block = right_blocks.begin()->getSourceBlock();
for (const auto & sample_column : right_sample_block) for (const auto & sample_column : right_sample_block)
{ {
positions.emplace_back(tmp_block.getPositionByName(sample_column.name)); positions.emplace_back(tmp_block.getPositionByName(sample_column.name));
@ -1287,12 +1442,12 @@ BlocksList HashJoin::releaseJoinedBlocks(bool restructure)
} }
} }
for (Block & saved_block : right_blocks) for (ScatteredBlock & saved_block : right_blocks)
{ {
Block restored_block; Block restored_block;
for (size_t i = 0; i < positions.size(); ++i) for (size_t i = 0; i < positions.size(); ++i)
{ {
auto & column = saved_block.getByPosition(positions[i]); auto & column = saved_block.getSourceBlock().getByPosition(positions[i]);
correctNullabilityInplace(column, is_nullable[i]); correctNullabilityInplace(column, is_nullable[i]);
restored_block.insert(column); restored_block.insert(column);
} }
@ -1318,7 +1473,8 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona
if (expression_sample_block.columns() != 1) if (expression_sample_block.columns() != 1)
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Unexpected expression in JOIN ON section. Expected single column, got '{}'", "Unexpected expression in JOIN ON section. Expected single column, got '{}'",
expression_sample_block.dumpStructure()); expression_sample_block.dumpStructure());
} }
@ -1326,7 +1482,8 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona
auto type = removeNullable(expression_sample_block.getByPosition(0).type); auto type = removeNullable(expression_sample_block.getByPosition(0).type);
if (!type->equals(*std::make_shared<DataTypeUInt8>())) if (!type->equals(*std::make_shared<DataTypeUInt8>()))
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Unexpected expression in JOIN ON section. Expected boolean (UInt8), got '{}'. expression:\n{}", "Unexpected expression in JOIN ON section. Expected boolean (UInt8), got '{}'. expression:\n{}",
expression_sample_block.getByPosition(0).type->getName(), expression_sample_block.getByPosition(0).type->getName(),
additional_filter_expression->dumpActions()); additional_filter_expression->dumpActions());
@ -1334,10 +1491,12 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona
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) || ((strictness == JoinStrictness::Semi || strictness == JoinStrictness::Any || strictness == JoinStrictness::Anti)
&& (isLeft(kind) || isRight(kind))) || (strictness == JoinStrictness::Any && (isInner(kind))); && (isLeft(kind) || isRight(kind)))
|| (strictness == JoinStrictness::Any && (isInner(kind)));
if (!is_supported) if (!is_supported)
{ {
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, throw Exception(
ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/FULL/RIGHT JOINs", "Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/FULL/RIGHT JOINs",
expression_sample_block.getByPosition(0).name); expression_sample_block.getByPosition(0).name);
} }
@ -1353,7 +1512,6 @@ bool HashJoin::isUsed(const Block * block_ptr, size_t row_idx) const
return used_flags->getUsedSafe(block_ptr, row_idx); return used_flags->getUsedSafe(block_ptr, row_idx);
} }
bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const
{ {
if (!table_join_->oneDisjunct()) if (!table_join_->oneDisjunct())
@ -1372,7 +1530,7 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]])
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged."); throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged.");
else else
{ {
auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref) auto merge_rows_into_one_block = [&](ScatteredBlocksList & blocks, RowRefList & rows_ref)
{ {
auto it = rows_ref.begin(); auto it = rows_ref.begin();
if (it.ok()) if (it.ok())
@ -1384,7 +1542,7 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]])
{ {
return; return;
} }
auto & block = blocks.back(); auto & block = blocks.back().getSourceBlock();
size_t start_row = block.rows(); size_t start_row = block.rows();
for (; it.ok(); ++it) for (; it.ok(); ++it)
{ {
@ -1401,23 +1559,22 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]])
} }
}; };
auto visit_rows_map = [&](BlocksList & blocks, MapsAll & rows_map) auto visit_rows_map = [&](ScatteredBlocksList & blocks, MapsAll & rows_map)
{ {
switch (data->type) switch (data->type)
{ {
#define M(TYPE) \ #define M(TYPE) \
case Type::TYPE: \ case Type::TYPE: { \
{\ rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \
rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \ break; \
break; \ }
}
APPLY_FOR_JOIN_VARIANTS(M) APPLY_FOR_JOIN_VARIANTS(M)
#undef M #undef M
default: default:
break; break;
} }
}; };
BlocksList sorted_blocks; ScatteredBlocksList sorted_blocks;
visit_rows_map(sorted_blocks, map); visit_rows_map(sorted_blocks, map);
doDebugAsserts(); doDebugAsserts();
data->blocks.swap(sorted_blocks); data->blocks.swap(sorted_blocks);

View File

@ -1,9 +1,11 @@
#pragma once #pragma once
#include <memory> #include <algorithm>
#include <variant>
#include <optional>
#include <deque> #include <deque>
#include <memory>
#include <optional>
#include <ranges>
#include <variant>
#include <vector> #include <vector>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
@ -12,22 +14,19 @@
#include <Interpreters/AggregationCommon.h> #include <Interpreters/AggregationCommon.h>
#include <Interpreters/RowRefs.h> #include <Interpreters/RowRefs.h>
#include <Common/Arena.h>
#include <Common/ColumnsHashing.h>
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/FixedHashMap.h>
#include <Storages/TableLockHolder.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h> #include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <QueryPipeline/SizeLimits.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Interpreters/HashJoin/ScatteredBlock.h>
#include <Storages/IStorage_fwd.h>
#include <Interpreters/IKeyValueEntity.h> #include <Interpreters/IKeyValueEntity.h>
#include <Interpreters/TemporaryDataOnDisk.h> #include <Interpreters/TemporaryDataOnDisk.h>
#include <QueryPipeline/SizeLimits.h>
#include <Storages/IStorage_fwd.h>
#include <Storages/TableLockHolder.h>
#include <Common/Arena.h>
#include <Common/ColumnsHashing.h>
#include <Common/HashTable/FixedHashMap.h>
#include <Common/HashTable/HashMap.h>
namespace DB namespace DB
{ {
@ -142,13 +141,21 @@ public:
*/ */
bool addBlockToJoin(const Block & source_block_, bool check_limits) override; bool addBlockToJoin(const Block & source_block_, bool check_limits) override;
/// Called directly from ConcurrentJoin::addBlockToJoin
bool addBlockToJoin(ScatteredBlock & source_block_, bool check_limits);
void checkTypesOfKeys(const Block & block) const override; void checkTypesOfKeys(const Block & block) const override;
using IJoin::joinBlock;
/** Join data from the map (that was previously built by calls to addBlockToJoin) to the block with data from "left" table. /** Join data from the map (that was previously built by calls to addBlockToJoin) to the block with data from "left" table.
* Could be called from different threads in parallel. * Could be called from different threads in parallel.
*/ */
void joinBlock(Block & block, ExtraBlockPtr & not_processed) override; void joinBlock(Block & block, ExtraBlockPtr & not_processed) override;
/// Called directly from ConcurrentJoin::joinBlock
void joinBlock(ScatteredBlock & block, ScatteredBlock & remaining_block);
/// Check joinGet arguments and infer the return type. /// Check joinGet arguments and infer the return type.
DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const;
@ -327,8 +334,17 @@ public:
using MapsVariant = std::variant<MapsOne, MapsAll, MapsAsof>; using MapsVariant = std::variant<MapsOne, MapsAll, MapsAsof>;
using RawBlockPtr = const Block *; using RawBlockPtr = const ScatteredBlock *;
using BlockNullmapList = std::deque<std::pair<RawBlockPtr, ColumnPtr>>; struct NullMapHolder
{
size_t allocatedBytes() const { return !column->empty() ? column->allocatedBytes() * block->rows() / column->size() : 0; }
RawBlockPtr block;
ColumnPtr column;
};
using BlockNullmapList = std::deque<NullMapHolder>;
using ScatteredBlocksList = std::list<ScatteredBlock>;
struct RightTableData struct RightTableData
{ {
@ -337,7 +353,7 @@ public:
std::vector<MapsVariant> maps; std::vector<MapsVariant> maps;
Block sample_block; /// Block as it would appear in the BlockList Block sample_block; /// Block as it would appear in the BlockList
BlocksList blocks; /// Blocks of "right" table. ScatteredBlocksList blocks; /// Blocks of "right" table.
BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed) BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed)
/// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows. /// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows.
@ -389,6 +405,9 @@ public:
void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; } void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; }
void materializeColumnsFromLeftBlock(Block & block) const;
Block materializeColumnsFromRightBlock(Block block) const;
private: private:
friend class NotJoinedHash; friend class NotJoinedHash;
@ -473,5 +492,4 @@ private:
void tryRerangeRightTableDataImpl(Map & map); void tryRerangeRightTableDataImpl(Map & map);
void doDebugAsserts() const; void doDebugAsserts() const;
}; };
} }

View File

@ -19,7 +19,7 @@ template <typename HashMap, typename KeyGetter>
struct Inserter struct Inserter
{ {
static ALWAYS_INLINE bool static ALWAYS_INLINE bool
insertOne(const HashJoin & join, HashMap & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) insertOne(const HashJoin & join, HashMap & map, KeyGetter & key_getter, const Block * stored_block, size_t i, Arena & pool)
{ {
auto emplace_result = key_getter.emplaceKey(map, i, pool); auto emplace_result = key_getter.emplaceKey(map, i, pool);
@ -31,7 +31,8 @@ struct Inserter
return false; return false;
} }
static ALWAYS_INLINE void insertAll(const HashJoin &, HashMap & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) static ALWAYS_INLINE void
insertAll(const HashJoin &, HashMap & map, KeyGetter & key_getter, const Block * stored_block, size_t i, Arena & pool)
{ {
auto emplace_result = key_getter.emplaceKey(map, i, pool); auto emplace_result = key_getter.emplaceKey(map, i, pool);
@ -45,7 +46,13 @@ struct Inserter
} }
static ALWAYS_INLINE void insertAsof( static ALWAYS_INLINE void insertAsof(
HashJoin & join, HashMap & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, const IColumn & asof_column) HashJoin & join,
HashMap & map,
KeyGetter & key_getter,
const Block * stored_block,
size_t i,
Arena & pool,
const IColumn & asof_column)
{ {
auto emplace_result = key_getter.emplaceKey(map, i, pool); auto emplace_result = key_getter.emplaceKey(map, i, pool);
typename HashMap::mapped_type * time_series_map = &emplace_result.getMapped(); typename HashMap::mapped_type * time_series_map = &emplace_result.getMapped();
@ -66,10 +73,10 @@ public:
HashJoin & join, HashJoin & join,
HashJoin::Type type, HashJoin::Type type,
MapsTemplate & maps, MapsTemplate & maps,
size_t rows,
const ColumnRawPtrs & key_columns, const ColumnRawPtrs & key_columns,
const Sizes & key_sizes, const Sizes & key_sizes,
Block * stored_block, const Block * stored_block,
const ScatteredBlock::Selector & selector,
ConstNullMapPtr null_map, ConstNullMapPtr null_map,
UInt8ColumnDataPtr join_mask, UInt8ColumnDataPtr join_mask,
Arena & pool, Arena & pool,
@ -83,14 +90,30 @@ public:
const Block & block_with_columns_to_add, const Block & block_with_columns_to_add,
const MapsTemplateVector & maps_, const MapsTemplateVector & maps_,
bool is_join_get = false); bool is_join_get = false);
static ScatteredBlock joinBlockImpl(
const HashJoin & join,
ScatteredBlock & block,
const Block & block_with_columns_to_add,
const MapsTemplateVector & maps_,
bool is_join_get = false);
private: private:
template <typename KeyGetter, bool is_asof_join> template <typename KeyGetter, bool is_asof_join>
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes); static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes);
template <typename KeyGetter, typename HashMap> template <typename KeyGetter, typename HashMap, typename Selector>
static size_t insertFromBlockImplTypeCase( static size_t insertFromBlockImplTypeCase(
HashJoin & join, HashMap & map, size_t rows, const ColumnRawPtrs & key_columns, HashJoin & join,
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted); HashMap & map,
const ColumnRawPtrs & key_columns,
const Sizes & key_sizes,
const Block * stored_block,
const Selector & selector,
ConstNullMapPtr null_map,
UInt8ColumnDataPtr join_mask,
Arena & pool,
bool & is_inserted);
template <typename AddedColumns> template <typename AddedColumns>
static size_t switchJoinRightColumns( static size_t switchJoinRightColumns(
@ -115,12 +138,13 @@ private:
/// Joins right table columns which indexes are present in right_indexes using specified map. /// 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). /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
template <typename KeyGetter, typename Map, bool need_filter, bool flag_per_row, typename AddedColumns> template <typename KeyGetter, typename Map, bool need_filter, bool flag_per_row, typename AddedColumns, typename Selector>
static size_t joinRightColumns( static size_t joinRightColumns(
std::vector<KeyGetter> && key_getter_vector, std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv, const std::vector<const Map *> & mapv,
AddedColumns & added_columns, AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags); JoinStuff::JoinUsedFlags & used_flags,
const Selector & selector);
template <bool need_filter> template <bool need_filter>
static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]); static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]);

View File

@ -1,5 +1,8 @@
#pragma once #pragma once
#include <type_traits>
#include <Interpreters/HashJoin/HashJoinMethods.h> #include <Interpreters/HashJoin/HashJoinMethods.h>
#include "Columns/IColumn.h"
#include "Interpreters/HashJoin/ScatteredBlock.h"
namespace DB namespace DB
{ {
@ -13,10 +16,10 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImpl(
HashJoin & join, HashJoin & join,
HashJoin::Type type, HashJoin::Type type,
MapsTemplate & maps, MapsTemplate & maps,
size_t rows,
const ColumnRawPtrs & key_columns, const ColumnRawPtrs & key_columns,
const Sizes & key_sizes, const Sizes & key_sizes,
Block * stored_block, const Block * stored_block,
const ScatteredBlock::Selector & selector,
ConstNullMapPtr null_map, ConstNullMapPtr null_map,
UInt8ColumnDataPtr join_mask, UInt8ColumnDataPtr join_mask,
Arena & pool, Arena & pool,
@ -33,9 +36,14 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImpl(
#define M(TYPE) \ #define M(TYPE) \
case HashJoin::Type::TYPE: \ case HashJoin::Type::TYPE: \
return insertFromBlockImplTypeCase< \ if (selector.isContinuousRange()) \
typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>( \ return insertFromBlockImplTypeCase< \
join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>( \
join, *maps.TYPE, key_columns, key_sizes, stored_block, selector.getRange(), null_map, join_mask, pool, is_inserted); \
else \
return insertFromBlockImplTypeCase< \
typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>( \
join, *maps.TYPE, key_columns, key_sizes, stored_block, selector.getIndexes(), null_map, join_mask, pool, is_inserted); \
break; break;
APPLY_FOR_JOIN_VARIANTS(M) APPLY_FOR_JOIN_VARIANTS(M)
@ -46,6 +54,22 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImpl(
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate> template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl( 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) const HashJoin & join, Block & block, const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get)
{
ScatteredBlock scattered_block{block};
auto ret = joinBlockImpl(join, scattered_block, block_with_columns_to_add, maps_, is_join_get);
ret.filterBySelector();
scattered_block.filterBySelector();
block = std::move(scattered_block.getSourceBlock());
return ret.getSourceBlock();
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
ScatteredBlock HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
const HashJoin & join,
ScatteredBlock & block,
const Block & block_with_columns_to_add,
const MapsTemplateVector & maps_,
bool is_join_get)
{ {
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features; constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
@ -56,16 +80,8 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; 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]); join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]);
} }
size_t existing_columns = block.columns(); auto & source_block = block.getSourceBlock();
size_t existing_columns = source_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 LEFT/INNER JOIN, the saved blocks do not contain keys.
* For FULL/RIGHT JOIN, the saved blocks contain keys; * For FULL/RIGHT JOIN, the saved blocks contain keys;
@ -90,26 +106,28 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
else else
added_columns.reserve(join_features.need_replication); added_columns.reserve(join_features.need_replication);
size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags); const size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags);
/// Do not hold memory for join_on_keys anymore /// Do not hold memory for join_on_keys anymore
added_columns.join_on_keys.clear(); added_columns.join_on_keys.clear();
Block remaining_block = sliceBlock(block, num_joined); auto remaining_block = block.cut(num_joined);
if (is_join_get) if (is_join_get)
added_columns.buildJoinGetOutput(); added_columns.buildJoinGetOutput();
else else
added_columns.buildOutput(); added_columns.buildOutput();
if constexpr (join_features.need_filter)
block.filter(added_columns.filter);
block.filterBySelector();
for (size_t i = 0; i < added_columns.size(); ++i) for (size_t i = 0; i < added_columns.size(); ++i)
block.insert(added_columns.moveColumn(i)); source_block.insert(added_columns.moveColumn(i));
std::vector<size_t> right_keys_to_replicate [[maybe_unused]]; std::vector<size_t> right_keys_to_replicate [[maybe_unused]];
if constexpr (join_features.need_filter) 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 /// 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) for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
{ {
@ -121,7 +139,7 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
const auto & left_column = block.getByName(join.required_right_keys_sources[i]); const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name); const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column); auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column);
block.insert(std::move(right_col)); source_block.insert(std::move(right_col));
} }
} }
else if (has_required_right_keys) else if (has_required_right_keys)
@ -137,28 +155,28 @@ Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
const auto & left_column = block.getByName(join.required_right_keys_sources[i]); 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); auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter);
block.insert(std::move(right_col)); source_block.insert(std::move(right_col));
if constexpr (join_features.need_replication) if constexpr (join_features.need_replication)
right_keys_to_replicate.push_back(block.getPositionByName(right_col_name)); right_keys_to_replicate.push_back(source_block.getPositionByName(right_col_name));
} }
} }
if constexpr (join_features.need_replication) if constexpr (join_features.need_replication)
{ {
std::unique_ptr<IColumn::Offsets> & offsets_to_replicate = added_columns.offsets_to_replicate; IColumn::Offsets & offsets = *added_columns.offsets_to_replicate;
/// If ALL ... JOIN - we replicate all the columns except the new ones. chassert(block);
chassert(offsets.size() == block.rows());
auto && columns = block.getSourceBlock().getColumns();
for (size_t i = 0; i < existing_columns; ++i) for (size_t i = 0; i < existing_columns; ++i)
{ columns[i] = columns[i]->replicate(offsets);
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate);
}
/// Replicate additional right keys
for (size_t pos : right_keys_to_replicate) for (size_t pos : right_keys_to_replicate)
{ columns[pos] = columns[pos]->replicate(offsets);
block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
} block.getSourceBlock().setColumns(columns);
block = ScatteredBlock(std::move(block).getSourceBlock());
} }
return remaining_block; return remaining_block;
} }
@ -180,14 +198,14 @@ KeyGetter HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::createKeyGetter(const
} }
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate> template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename KeyGetter, typename HashMap> template <typename KeyGetter, typename HashMap, typename Selector>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImplTypeCase( size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImplTypeCase(
HashJoin & join, HashJoin & join,
HashMap & map, HashMap & map,
size_t rows,
const ColumnRawPtrs & key_columns, const ColumnRawPtrs & key_columns,
const Sizes & key_sizes, const Sizes & key_sizes,
Block * stored_block, const Block * stored_block,
const Selector & selector,
ConstNullMapPtr null_map, ConstNullMapPtr null_map,
UInt8ColumnDataPtr join_mask, UInt8ColumnDataPtr join_mask,
Arena & pool, Arena & pool,
@ -205,9 +223,22 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImplTypeC
/// For ALL and ASOF join always insert values /// For ALL and ASOF join always insert values
is_inserted = !mapped_one || is_asof_join; is_inserted = !mapped_one || is_asof_join;
size_t rows = 0;
if constexpr (std::is_same_v<std::decay_t<Selector>, ScatteredBlock::Indexes>)
rows = selector.getData().size();
else
rows = selector.second - selector.first;
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
{ {
if (null_map && (*null_map)[i]) size_t ind = 0;
if constexpr (std::is_same_v<std::decay_t<Selector>, ScatteredBlock::Indexes>)
ind = selector.getData()[i];
else
ind = selector.first + i;
chassert(!null_map || ind < null_map->size());
if (null_map && (*null_map)[ind])
{ {
/// nulls are not inserted into hash table, /// nulls are not inserted into hash table,
/// keep them for RIGHT and FULL joins /// keep them for RIGHT and FULL joins
@ -216,15 +247,16 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImplTypeC
} }
/// Check condition for right table from ON section /// Check condition for right table from ON section
if (join_mask && !(*join_mask)[i]) chassert(!join_mask || ind < join_mask->size());
if (join_mask && !(*join_mask)[ind])
continue; continue;
if constexpr (is_asof_join) if constexpr (is_asof_join)
Inserter<HashMap, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); Inserter<HashMap, KeyGetter>::insertAsof(join, map, key_getter, stored_block, ind, pool, *asof_column);
else if constexpr (mapped_one) else if constexpr (mapped_one)
is_inserted |= Inserter<HashMap, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool); is_inserted |= Inserter<HashMap, KeyGetter>::insertOne(join, map, key_getter, stored_block, ind, pool);
else else
Inserter<HashMap, KeyGetter>::insertAll(join, map, key_getter, stored_block, i, pool); Inserter<HashMap, KeyGetter>::insertAll(join, map, key_getter, stored_block, ind, pool);
} }
return map.getBufferSizeInCells(); return map.getBufferSizeInCells();
} }
@ -318,26 +350,43 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumnsSwitchMu
if (added_columns.additional_filter_expression) if (added_columns.additional_filter_expression)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN");
return mapv.size() > 1 ? joinRightColumns<KeyGetter, Map, need_filter, true>( auto & block = added_columns.src_block;
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags) if (block.getSelector().isContinuousRange())
: joinRightColumns<KeyGetter, Map, need_filter, false>( {
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags); if (mapv.size() > 1)
return joinRightColumns<KeyGetter, Map, need_filter, true>(
std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getRange());
else
return joinRightColumns<KeyGetter, Map, need_filter, false>(
std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getRange());
}
else
{
if (mapv.size() > 1)
return joinRightColumns<KeyGetter, Map, need_filter, true>(
std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getIndexes());
else
return joinRightColumns<KeyGetter, Map, need_filter, false>(
std::move(key_getter_vector), mapv, added_columns, used_flags, block.getSelector().getIndexes());
}
} }
/// Joins right table columns which indexes are present in right_indexes using specified map. /// 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). /// 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 <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
template <typename KeyGetter, typename Map, bool need_filter, bool flag_per_row, typename AddedColumns> template <typename KeyGetter, typename Map, bool need_filter, bool flag_per_row, typename AddedColumns, typename Selector>
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumns( size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumns(
std::vector<KeyGetter> && key_getter_vector, std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv, const std::vector<const Map *> & mapv,
AddedColumns & added_columns, AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags) JoinStuff::JoinUsedFlags & used_flags,
const Selector & selector)
{ {
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features; constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
size_t rows = added_columns.rows_to_add; auto & block = added_columns.src_block;
size_t rows = block.rows();
if constexpr (need_filter) if constexpr (need_filter)
added_columns.filter = IColumn::Filter(rows, 0); added_columns.filter = IColumn::Filter(rows, 0);
if constexpr (!flag_per_row && (STRICTNESS == JoinStrictness::All || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Right))) if constexpr (!flag_per_row && (STRICTNESS == JoinStrictness::All || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Right)))
@ -353,6 +402,12 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumns(
size_t i = 0; size_t i = 0;
for (; i < rows; ++i) for (; i < rows; ++i)
{ {
size_t ind = 0;
if constexpr (std::is_same_v<std::decay_t<Selector>, ScatteredBlock::Indexes>)
ind = selector.getData()[i];
else
ind = selector.first + i;
if constexpr (join_features.need_replication) if constexpr (join_features.need_replication)
{ {
if (unlikely(current_offset >= max_joined_block_rows)) if (unlikely(current_offset >= max_joined_block_rows))
@ -368,12 +423,12 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumns(
for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) 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]; const auto & join_keys = added_columns.join_on_keys[onexpr_idx];
if (join_keys.null_map && (*join_keys.null_map)[i]) if (join_keys.null_map && (*join_keys.null_map)[ind])
continue; continue;
bool row_acceptable = !join_keys.isRowFiltered(i); bool row_acceptable = !join_keys.isRowFiltered(ind);
using FindResult = typename KeyGetter::FindResult; using FindResult = typename KeyGetter::FindResult;
auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), ind, pool) : FindResult();
if (find_result.isFound()) if (find_result.isFound())
{ {
@ -383,7 +438,7 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumns(
{ {
const IColumn & left_asof_key = added_columns.leftAsofKey(); const IColumn & left_asof_key = added_columns.leftAsofKey();
auto row_ref = mapped->findAsof(left_asof_key, i); auto row_ref = mapped->findAsof(left_asof_key, ind);
if (row_ref && row_ref->block) if (row_ref && row_ref->block)
{ {
setUsed<need_filter>(added_columns.filter, i); setUsed<need_filter>(added_columns.filter, i);
@ -848,23 +903,6 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumnsWithAddt
return left_row_iter; 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> template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
ColumnWithTypeAndName HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::copyLeftKeyColumnToRight( ColumnWithTypeAndName HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::copyLeftKeyColumnToRight(
const DataTypePtr & right_key_type, const DataTypePtr & right_key_type,

View File

@ -0,0 +1,337 @@
#pragma once
#include <Columns/ColumnsNumber.h>
#include <Columns/IColumn.h>
#include <Core/Block.h>
#include <base/defines.h>
#include <Common/PODArray.h>
#include <Poco/Logger.h>
#include <Common/logger_useful.h>
#include <boost/noncopyable.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace detail
{
/// Previously ConcurrentHashJoin used IColumn::scatter method to split input blocks to sub-blocks by hash.
/// To avoid copying of columns, we introduce a new class ScatteredBlock that holds a block and a selector.
/// So now each threads get a copy of the source input block and a selector that tells which rows are meant for the given thread.
/// Selector can be seen as just a list of indexes or rows that belong to the given thread.
/// One optimization is to use a continuous range instead of explicit list of indexes when selector contains all indexes from [L, R).
class Selector
{
public:
using Range = std::pair<size_t, size_t>;
using Indexes = ColumnUInt64;
using IndexesPtr = ColumnUInt64::MutablePtr;
/// [begin, end)
Selector(size_t begin, size_t end) : data(Range{begin, end}) { }
Selector() : Selector(0, 0) { }
explicit Selector(size_t size) : Selector(0, size) { }
explicit Selector(IndexesPtr && selector_) : data(initializeFromSelector(std::move(selector_))) { }
class Iterator
{
public:
using iterator_category = std::forward_iterator_tag;
using value_type = size_t;
using difference_type = std::ptrdiff_t;
using pointer = size_t *;
using reference = size_t &;
Iterator(const Selector & selector_, size_t idx_) : selector(selector_), idx(idx_) { }
size_t ALWAYS_INLINE operator*() const
{
chassert(idx < selector.size());
if (idx >= selector.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", idx, selector.size());
return selector[idx];
}
Iterator & ALWAYS_INLINE operator++()
{
if (idx >= selector.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} out of range size {}", idx, selector.size());
++idx;
return *this;
}
bool ALWAYS_INLINE operator!=(const Iterator & other) const { return idx != other.idx; }
private:
const Selector & selector;
size_t idx;
};
Iterator begin() const { return Iterator(*this, 0); }
Iterator end() const { return Iterator(*this, size()); }
size_t ALWAYS_INLINE operator[](size_t idx) const
{
chassert(idx < size());
if (std::holds_alternative<Range>(data))
{
const auto range = std::get<Range>(data);
return range.first + idx;
}
else
{
return std::get<IndexesPtr>(data)->getData()[idx];
}
}
size_t size() const
{
if (std::holds_alternative<Range>(data))
{
const auto range = std::get<Range>(data);
return range.second - range.first;
}
else
{
return std::get<IndexesPtr>(data)->size();
}
}
/// First selector contains first `num_rows` rows, second selector contains the rest
std::pair<Selector, Selector> split(size_t num_rows)
{
chassert(num_rows <= size());
if (std::holds_alternative<Range>(data))
{
const auto range = std::get<Range>(data);
if (num_rows == 0)
return {Selector(), Selector{range.first, range.second}};
if (num_rows == size())
return {Selector{range.first, range.second}, Selector()};
return {Selector(range.first, range.first + num_rows), Selector(range.first + num_rows, range.second)};
}
else
{
const auto & selector = std::get<IndexesPtr>(data)->getData();
auto && left = Selector(Indexes::create(selector.begin(), selector.begin() + num_rows));
auto && right = Selector(Indexes::create(selector.begin() + num_rows, selector.end()));
return {std::move(left), std::move(right)};
}
}
bool isContinuousRange() const { return std::holds_alternative<Range>(data); }
Range getRange() const
{
chassert(isContinuousRange());
return std::get<Range>(data);
}
const Indexes & getIndexes() const
{
chassert(!isContinuousRange());
return *std::get<IndexesPtr>(data);
}
std::string toString() const
{
if (std::holds_alternative<Range>(data))
{
const auto range = std::get<Range>(data);
return fmt::format("[{}, {})", range.first, range.second);
}
else
{
const auto & selector = std::get<IndexesPtr>(data)->getData();
return fmt::format("({})", fmt::join(selector, ","));
}
}
private:
using Data = std::variant<Range, IndexesPtr>;
Data initializeFromSelector(IndexesPtr && selector_)
{
const auto & selector = selector_->getData();
if (selector.empty())
return Range{0, 0};
/// selector represents continuous range
if (selector.back() == selector.front() + selector.size() - 1)
return Range{selector.front(), selector.front() + selector.size()};
return std::move(selector_);
}
Data data;
};
}
/// Source block + list of selected rows. See detail::Selector for more details.
struct ScatteredBlock : private boost::noncopyable
{
using Selector = detail::Selector;
using Indexes = Selector::Indexes;
using IndexesPtr = Selector::IndexesPtr;
ScatteredBlock() = default;
explicit ScatteredBlock(Block block_) : block(std::move(block_)), selector(block.rows()) { }
ScatteredBlock(Block block_, IndexesPtr && selector_) : block(std::move(block_)), selector(std::move(selector_)) { }
ScatteredBlock(Block block_, Selector selector_) : block(std::move(block_)), selector(std::move(selector_)) { }
ScatteredBlock(ScatteredBlock && other) noexcept : block(std::move(other.block)), selector(std::move(other.selector))
{
other.block.clear();
other.selector = {};
}
ScatteredBlock & operator=(ScatteredBlock && other) noexcept
{
if (this != &other)
{
block = std::move(other.block);
selector = std::move(other.selector);
other.block.clear();
other.selector = {};
}
return *this;
}
Block & getSourceBlock() & { return block; }
const Block & getSourceBlock() const & { return block; }
Block && getSourceBlock() && { return std::move(block); }
const auto & getSelector() const { return selector; }
explicit operator bool() const { return !!block; }
/// Accounts only selected rows
size_t rows() const { return selector.size(); }
/// In case of scattered block we account proportional share of the source block bytes.
/// For not scattered columns it will be trivial (bytes * N / N) calculation.
size_t allocatedBytes() const { return block.rows() ? block.allocatedBytes() * rows() / block.rows() : 0; }
ScatteredBlock shrinkToFit() const
{
if (wasScattered())
{
LOG_TEST(getLogger("HashJoin"), "shrinkToFit() is not supported for ScatteredBlock because blocks are shared");
return ScatteredBlock{block};
}
return ScatteredBlock{block.shrinkToFit()};
}
ScatteredBlock compress() const
{
if (wasScattered())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot compress ScatteredBlock");
return ScatteredBlock{block.compress()};
}
const auto & getByPosition(size_t i) const { return block.getByPosition(i); }
/// Whether `block` was scattered, i.e. `selector` != [0, block.rows())
bool wasScattered() const
{
return selector.size() != block.rows();
}
const ColumnWithTypeAndName & getByName(const std::string & name) const
{
return block.getByName(name);
}
/// Filters selector by mask discarding rows for which filter is false
void filter(const IColumn::Filter & filter)
{
chassert(block && block.rows() == filter.size());
IndexesPtr new_selector = Indexes::create();
new_selector->reserve(selector.size());
std::copy_if(
selector.begin(), selector.end(), std::back_inserter(new_selector->getData()), [&](size_t idx) { return filter[idx]; });
selector = Selector(std::move(new_selector));
}
/// Applies `selector` to the `block` in-place
void filterBySelector()
{
if (!block || !wasScattered())
return;
if (selector.isContinuousRange())
{
const auto range = selector.getRange();
for (size_t i = 0; i < block.columns(); ++i)
{
auto & col = block.getByPosition(i);
col.column = col.column->cut(range.first, range.second - range.first);
}
selector = Selector(block.rows());
return;
}
/// The general case when `selector` is non-trivial (likely the result of applying a filter)
auto columns = block.getColumns();
for (auto & col : columns)
col = col->index(selector.getIndexes(), /*limit*/ 0);
block.setColumns(columns);
selector = Selector(block.rows());
}
/// Cut first `num_rows` rows from `block` in place and returns block with remaining rows
ScatteredBlock cut(size_t num_rows)
{
SCOPE_EXIT(filterBySelector());
if (num_rows >= rows())
return ScatteredBlock{Block{}};
chassert(block);
auto && [first_num_rows, remaining_selector] = selector.split(num_rows);
auto remaining = ScatteredBlock{block, std::move(remaining_selector)};
selector = std::move(first_num_rows);
return remaining;
}
private:
Block block;
Selector selector;
};
using ScatteredBlocks = std::vector<ScatteredBlock>;
struct ExtraScatteredBlocks
{
ScatteredBlocks remaining_blocks;
bool rows() const
{
return std::ranges::any_of(remaining_blocks, [](const auto & bl) { return bl.rows(); });
}
};
}

View File

@ -2,9 +2,10 @@
#include <memory> #include <memory>
#include <Core/Names.h>
#include <Core/Block.h>
#include <Columns/IColumn.h> #include <Columns/IColumn.h>
#include <Core/Block.h>
#include <Core/Names.h>
#include <Interpreters/HashJoin/ScatteredBlock.h>
#include <Common/Exception.h> #include <Common/Exception.h>
namespace DB namespace DB
@ -89,6 +90,13 @@ public:
/// Could be called from different threads in parallel. /// Could be called from different threads in parallel.
virtual void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) = 0; virtual void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) = 0;
virtual bool isScatteredJoin() const { return false; }
virtual void joinBlock(
[[maybe_unused]] Block & block, [[maybe_unused]] ExtraScatteredBlocks & extra_blocks, [[maybe_unused]] std::vector<Block> & res)
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "joinBlock is not supported for {}", getName());
}
/** Set/Get totals for right table /** Set/Get totals for right table
* Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. * Keep "totals" (separate part of dataset, see WITH TOTALS) to use later.
*/ */

View File

@ -1887,6 +1887,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
joined_plan->getCurrentHeader(), joined_plan->getCurrentHeader(),
expressions.join, expressions.join,
settings[Setting::max_block_size], settings[Setting::max_block_size],
0,
max_streams, max_streams,
analysis_result.optimize_read_in_order); analysis_result.optimize_read_in_order);

View File

@ -45,6 +45,16 @@ void Expected::highlight(HighlightedRange range)
return; return;
auto it = highlights.lower_bound(range); auto it = highlights.lower_bound(range);
/// Highlights are sorted by their starting position.
/// lower_bound(range) will find the first highlight where begin >= range.begin.
/// However, this does not ensure that the previous highlight's end <= range.begin.
/// By checking the previous highlight, if it exists, we ensure that
/// for each highlight x and the next one y: x.end <= y.begin, thus preventing any overlap.
if (it != highlights.begin())
it = std::prev(it);
while (it != highlights.end() && range.begin < it->end) while (it != highlights.end() && range.begin < it->end)
{ {
if (intersects(range.begin, range.end, it->begin, it->end)) if (intersects(range.begin, range.end, it->begin, it->end))

View File

@ -2,8 +2,8 @@
#include <Core/Settings.h> #include <Core/Settings.h>
#include <Common/scope_guard_safe.h>
#include <Core/ParallelReplicasMode.h> #include <Core/ParallelReplicasMode.h>
#include <Common/scope_guard_safe.h>
#include <Columns/ColumnAggregateFunction.h> #include <Columns/ColumnAggregateFunction.h>
@ -104,6 +104,7 @@ namespace Setting
extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere;
extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool optimize_move_to_prewhere_if_final;
extern const SettingsBool use_concurrency_control; extern const SettingsBool use_concurrency_control;
extern const SettingsUInt64 min_joined_block_size_bytes;
} }
namespace ErrorCodes namespace ErrorCodes
@ -1267,11 +1268,13 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
plan_to_add_cast.addStep(std::move(cast_join_columns_step)); plan_to_add_cast.addStep(std::move(cast_join_columns_step));
} }
JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, JoinTreeQueryPlan buildQueryPlanForJoinNode(
const QueryTreeNodePtr & join_table_expression,
JoinTreeQueryPlan left_join_tree_query_plan, JoinTreeQueryPlan left_join_tree_query_plan,
JoinTreeQueryPlan right_join_tree_query_plan, JoinTreeQueryPlan right_join_tree_query_plan,
const ColumnIdentifierSet & outer_scope_columns, const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context) PlannerContextPtr & planner_context,
const SelectQueryInfo & select_query_info)
{ {
auto & join_node = join_table_expression->as<JoinNode &>(); auto & join_node = join_table_expression->as<JoinNode &>();
if (left_join_tree_query_plan.from_stage != QueryProcessingStage::FetchColumns) if (left_join_tree_query_plan.from_stage != QueryProcessingStage::FetchColumns)
@ -1554,7 +1557,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
} }
const Block & right_header = right_plan.getCurrentHeader(); const Block & right_header = right_plan.getCurrentHeader();
auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context); auto join_algorithm = chooseJoinAlgorithm(
table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context, select_query_info);
auto result_plan = QueryPlan(); auto result_plan = QueryPlan();
@ -1646,6 +1650,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
right_plan.getCurrentHeader(), right_plan.getCurrentHeader(),
std::move(join_algorithm), std::move(join_algorithm),
settings[Setting::max_block_size], settings[Setting::max_block_size],
settings[Setting::min_joined_block_size_bytes],
settings[Setting::max_threads], settings[Setting::max_threads],
false /*optimize_read_in_order*/); false /*optimize_read_in_order*/);
@ -1912,11 +1917,13 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
auto left_query_plan = std::move(query_plans_stack.back()); auto left_query_plan = std::move(query_plans_stack.back());
query_plans_stack.pop_back(); query_plans_stack.pop_back();
query_plans_stack.push_back(buildQueryPlanForJoinNode(table_expression, query_plans_stack.push_back(buildQueryPlanForJoinNode(
table_expression,
std::move(left_query_plan), std::move(left_query_plan),
std::move(right_query_plan), std::move(right_query_plan),
table_expressions_outer_scope_columns[i], table_expressions_outer_scope_columns[i],
planner_context)); planner_context,
select_query_info));
} }
else else
{ {

View File

@ -789,12 +789,14 @@ std::shared_ptr<DirectKeyValueJoin> tryDirectJoin(const std::shared_ptr<TableJoi
} }
} }
static std::shared_ptr<IJoin> tryCreateJoin(JoinAlgorithm algorithm, static std::shared_ptr<IJoin> tryCreateJoin(
JoinAlgorithm algorithm,
std::shared_ptr<TableJoin> & table_join, std::shared_ptr<TableJoin> & table_join,
const QueryTreeNodePtr & right_table_expression, const QueryTreeNodePtr & right_table_expression,
const Block & left_table_expression_header, const Block & left_table_expression_header,
const Block & right_table_expression_header, const Block & right_table_expression_header,
const PlannerContextPtr & planner_context) const PlannerContextPtr & planner_context,
const SelectQueryInfo & select_query_info)
{ {
if (table_join->kind() == JoinKind::Paste) if (table_join->kind() == JoinKind::Paste)
return std::make_shared<PasteJoin>(table_join, right_table_expression_header); return std::make_shared<PasteJoin>(table_join, right_table_expression_header);
@ -824,7 +826,7 @@ static std::shared_ptr<IJoin> tryCreateJoin(JoinAlgorithm algorithm,
{ {
const auto & settings = query_context->getSettingsRef(); const auto & settings = query_context->getSettingsRef();
StatsCollectingParams params{ StatsCollectingParams params{
calculateCacheKey(table_join, right_table_expression), calculateCacheKey(table_join, right_table_expression, select_query_info),
settings[Setting::collect_hash_table_stats_during_joins], settings[Setting::collect_hash_table_stats_during_joins],
query_context->getServerSettings()[ServerSetting::max_entries_for_hash_table_stats], query_context->getServerSettings()[ServerSetting::max_entries_for_hash_table_stats],
settings[Setting::max_size_to_preallocate_for_joins]}; settings[Setting::max_size_to_preallocate_for_joins]};
@ -866,11 +868,13 @@ static std::shared_ptr<IJoin> tryCreateJoin(JoinAlgorithm algorithm,
return nullptr; return nullptr;
} }
std::shared_ptr<IJoin> chooseJoinAlgorithm(std::shared_ptr<TableJoin> & table_join, std::shared_ptr<IJoin> chooseJoinAlgorithm(
std::shared_ptr<TableJoin> & table_join,
const QueryTreeNodePtr & right_table_expression, const QueryTreeNodePtr & right_table_expression,
const Block & left_table_expression_header, const Block & left_table_expression_header,
const Block & right_table_expression_header, const Block & right_table_expression_header,
const PlannerContextPtr & planner_context) const PlannerContextPtr & planner_context,
const SelectQueryInfo & select_query_info)
{ {
if (table_join->getMixedJoinExpression() if (table_join->getMixedJoinExpression()
&& !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH)
@ -926,7 +930,14 @@ std::shared_ptr<IJoin> chooseJoinAlgorithm(std::shared_ptr<TableJoin> & table_jo
for (auto algorithm : table_join->getEnabledJoinAlgorithms()) for (auto algorithm : table_join->getEnabledJoinAlgorithms())
{ {
auto join = tryCreateJoin(algorithm, table_join, right_table_expression, left_table_expression_header, right_table_expression_header, planner_context); auto join = tryCreateJoin(
algorithm,
table_join,
right_table_expression,
left_table_expression_header,
right_table_expression_header,
planner_context,
select_query_info);
if (join) if (join)
return join; return join;
} }

View File

@ -12,6 +12,8 @@
namespace DB namespace DB
{ {
struct SelectQueryInfo;
/** Join clause represent single JOIN ON section clause. /** Join clause represent single JOIN ON section clause.
* Join clause consists of JOIN keys and conditions. * Join clause consists of JOIN keys and conditions.
* *
@ -218,10 +220,11 @@ std::optional<bool> tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join
* Table join structure can be modified during JOIN algorithm choosing for special JOIN algorithms. * Table join structure can be modified during JOIN algorithm choosing for special JOIN algorithms.
* For example JOIN with Dictionary engine, or JOIN with JOIN engine. * For example JOIN with Dictionary engine, or JOIN with JOIN engine.
*/ */
std::shared_ptr<IJoin> chooseJoinAlgorithm(std::shared_ptr<TableJoin> & table_join, std::shared_ptr<IJoin> chooseJoinAlgorithm(
std::shared_ptr<TableJoin> & table_join,
const QueryTreeNodePtr & right_table_expression, const QueryTreeNodePtr & right_table_expression,
const Block & left_table_expression_header, const Block & left_table_expression_header,
const Block & right_table_expression_header, const Block & right_table_expression_header,
const PlannerContextPtr & planner_context); const PlannerContextPtr & planner_context,
const SelectQueryInfo & select_query_info);
} }

View File

@ -1,9 +1,10 @@
#include <Processors/QueryPlan/JoinStep.h> #include <IO/Operators.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Processors/Transforms/JoiningTransform.h>
#include <Interpreters/IJoin.h> #include <Interpreters/IJoin.h>
#include <Interpreters/TableJoin.h> #include <Interpreters/TableJoin.h>
#include <IO/Operators.h> #include <Processors/QueryPlan/JoinStep.h>
#include <Processors/Transforms/JoiningTransform.h>
#include <Processors/Transforms/SquashingTransform.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Common/JSONBuilder.h> #include <Common/JSONBuilder.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
@ -43,9 +44,14 @@ JoinStep::JoinStep(
const Header & right_header_, const Header & right_header_,
JoinPtr join_, JoinPtr join_,
size_t max_block_size_, size_t max_block_size_,
size_t min_block_size_bytes_,
size_t max_streams_, size_t max_streams_,
bool keep_left_read_in_order_) bool keep_left_read_in_order_)
: join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_) : join(std::move(join_))
, max_block_size(max_block_size_)
, min_block_size_bytes(min_block_size_bytes_)
, max_streams(max_streams_)
, keep_left_read_in_order(keep_left_read_in_order_)
{ {
updateInputHeaders({left_header_, right_header_}); updateInputHeaders({left_header_, right_header_});
} }
@ -63,15 +69,24 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines
return joined_pipeline; return joined_pipeline;
} }
return QueryPipelineBuilder::joinPipelinesRightLeft( auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft(
std::move(pipelines[0]), std::move(pipelines[0]),
std::move(pipelines[1]), std::move(pipelines[1]),
join, join,
*output_header, *output_header,
max_block_size, max_block_size,
min_block_size_bytes,
max_streams, max_streams,
keep_left_read_in_order, keep_left_read_in_order,
&processors); &processors);
if (join->supportParallelJoin())
{
pipeline->addSimpleTransform([&](const Block & header)
{ return std::make_shared<SimpleSquashingChunksTransform>(header, 0, min_block_size_bytes); });
}
return pipeline;
} }
bool JoinStep::allowPushDownToRight() const bool JoinStep::allowPushDownToRight() const

View File

@ -18,6 +18,7 @@ public:
const Header & right_header_, const Header & right_header_,
JoinPtr join_, JoinPtr join_,
size_t max_block_size_, size_t max_block_size_,
size_t min_block_size_bytes_,
size_t max_streams_, size_t max_streams_,
bool keep_left_read_in_order_); bool keep_left_read_in_order_);
@ -39,6 +40,7 @@ private:
JoinPtr join; JoinPtr join;
size_t max_block_size; size_t max_block_size;
size_t min_block_size_bytes;
size_t max_streams; size_t max_streams;
bool keep_left_read_in_order; bool keep_left_read_in_order;
}; };

View File

@ -647,7 +647,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
range.begin = exact_ranges[i].end; range.begin = exact_ranges[i].end;
ordinary_reading_marks -= exact_ranges[i].end - exact_ranges[i].begin; ordinary_reading_marks -= exact_ranges[i].end - exact_ranges[i].begin;
exact_count += part_with_ranges.data_part->index_granularity.getRowsCountInRange(exact_ranges[i]); exact_count += part_with_ranges.data_part->index_granularity->getRowsCountInRange(exact_ranges[i]);
++i; ++i;
} }

View File

@ -201,7 +201,7 @@ public:
size_t getMarkRows(size_t part_idx, size_t mark) const size_t getMarkRows(size_t part_idx, size_t mark) const
{ {
return parts[part_idx].data_part->index_granularity.getMarkRows(mark); return parts[part_idx].data_part->index_granularity->getMarkRows(mark);
} }
private: private:
const RangesInDataParts & parts; const RangesInDataParts & parts;
@ -444,7 +444,7 @@ SplitPartsRangesResult splitPartsRanges(RangesInDataParts ranges_in_data_parts,
parts_ranges.push_back( parts_ranges.push_back(
{index_access.getValue(part_index, range.begin), range, part_index, PartsRangesIterator::EventType::RangeStart}); {index_access.getValue(part_index, range.begin), range, part_index, PartsRangesIterator::EventType::RangeStart});
const bool value_is_defined_at_end_mark = range.end < index_granularity.getMarksCount(); const bool value_is_defined_at_end_mark = range.end < index_granularity->getMarksCount();
if (!value_is_defined_at_end_mark) if (!value_is_defined_at_end_mark)
continue; continue;
@ -667,7 +667,7 @@ std::pair<std::vector<RangesInDataParts>, std::vector<Values>> splitIntersecting
PartRangeIndex parts_range_start_index(parts_range_start); PartRangeIndex parts_range_start_index(parts_range_start);
parts_ranges_queue.push({std::move(parts_range_start), std::move(parts_range_start_index)}); parts_ranges_queue.push({std::move(parts_range_start), std::move(parts_range_start_index)});
const bool value_is_defined_at_end_mark = range.end < index_granularity.getMarksCount(); const bool value_is_defined_at_end_mark = range.end < index_granularity->getMarksCount();
if (!value_is_defined_at_end_mark) if (!value_is_defined_at_end_mark)
continue; continue;

View File

@ -667,7 +667,7 @@ Pipe ReadFromMergeTree::readInOrder(
part_with_ranges.ranges.size(), part_with_ranges.ranges.size(),
read_type == ReadType::InReverseOrder ? " reverse " : " ", read_type == ReadType::InReverseOrder ? " reverse " : " ",
part_with_ranges.data_part->name, total_rows, part_with_ranges.data_part->name, total_rows,
part_with_ranges.data_part->index_granularity.getMarkStartingRow(part_with_ranges.ranges.front().begin)); part_with_ranges.data_part->index_granularity->getMarkStartingRow(part_with_ranges.ranges.front().begin));
MergeTreeSelectAlgorithmPtr algorithm; MergeTreeSelectAlgorithmPtr algorithm;
if (read_type == ReadType::InReverseOrder) if (read_type == ReadType::InReverseOrder)
@ -1759,7 +1759,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
return std::make_shared<AnalysisResult>(std::move(result)); return std::make_shared<AnalysisResult>(std::move(result));
for (const auto & part : parts) for (const auto & part : parts)
total_marks_pk += part->index_granularity.getMarksCountWithoutFinal(); total_marks_pk += part->index_granularity->getMarksCountWithoutFinal();
parts_before_pk = parts.size(); parts_before_pk = parts.size();
auto reader_settings = getMergeTreeReaderSettings(context_, query_info_); auto reader_settings = getMergeTreeReaderSettings(context_, query_info_);

View File

@ -75,8 +75,9 @@ IProcessor::Status JoiningTransform::prepare()
/// Output if has data. /// Output if has data.
if (has_output) if (has_output)
{ {
output.push(std::move(output_chunk)); output.push(std::move(output_chunks.front()));
has_output = false; output_chunks.pop_front();
has_output = !output_chunks.empty();
return Status::PortFull; return Status::PortFull;
} }
@ -122,10 +123,10 @@ void JoiningTransform::work()
{ {
if (has_input) if (has_input)
{ {
chassert(output_chunks.empty());
transform(input_chunk); transform(input_chunk);
output_chunk.swap(input_chunk);
has_input = not_processed != nullptr; has_input = not_processed != nullptr;
has_output = !output_chunk.empty(); has_output = !output_chunks.empty();
} }
else else
{ {
@ -153,8 +154,7 @@ void JoiningTransform::work()
return; return;
} }
auto rows = block.rows(); output_chunks.emplace_back(block.getColumns(), block.rows());
output_chunk.setColumns(block.getColumns(), rows);
has_output = true; has_output = true;
} }
} }
@ -173,7 +173,7 @@ void JoiningTransform::transform(Chunk & chunk)
} }
} }
Block block; Blocks res;
if (on_totals) if (on_totals)
{ {
const auto & left_totals = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); const auto & left_totals = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns());
@ -184,39 +184,58 @@ void JoiningTransform::transform(Chunk & chunk)
if (default_totals && !right_totals) if (default_totals && !right_totals)
return; return;
block = outputs.front().getHeader().cloneEmpty(); res.emplace_back();
JoinCommon::joinTotals(left_totals, right_totals, join->getTableJoin(), block); res.back() = outputs.front().getHeader().cloneEmpty();
JoinCommon::joinTotals(left_totals, right_totals, join->getTableJoin(), res.back());
} }
else else
block = readExecute(chunk); res = readExecute(chunk);
auto num_rows = block.rows();
chunk.setColumns(block.getColumns(), num_rows); std::ranges::for_each(res, [this](Block & block) { output_chunks.emplace_back(block.getColumns(), block.rows()); });
} }
Block JoiningTransform::readExecute(Chunk & chunk) Blocks JoiningTransform::readExecute(Chunk & chunk)
{ {
Block res; Blocks res;
Block block;
auto join_block = [&]()
{
if (join->isScatteredJoin())
{
join->joinBlock(block, remaining_blocks, res);
if (remaining_blocks.rows())
not_processed = std::make_shared<ExtraBlock>();
else
not_processed.reset();
}
else
{
join->joinBlock(block, not_processed);
res.push_back(std::move(block));
}
};
if (!not_processed) if (!not_processed)
{ {
if (chunk.hasColumns()) if (chunk.hasColumns())
res = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); block = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns());
if (res) if (block)
join->joinBlock(res, not_processed); join_block();
} }
else if (not_processed->empty()) /// There's not processed data inside expression. else if (not_processed->empty()) /// There's not processed data inside expression.
{ {
if (chunk.hasColumns()) if (chunk.hasColumns())
res = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); block = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns());
not_processed.reset(); not_processed.reset();
join->joinBlock(res, not_processed); join_block();
} }
else else
{ {
res = std::move(not_processed->block); block = std::move(not_processed->block);
join->joinBlock(res, not_processed); join_block();
} }
return res; return res;

View File

@ -1,6 +1,10 @@
#pragma once #pragma once
#include <Processors/IProcessor.h>
#include <Interpreters/HashJoin/ScatteredBlock.h>
#include <Processors/Chunk.h> #include <Processors/Chunk.h>
#include <Processors/IProcessor.h>
#include <deque>
#include <memory> #include <memory>
namespace DB namespace DB
@ -66,7 +70,7 @@ protected:
private: private:
Chunk input_chunk; Chunk input_chunk;
Chunk output_chunk; std::deque<Chunk> output_chunks;
bool has_input = false; bool has_input = false;
bool has_output = false; bool has_output = false;
bool stop_reading = false; bool stop_reading = false;
@ -80,13 +84,16 @@ private:
bool default_totals; bool default_totals;
bool initialized = false; bool initialized = false;
/// Only used with ConcurrentHashJoin
ExtraScatteredBlocks remaining_blocks;
ExtraBlockPtr not_processed; ExtraBlockPtr not_processed;
FinishCounterPtr finish_counter; FinishCounterPtr finish_counter;
IBlocksStreamPtr non_joined_blocks; IBlocksStreamPtr non_joined_blocks;
size_t max_block_size; size_t max_block_size;
Block readExecute(Chunk & chunk); Blocks readExecute(Chunk & chunk);
}; };
/// Fills Join with block from right table. /// Fills Join with block from right table.

View File

@ -78,7 +78,7 @@ Chunk SimpleSquashingChunksTransform::generate()
bool SimpleSquashingChunksTransform::canGenerate() bool SimpleSquashingChunksTransform::canGenerate()
{ {
return !squashed_chunk.empty(); return squashed_chunk.hasRows();
} }
Chunk SimpleSquashingChunksTransform::getRemaining() Chunk SimpleSquashingChunksTransform::getRemaining()

View File

@ -26,6 +26,7 @@
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h> #include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
#include <Processors/Transforms/PartialSortingTransform.h> #include <Processors/Transforms/PartialSortingTransform.h>
#include <Processors/Transforms/PasteJoinTransform.h> #include <Processors/Transforms/PasteJoinTransform.h>
#include <Processors/Transforms/SquashingTransform.h>
#include <Processors/Transforms/TotalsHavingTransform.h> #include <Processors/Transforms/TotalsHavingTransform.h>
#include <QueryPipeline/narrowPipe.h> #include <QueryPipeline/narrowPipe.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
@ -385,6 +386,7 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
JoinPtr join, JoinPtr join,
const Block & output_header, const Block & output_header,
size_t max_block_size, size_t max_block_size,
size_t min_block_size_bytes,
size_t max_streams, size_t max_streams,
bool keep_left_read_in_order, bool keep_left_read_in_order,
Processors * collected_processors) Processors * collected_processors)
@ -441,9 +443,12 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
Processors processors; Processors processors;
for (auto & outport : outports) for (auto & outport : outports)
{ {
auto squashing = std::make_shared<SimpleSquashingChunksTransform>(right->getHeader(), 0, min_block_size_bytes);
connect(*outport, squashing->getInputs().front());
processors.emplace_back(squashing);
auto adding_joined = std::make_shared<FillingRightJoinSideTransform>(right->getHeader(), join); auto adding_joined = std::make_shared<FillingRightJoinSideTransform>(right->getHeader(), join);
connect(*outport, adding_joined->getInputs().front()); connect(squashing->getOutputPort(), adding_joined->getInputs().front());
processors.emplace_back(adding_joined); processors.emplace_back(std::move(adding_joined));
} }
return processors; return processors;
}; };
@ -497,10 +502,13 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
Block left_header = left->getHeader(); Block left_header = left->getHeader();
for (size_t i = 0; i < num_streams; ++i) for (size_t i = 0; i < num_streams; ++i)
{ {
auto squashing = std::make_shared<SimpleSquashingChunksTransform>(left->getHeader(), 0, min_block_size_bytes);
connect(**lit, squashing->getInputs().front());
auto joining = std::make_shared<JoiningTransform>( auto joining = std::make_shared<JoiningTransform>(
left_header, output_header, join, max_block_size, false, default_totals, finish_counter); left_header, output_header, join, max_block_size, false, default_totals, finish_counter);
connect(**lit, joining->getInputs().front()); connect(squashing->getOutputPort(), joining->getInputs().front());
connect(**rit, joining->getInputs().back()); connect(**rit, joining->getInputs().back());
if (delayed_root) if (delayed_root)
{ {
@ -532,6 +540,7 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
if (collected_processors) if (collected_processors)
collected_processors->emplace_back(joining); collected_processors->emplace_back(joining);
left->pipe.processors->emplace_back(std::move(squashing));
left->pipe.processors->emplace_back(std::move(joining)); left->pipe.processors->emplace_back(std::move(joining));
} }

View File

@ -126,6 +126,7 @@ public:
JoinPtr join, JoinPtr join,
const Block & output_header, const Block & output_header,
size_t max_block_size, size_t max_block_size,
size_t min_block_size_bytes,
size_t max_streams, size_t max_streams,
bool keep_left_read_in_order, bool keep_left_read_in_order,
Processors * collected_processors = nullptr); Processors * collected_processors = nullptr);

View File

@ -29,6 +29,7 @@
#include <Storages/MergeTree/checkDataPart.h> #include <Storages/MergeTree/checkDataPart.h>
#include <Storages/MergeTree/Backup.h> #include <Storages/MergeTree/Backup.h>
#include <Storages/StorageReplicatedMergeTree.h> #include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeIndexGranularityAdaptive.h>
#include <base/JSON.h> #include <base/JSON.h>
#include <boost/algorithm/string/join.hpp> #include <boost/algorithm/string/join.hpp>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
@ -626,11 +627,12 @@ UInt64 IMergeTreeDataPart::getIndexSizeInAllocatedBytes() const
UInt64 IMergeTreeDataPart::getIndexGranularityBytes() const UInt64 IMergeTreeDataPart::getIndexGranularityBytes() const
{ {
return index_granularity.getBytesSize(); return index_granularity->getBytesSize();
} }
UInt64 IMergeTreeDataPart::getIndexGranularityAllocatedBytes() const UInt64 IMergeTreeDataPart::getIndexGranularityAllocatedBytes() const
{ {
return index_granularity.getBytesAllocated(); return index_granularity->getBytesAllocated();
} }
void IMergeTreeDataPart::assertState(const std::initializer_list<MergeTreeDataPartState> & affordable_states) const void IMergeTreeDataPart::assertState(const std::initializer_list<MergeTreeDataPartState> & affordable_states) const
@ -661,7 +663,7 @@ void IMergeTreeDataPart::assertOnDisk() const
UInt64 IMergeTreeDataPart::getMarksCount() const UInt64 IMergeTreeDataPart::getMarksCount() const
{ {
return index_granularity.getMarksCount(); return index_granularity->getMarksCount();
} }
UInt64 IMergeTreeDataPart::getExistingBytesOnDisk() const UInt64 IMergeTreeDataPart::getExistingBytesOnDisk() const
@ -746,7 +748,6 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
loadChecksums(require_columns_checksums); loadChecksums(require_columns_checksums);
loadIndexGranularity(); loadIndexGranularity();
index_granularity.shrinkToFitInMemory();
if (!(*storage.getSettings())[MergeTreeSetting::primary_key_lazy_load]) if (!(*storage.getSettings())[MergeTreeSetting::primary_key_lazy_load])
getIndex(); getIndex();
@ -942,13 +943,13 @@ void IMergeTreeDataPart::loadIndex() const
for (size_t i = 0; i < key_size; ++i) for (size_t i = 0; i < key_size; ++i)
{ {
loaded_index[i] = primary_key.data_types[i]->createColumn(); loaded_index[i] = primary_key.data_types[i]->createColumn();
loaded_index[i]->reserve(index_granularity.getMarksCount()); loaded_index[i]->reserve(index_granularity->getMarksCount());
} }
String index_name = "primary" + getIndexExtensionFromFilesystem(getDataPartStorage()); String index_name = "primary" + getIndexExtensionFromFilesystem(getDataPartStorage());
String index_path = fs::path(getDataPartStorage().getRelativePath()) / index_name; String index_path = fs::path(getDataPartStorage().getRelativePath()) / index_name;
auto index_file = metadata_manager->read(index_name); auto index_file = metadata_manager->read(index_name);
size_t marks_count = index_granularity.getMarksCount(); size_t marks_count = index_granularity->getMarksCount();
Serializations key_serializations(key_size); Serializations key_serializations(key_size);
for (size_t j = 0; j < key_size; ++j) for (size_t j = 0; j < key_size; ++j)
@ -988,6 +989,8 @@ void IMergeTreeDataPart::loadIndex() const
"{}, read: {})", index_path, marks_count, loaded_index[i]->size()); "{}, read: {})", index_path, marks_count, loaded_index[i]->size());
} }
LOG_TEST(storage.log, "Loaded primary key index for part {}, {} columns are kept in memory", name, key_size);
if (!index_file->eof()) if (!index_file->eof())
throw Exception(ErrorCodes::EXPECTED_END_OF_FILE, "Index file {} is unexpectedly long", index_path); throw Exception(ErrorCodes::EXPECTED_END_OF_FILE, "Index file {} is unexpectedly long", index_path);
@ -1361,7 +1364,7 @@ void IMergeTreeDataPart::loadRowsCount()
assertEOF(*buf); assertEOF(*buf);
}; };
if (index_granularity.empty()) if (index_granularity->empty())
{ {
rows_count = 0; rows_count = 0;
} }
@ -1396,9 +1399,9 @@ void IMergeTreeDataPart::loadRowsCount()
backQuote(column.name), rows_in_column, name, rows_count); backQuote(column.name), rows_in_column, name, rows_count);
} }
size_t last_possibly_incomplete_mark_rows = index_granularity.getLastNonFinalMarkRows(); size_t last_possibly_incomplete_mark_rows = index_granularity->getLastNonFinalMarkRows();
/// All this rows have to be written in column /// All this rows have to be written in column
size_t index_granularity_without_last_mark = index_granularity.getTotalRows() - last_possibly_incomplete_mark_rows; size_t index_granularity_without_last_mark = index_granularity->getTotalRows() - last_possibly_incomplete_mark_rows;
/// We have more rows in column than in index granularity without last possibly incomplete mark /// We have more rows in column than in index granularity without last possibly incomplete mark
if (rows_in_column < index_granularity_without_last_mark) if (rows_in_column < index_granularity_without_last_mark)
{ {
@ -1408,7 +1411,7 @@ void IMergeTreeDataPart::loadRowsCount()
"and size of single value, " "and size of single value, "
"but index granularity in part {} without last mark has {} rows, which " "but index granularity in part {} without last mark has {} rows, which "
"is more than in column", "is more than in column",
backQuote(column.name), rows_in_column, name, index_granularity.getTotalRows()); backQuote(column.name), rows_in_column, name, index_granularity->getTotalRows());
} }
/// In last mark we actually written less or equal rows than stored in last mark of index granularity /// In last mark we actually written less or equal rows than stored in last mark of index granularity
@ -1456,8 +1459,8 @@ void IMergeTreeDataPart::loadRowsCount()
column.name, column_size, sizeof_field); column.name, column_size, sizeof_field);
} }
size_t last_mark_index_granularity = index_granularity.getLastNonFinalMarkRows(); size_t last_mark_index_granularity = index_granularity->getLastNonFinalMarkRows();
size_t rows_approx = index_granularity.getTotalRows(); size_t rows_approx = index_granularity->getTotalRows();
if (!(rows_count <= rows_approx && rows_approx < rows_count + last_mark_index_granularity)) if (!(rows_count <= rows_approx && rows_approx < rows_count + last_mark_index_granularity))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of column {}: " throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of column {}: "
"{} rows, expected {}+-{} rows according to the index", "{} rows, expected {}+-{} rows according to the index",
@ -1520,7 +1523,7 @@ UInt64 IMergeTreeDataPart::readExistingRowsCount()
while (current_row < rows_count) while (current_row < rows_count)
{ {
size_t rows_to_read = index_granularity.getMarkRows(current_mark); size_t rows_to_read = index_granularity->getMarkRows(current_mark);
continue_reading = (current_mark != 0); continue_reading = (current_mark != 0);
Columns result; Columns result;
@ -1968,6 +1971,9 @@ void IMergeTreeDataPart::initializeIndexGranularityInfo()
index_granularity_info = MergeTreeIndexGranularityInfo(storage, *mrk_type); index_granularity_info = MergeTreeIndexGranularityInfo(storage, *mrk_type);
else else
index_granularity_info = MergeTreeIndexGranularityInfo(storage, part_type); index_granularity_info = MergeTreeIndexGranularityInfo(storage, part_type);
/// It may be converted to constant index granularity after loading it.
index_granularity = std::make_unique<MergeTreeIndexGranularityAdaptive>();
} }
void IMergeTreeDataPart::remove() void IMergeTreeDataPart::remove()
@ -2241,9 +2247,9 @@ void IMergeTreeDataPart::checkConsistency(bool require_part_metadata) const
"part_state: [{}]", "part_state: [{}]",
columns.toString(), columns.toString(),
index_granularity_info.getMarkSizeInBytes(columns.size()), index_granularity_info.getMarkSizeInBytes(columns.size()),
index_granularity.getMarksCount(), index_granularity->getMarksCount(),
index_granularity_info.describe(), index_granularity_info.describe(),
index_granularity.describe(), index_granularity->describe(),
part_state); part_state);
e.addMessage(debug_info); e.addMessage(debug_info);

View File

@ -321,7 +321,7 @@ public:
/// Amount of rows between marks /// Amount of rows between marks
/// As index always loaded into memory /// As index always loaded into memory
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityPtr index_granularity;
/// Index that for each part stores min and max values of a set of columns. This allows quickly excluding /// Index that for each part stores min and max values of a set of columns. This allows quickly excluding
/// parts based on conditions on these columns imposed by a query. /// parts based on conditions on these columns imposed by a query.

View File

@ -1,5 +1,6 @@
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h> #include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
#include <Common/MemoryTrackerBlockerInThread.h> #include <Common/MemoryTrackerBlockerInThread.h>
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
#include <Columns/ColumnSparse.h> #include <Columns/ColumnSparse.h>
namespace DB namespace DB
@ -11,7 +12,6 @@ namespace ErrorCodes
extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int NO_SUCH_COLUMN_IN_TABLE;
} }
Block getIndexBlockAndPermute(const Block & block, const Names & names, const IColumn::Permutation * permutation) Block getIndexBlockAndPermute(const Block & block, const Names & names, const IColumn::Permutation * permutation)
{ {
Block result; Block result;
@ -57,7 +57,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter(
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const MergeTreeWriterSettings & settings_, const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_) MergeTreeIndexGranularityPtr index_granularity_)
: data_part_name(data_part_name_) : data_part_name(data_part_name_)
, serializations(serializations_) , serializations(serializations_)
, index_granularity_info(index_granularity_info_) , index_granularity_info(index_granularity_info_)
@ -68,7 +68,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter(
, settings(settings_) , settings(settings_)
, with_final_mark(settings.can_use_adaptive_granularity) , with_final_mark(settings.can_use_adaptive_granularity)
, data_part_storage(data_part_storage_) , data_part_storage(data_part_storage_)
, index_granularity(index_granularity_) , index_granularity(std::move(index_granularity_))
{ {
} }
@ -145,7 +145,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter(
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity); MergeTreeIndexGranularityPtr computed_index_granularity);
MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter(
const String & data_part_name_, const String & data_part_name_,
@ -162,8 +162,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter(
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity); MergeTreeIndexGranularityPtr computed_index_granularity);
MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter(
MergeTreeDataPartType part_type, MergeTreeDataPartType part_type,
@ -182,12 +181,26 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter(
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity) MergeTreeIndexGranularityPtr computed_index_granularity)
{ {
if (part_type == MergeTreeDataPartType::Compact) if (part_type == MergeTreeDataPartType::Compact)
return createMergeTreeDataPartCompactWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, return createMergeTreeDataPartCompactWriter(
index_granularity_info_, storage_settings_, columns_list, column_positions, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, data_part_name_,
marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); logger_name_,
serializations_,
data_part_storage_,
index_granularity_info_,
storage_settings_,
columns_list,
column_positions,
metadata_snapshot,
virtual_columns,
indices_to_recalc,
stats_to_recalc_,
marks_file_extension_,
default_codec_,
writer_settings,
std::move(computed_index_granularity));
if (part_type == MergeTreeDataPartType::Wide) if (part_type == MergeTreeDataPartType::Wide)
return createMergeTreeDataPartWideWriter( return createMergeTreeDataPartWideWriter(
data_part_name_, data_part_name_,
@ -204,7 +217,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter(
marks_file_extension_, marks_file_extension_,
default_codec_, default_codec_,
writer_settings, writer_settings,
computed_index_granularity); std::move(computed_index_granularity));
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown part type: {}", part_type.toString()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown part type: {}", part_type.toString());
} }

View File

@ -36,7 +36,7 @@ public:
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const VirtualsDescriptionPtr & virtual_columns_, const VirtualsDescriptionPtr & virtual_columns_,
const MergeTreeWriterSettings & settings_, const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_ = {}); MergeTreeIndexGranularityPtr index_granularity_);
virtual ~IMergeTreeDataPartWriter(); virtual ~IMergeTreeDataPartWriter();
@ -52,7 +52,7 @@ public:
PlainMarksByName releaseCachedMarks(); PlainMarksByName releaseCachedMarks();
const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; } MergeTreeIndexGranularityPtr getIndexGranularity() const { return index_granularity; }
virtual Block getColumnsSample() const = 0; virtual Block getColumnsSample() const = 0;
@ -76,7 +76,7 @@ protected:
MutableDataPartStoragePtr data_part_storage; MutableDataPartStoragePtr data_part_storage;
MutableColumns index_columns; MutableColumns index_columns;
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityPtr index_granularity;
/// Marks that will be saved to cache on finish. /// Marks that will be saved to cache on finish.
PlainMarksByName cached_marks; PlainMarksByName cached_marks;
}; };
@ -101,6 +101,6 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter(
const String & marks_file_extension, const String & marks_file_extension,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity); MergeTreeIndexGranularityPtr computed_index_granularity);
} }

View File

@ -29,7 +29,7 @@ public:
virtual void write(const Block & block) = 0; virtual void write(const Block & block) = 0;
const MergeTreeIndexGranularity & getIndexGranularity() const MergeTreeIndexGranularityPtr getIndexGranularity() const
{ {
return writer->getIndexGranularity(); return writer->getIndexGranularity();
} }

View File

@ -51,7 +51,7 @@ public:
const MergeTreeIndexGranularityInfo & getIndexGranularityInfo() const override { return data_part->index_granularity_info; } const MergeTreeIndexGranularityInfo & getIndexGranularityInfo() const override { return data_part->index_granularity_info; }
const MergeTreeIndexGranularity & getIndexGranularity() const override { return data_part->index_granularity; } const MergeTreeIndexGranularity & getIndexGranularity() const override { return *data_part->index_granularity; }
const SerializationInfoByName & getSerializationInfos() const override { return data_part->getSerializationInfos(); } const SerializationInfoByName & getSerializationInfos() const override { return data_part->getSerializationInfos(); }

View File

@ -52,7 +52,7 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta
total_size_bytes_compressed += source_part->getBytesOnDisk(); total_size_bytes_compressed += source_part->getBytesOnDisk();
total_size_bytes_uncompressed += source_part->getTotalColumnsSize().data_uncompressed; total_size_bytes_uncompressed += source_part->getTotalColumnsSize().data_uncompressed;
total_size_marks += source_part->getMarksCount(); total_size_marks += source_part->getMarksCount();
total_rows_count += source_part->index_granularity.getTotalRows(); total_rows_count += source_part->index_granularity->getTotalRows();
} }
if (!future_part->parts.empty()) if (!future_part->parts.empty())

View File

@ -8,6 +8,7 @@
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Core/Settings.h> #include <Core/Settings.h>
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
#include <Compression/CompressedWriteBuffer.h> #include <Compression/CompressedWriteBuffer.h>
#include <DataTypes/ObjectUtils.h> #include <DataTypes/ObjectUtils.h>
#include <DataTypes/Serializations/SerializationInfo.h> #include <DataTypes/Serializations/SerializationInfo.h>
@ -72,6 +73,7 @@ namespace CurrentMetrics
namespace DB namespace DB
{ {
namespace Setting namespace Setting
{ {
extern const SettingsBool compile_sort_description; extern const SettingsBool compile_sort_description;
@ -99,6 +101,7 @@ namespace MergeTreeSetting
extern const MergeTreeSettingsUInt64 vertical_merge_algorithm_min_rows_to_activate; extern const MergeTreeSettingsUInt64 vertical_merge_algorithm_min_rows_to_activate;
extern const MergeTreeSettingsBool vertical_merge_remote_filesystem_prefetch; extern const MergeTreeSettingsBool vertical_merge_remote_filesystem_prefetch;
extern const MergeTreeSettingsBool prewarm_mark_cache; extern const MergeTreeSettingsBool prewarm_mark_cache;
extern const MergeTreeSettingsBool use_const_adaptive_granularity;
} }
namespace ErrorCodes namespace ErrorCodes
@ -412,10 +415,11 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
}; };
auto mutations_snapshot = global_ctx->data->getMutationsSnapshot(params); auto mutations_snapshot = global_ctx->data->getMutationsSnapshot(params);
auto storage_settings = global_ctx->data->getSettings();
SerializationInfo::Settings info_settings = SerializationInfo::Settings info_settings =
{ {
.ratio_of_defaults_for_sparse = (*global_ctx->data->getSettings())[MergeTreeSetting::ratio_of_defaults_for_sparse_serialization], .ratio_of_defaults_for_sparse = (*storage_settings)[MergeTreeSetting::ratio_of_defaults_for_sparse_serialization],
.choose_kind = true, .choose_kind = true,
}; };
@ -464,6 +468,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
ctx->sum_input_rows_upper_bound = global_ctx->merge_list_element_ptr->total_rows_count; ctx->sum_input_rows_upper_bound = global_ctx->merge_list_element_ptr->total_rows_count;
ctx->sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed; ctx->sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed;
ctx->sum_uncompressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_uncompressed;
global_ctx->chosen_merge_algorithm = chooseMergeAlgorithm(); global_ctx->chosen_merge_algorithm = chooseMergeAlgorithm();
global_ctx->merge_list_element_ptr->merge_algorithm.store(global_ctx->chosen_merge_algorithm, std::memory_order_relaxed); global_ctx->merge_list_element_ptr->merge_algorithm.store(global_ctx->chosen_merge_algorithm, std::memory_order_relaxed);
@ -507,8 +512,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge algorithm must be chosen"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge algorithm must be chosen");
} }
/// If merge is vertical we cannot calculate it bool use_adaptive_granularity = global_ctx->new_data_part->index_granularity_info.mark_type.adaptive;
ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical); bool use_const_adaptive_granularity = (*storage_settings)[MergeTreeSetting::use_const_adaptive_granularity];
/// If merge is vertical we cannot calculate it.
/// If granularity is constant we don't need to calculate it.
ctx->blocks_are_granules_size = use_adaptive_granularity
&& !use_const_adaptive_granularity
&& global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical;
/// Merged stream will be created and available as merged_stream variable /// Merged stream will be created and available as merged_stream variable
createMergedStream(); createMergedStream();
@ -550,7 +561,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
} }
} }
bool save_marks_in_cache = (*global_ctx->data->getSettings())[MergeTreeSetting::prewarm_mark_cache] && global_ctx->context->getMarkCache(); auto index_granularity_ptr = createMergeTreeIndexGranularity(
ctx->sum_input_rows_upper_bound,
ctx->sum_uncompressed_bytes_upper_bound,
*storage_settings,
global_ctx->new_data_part->index_granularity_info,
ctx->blocks_are_granules_size);
bool save_marks_in_cache = (*storage_settings)[MergeTreeSetting::prewarm_mark_cache] && global_ctx->context->getMarkCache();
global_ctx->to = std::make_shared<MergedBlockOutputStream>( global_ctx->to = std::make_shared<MergedBlockOutputStream>(
global_ctx->new_data_part, global_ctx->new_data_part,
@ -559,6 +577,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
MergeTreeIndexFactory::instance().getMany(global_ctx->merging_skip_indexes), MergeTreeIndexFactory::instance().getMany(global_ctx->merging_skip_indexes),
getStatisticsForColumns(global_ctx->merging_columns, global_ctx->metadata_snapshot), getStatisticsForColumns(global_ctx->merging_columns, global_ctx->metadata_snapshot),
ctx->compression_codec, ctx->compression_codec,
std::move(index_granularity_ptr),
global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID, global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID,
/*reset_columns=*/ true, /*reset_columns=*/ true,
save_marks_in_cache, save_marks_in_cache,
@ -1100,12 +1119,12 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
global_ctx->new_data_part, global_ctx->new_data_part,
global_ctx->metadata_snapshot, global_ctx->metadata_snapshot,
columns_list, columns_list,
ctx->compression_codec,
column_pipepline.indexes_to_recalc, column_pipepline.indexes_to_recalc,
getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot),
ctx->compression_codec,
global_ctx->to->getIndexGranularity(),
&global_ctx->written_offset_columns, &global_ctx->written_offset_columns,
save_marks_in_cache, save_marks_in_cache);
global_ctx->to->getIndexGranularity());
ctx->column_elems_written = 0; ctx->column_elems_written = 0;
} }

View File

@ -243,7 +243,6 @@ private:
bool need_remove_expired_values{false}; bool need_remove_expired_values{false};
bool force_ttl{false}; bool force_ttl{false};
CompressionCodecPtr compression_codec{nullptr}; CompressionCodecPtr compression_codec{nullptr};
size_t sum_input_rows_upper_bound{0};
std::shared_ptr<RowsSourcesTemporaryFile> rows_sources_temporary_file; std::shared_ptr<RowsSourcesTemporaryFile> rows_sources_temporary_file;
std::optional<ColumnSizeEstimator> column_sizes{}; std::optional<ColumnSizeEstimator> column_sizes{};
@ -261,7 +260,9 @@ private:
std::function<bool()> is_cancelled{}; std::function<bool()> is_cancelled{};
/// Local variables for this stage /// Local variables for this stage
size_t sum_input_rows_upper_bound{0};
size_t sum_compressed_bytes_upper_bound{0}; size_t sum_compressed_bytes_upper_bound{0};
size_t sum_uncompressed_bytes_upper_bound{0};
bool blocks_are_granules_size{false}; bool blocks_are_granules_size{false};
LoggerPtr log{getLogger("MergeTask::PrepareStage")}; LoggerPtr log{getLogger("MergeTask::PrepareStage")};

View File

@ -83,6 +83,7 @@
#include <Storages/StorageMergeTree.h> #include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h> #include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/VirtualColumnUtils.h> #include <Storages/VirtualColumnUtils.h>
#include <Storages/MergeTree/MergeTreeIndexGranularityAdaptive.h>
#include <boost/range/algorithm_ext/erase.hpp> #include <boost/range/algorithm_ext/erase.hpp>
#include <boost/algorithm/string/join.hpp> #include <boost/algorithm/string/join.hpp>
@ -7237,7 +7238,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
/// It's extremely rare that some parts have final marks while others don't. To make it /// It's extremely rare that some parts have final marks while others don't. To make it
/// straightforward, disable minmax_count projection when `max(pk)' encounters any part with /// straightforward, disable minmax_count projection when `max(pk)' encounters any part with
/// no final mark. /// no final mark.
if (need_primary_key_max_column && !part->index_granularity.hasFinalMark()) if (need_primary_key_max_column && !part->index_granularity->hasFinalMark())
return {}; return {};
real_parts.push_back(part); real_parts.push_back(part);
@ -8960,10 +8961,15 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::createE
auto compression_codec = getContext()->chooseCompressionCodec(0, 0); auto compression_codec = getContext()->chooseCompressionCodec(0, 0);
const auto & index_factory = MergeTreeIndexFactory::instance(); const auto & index_factory = MergeTreeIndexFactory::instance();
MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, MergedBlockOutputStream out(
new_data_part,
metadata_snapshot,
columns,
index_factory.getMany(metadata_snapshot->getSecondaryIndices()), index_factory.getMany(metadata_snapshot->getSecondaryIndices()),
ColumnsStatistics{}, ColumnsStatistics{},
compression_codec, txn ? txn->tid : Tx::PrehistoricTID); compression_codec,
std::make_shared<MergeTreeIndexGranularityAdaptive>(),
txn ? txn->tid : Tx::PrehistoricTID);
bool sync_on_insert = (*settings)[MergeTreeSetting::fsync_after_insert]; bool sync_on_insert = (*settings)[MergeTreeSetting::fsync_after_insert];

View File

@ -3,6 +3,7 @@
#include <Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.h> #include <Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h> #include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h> #include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
namespace DB namespace DB
@ -15,6 +16,11 @@ namespace ErrorCodes
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART; extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
} }
namespace MergeTreeSetting
{
extern MergeTreeSettingsBool enable_index_granularity_compression;
}
MergeTreeDataPartCompact::MergeTreeDataPartCompact( MergeTreeDataPartCompact::MergeTreeDataPartCompact(
const MergeTreeData & storage_, const MergeTreeData & storage_,
const String & name_, const String & name_,
@ -62,7 +68,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter(
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity) MergeTreeIndexGranularityPtr computed_index_granularity)
{ {
NamesAndTypesList ordered_columns_list; NamesAndTypesList ordered_columns_list;
std::copy_if(columns_list.begin(), columns_list.end(), std::back_inserter(ordered_columns_list), std::copy_if(columns_list.begin(), columns_list.end(), std::back_inserter(ordered_columns_list),
@ -76,7 +82,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter(
data_part_name_, logger_name_, serializations_, data_part_storage_, data_part_name_, logger_name_, serializations_, data_part_storage_,
index_granularity_info_, storage_settings_, ordered_columns_list, metadata_snapshot, virtual_columns, index_granularity_info_, storage_settings_, ordered_columns_list, metadata_snapshot, virtual_columns,
indices_to_recalc, stats_to_recalc_, marks_file_extension_, indices_to_recalc, stats_to_recalc_, marks_file_extension_,
default_codec_, writer_settings, computed_index_granularity); default_codec_, writer_settings, std::move(computed_index_granularity));
} }
@ -95,8 +101,11 @@ void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*eac
} }
void MergeTreeDataPartCompact::loadIndexGranularityImpl( void MergeTreeDataPartCompact::loadIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_, MergeTreeIndexGranularityPtr & index_granularity_ptr,
size_t columns_count, const IDataPartStorage & data_part_storage_) const MergeTreeIndexGranularityInfo & index_granularity_info_,
size_t columns_count,
const IDataPartStorage & data_part_storage_,
const MergeTreeSettings & storage_settings)
{ {
if (!index_granularity_info_.mark_type.adaptive) if (!index_granularity_info_.mark_type.adaptive)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeDataPartCompact cannot be created with non-adaptive granularity."); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeDataPartCompact cannot be created with non-adaptive granularity.");
@ -122,10 +131,14 @@ void MergeTreeDataPartCompact::loadIndexGranularityImpl(
marks_reader->ignore(columns_count * sizeof(MarkInCompressedFile)); marks_reader->ignore(columns_count * sizeof(MarkInCompressedFile));
size_t granularity; size_t granularity;
readBinaryLittleEndian(granularity, *marks_reader); readBinaryLittleEndian(granularity, *marks_reader);
index_granularity_.appendMark(granularity); index_granularity_ptr->appendMark(granularity);
} }
index_granularity_.setInitialized(); if (storage_settings[MergeTreeSetting::enable_index_granularity_compression])
{
if (auto new_granularity_ptr = index_granularity_ptr->optimize())
index_granularity_ptr = std::move(new_granularity_ptr);
}
} }
void MergeTreeDataPartCompact::loadIndexGranularity() void MergeTreeDataPartCompact::loadIndexGranularity()
@ -133,7 +146,7 @@ void MergeTreeDataPartCompact::loadIndexGranularity()
if (columns.empty()) if (columns.empty())
throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No columns in part {}", name); throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No columns in part {}", name);
loadIndexGranularityImpl(index_granularity, index_granularity_info, columns.size(), getDataPartStorage()); loadIndexGranularityImpl(index_granularity, index_granularity_info, columns.size(), getDataPartStorage(), *storage.getSettings());
} }
void MergeTreeDataPartCompact::loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const void MergeTreeDataPartCompact::loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const
@ -152,7 +165,7 @@ void MergeTreeDataPartCompact::loadMarksToCache(const Names & column_names, Mark
info_for_read, info_for_read,
mark_cache, mark_cache,
index_granularity_info.getMarksFilePath(DATA_FILE_NAME), index_granularity_info.getMarksFilePath(DATA_FILE_NAME),
index_granularity.getMarksCount(), index_granularity->getMarksCount(),
index_granularity_info, index_granularity_info,
/*save_marks_in_cache=*/ true, /*save_marks_in_cache=*/ true,
read_settings, read_settings,
@ -227,7 +240,7 @@ void MergeTreeDataPartCompact::doCheckConsistency(bool require_part_metadata) co
getDataPartStorage().getRelativePath(), getDataPartStorage().getRelativePath(),
std::string(fs::path(getDataPartStorage().getFullPath()) / mrk_file_name)); std::string(fs::path(getDataPartStorage().getFullPath()) / mrk_file_name));
UInt64 expected_file_size = index_granularity_info.getMarkSizeInBytes(columns.size()) * index_granularity.getMarksCount(); UInt64 expected_file_size = index_granularity_info.getMarkSizeInBytes(columns.size()) * index_granularity->getMarksCount();
if (expected_file_size != file_size) if (expected_file_size != file_size)
throw Exception( throw Exception(
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART,

View File

@ -60,8 +60,11 @@ public:
protected: protected:
static void loadIndexGranularityImpl( static void loadIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_, MergeTreeIndexGranularityPtr & index_granularity_,
size_t columns_count, const IDataPartStorage & data_part_storage_); const MergeTreeIndexGranularityInfo & index_granularity_info_,
size_t columns_count,
const IDataPartStorage & data_part_storage_,
const MergeTreeSettings & storage_settings);
void doCheckConsistency(bool require_part_metadata) const override; void doCheckConsistency(bool require_part_metadata) const override;

View File

@ -3,6 +3,8 @@
#include <Storages/MergeTree/MergeTreeDataPartWriterWide.h> #include <Storages/MergeTree/MergeTreeDataPartWriterWide.h>
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h> #include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h> #include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
#include <Storages/MergeTree/MergeTreeIndexGranularityConstant.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
@ -17,6 +19,11 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
namespace MergeTreeSetting
{
extern MergeTreeSettingsBool enable_index_granularity_compression;
}
MergeTreeDataPartWide::MergeTreeDataPartWide( MergeTreeDataPartWide::MergeTreeDataPartWide(
const MergeTreeData & storage_, const MergeTreeData & storage_,
const String & name_, const String & name_,
@ -68,14 +75,14 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter(
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings, const MergeTreeWriterSettings & writer_settings,
const MergeTreeIndexGranularity & computed_index_granularity) MergeTreeIndexGranularityPtr computed_index_granularity)
{ {
return std::make_unique<MergeTreeDataPartWriterWide>( return std::make_unique<MergeTreeDataPartWriterWide>(
data_part_name_, logger_name_, serializations_, data_part_storage_, data_part_name_, logger_name_, serializations_, data_part_storage_,
index_granularity_info_, storage_settings_, columns_list, index_granularity_info_, storage_settings_, columns_list,
metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_,
marks_file_extension_, marks_file_extension_,
default_codec_, writer_settings, computed_index_granularity); default_codec_, writer_settings, std::move(computed_index_granularity));
} }
@ -114,8 +121,11 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl(
} }
void MergeTreeDataPartWide::loadIndexGranularityImpl( void MergeTreeDataPartWide::loadIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, MergeTreeIndexGranularityPtr & index_granularity_ptr,
const IDataPartStorage & data_part_storage_, const std::string & any_column_file_name) MergeTreeIndexGranularityInfo & index_granularity_info_,
const IDataPartStorage & data_part_storage_,
const std::string & any_column_file_name,
const MergeTreeSettings & storage_settings)
{ {
index_granularity_info_.changeGranularityIfRequired(data_part_storage_); index_granularity_info_.changeGranularityIfRequired(data_part_storage_);
@ -127,12 +137,13 @@ void MergeTreeDataPartWide::loadIndexGranularityImpl(
std::string(fs::path(data_part_storage_.getFullPath()) / marks_file_path)); std::string(fs::path(data_part_storage_.getFullPath()) / marks_file_path));
size_t marks_file_size = data_part_storage_.getFileSize(marks_file_path); size_t marks_file_size = data_part_storage_.getFileSize(marks_file_path);
size_t fixed_granularity = index_granularity_info_.fixed_index_granularity;
if (!index_granularity_info_.mark_type.adaptive && !index_granularity_info_.mark_type.compressed) if (!index_granularity_info_.mark_type.adaptive && !index_granularity_info_.mark_type.compressed)
{ {
/// The most easy way - no need to read the file, everything is known from its size. /// The most easy way - no need to read the file, everything is known from its size.
size_t marks_count = marks_file_size / index_granularity_info_.getMarkSizeInBytes(); size_t marks_count = marks_file_size / index_granularity_info_.getMarkSizeInBytes();
index_granularity_.resizeWithFixedGranularity(marks_count, index_granularity_info_.fixed_index_granularity); /// all the same index_granularity_ptr = std::make_shared<MergeTreeIndexGranularityConstant>(fixed_granularity, fixed_granularity, marks_count, false);
} }
else else
{ {
@ -145,6 +156,7 @@ void MergeTreeDataPartWide::loadIndexGranularityImpl(
marks_reader = std::make_unique<CompressedReadBufferFromFile>(std::move(marks_file)); marks_reader = std::make_unique<CompressedReadBufferFromFile>(std::move(marks_file));
size_t marks_count = 0; size_t marks_count = 0;
while (!marks_reader->eof()) while (!marks_reader->eof())
{ {
MarkInCompressedFile mark; MarkInCompressedFile mark;
@ -157,15 +169,20 @@ void MergeTreeDataPartWide::loadIndexGranularityImpl(
if (index_granularity_info_.mark_type.adaptive) if (index_granularity_info_.mark_type.adaptive)
{ {
readBinaryLittleEndian(granularity, *marks_reader); readBinaryLittleEndian(granularity, *marks_reader);
index_granularity_.appendMark(granularity); index_granularity_ptr->appendMark(granularity);
} }
} }
if (!index_granularity_info_.mark_type.adaptive) if (!index_granularity_info_.mark_type.adaptive)
index_granularity_.resizeWithFixedGranularity(marks_count, index_granularity_info_.fixed_index_granularity); /// all the same {
index_granularity_ptr = std::make_shared<MergeTreeIndexGranularityConstant>(fixed_granularity, fixed_granularity, marks_count, false);
}
else if (storage_settings[MergeTreeSetting::enable_index_granularity_compression])
{
if (auto new_granularity_ptr = index_granularity_ptr->optimize())
index_granularity_ptr = std::move(new_granularity_ptr);
}
} }
index_granularity_.setInitialized();
} }
void MergeTreeDataPartWide::loadIndexGranularity() void MergeTreeDataPartWide::loadIndexGranularity()
@ -179,7 +196,7 @@ void MergeTreeDataPartWide::loadIndexGranularity()
"There are no files for column {} in part {}", "There are no files for column {} in part {}",
columns.front().name, getDataPartStorage().getFullPath()); columns.front().name, getDataPartStorage().getFullPath());
loadIndexGranularityImpl(index_granularity, index_granularity_info, getDataPartStorage(), *any_column_filename); loadIndexGranularityImpl(index_granularity, index_granularity_info, getDataPartStorage(), *any_column_filename, *storage.getSettings());
} }
void MergeTreeDataPartWide::loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const void MergeTreeDataPartWide::loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const
@ -209,7 +226,7 @@ void MergeTreeDataPartWide::loadMarksToCache(const Names & column_names, MarkCac
info_for_read, info_for_read,
mark_cache, mark_cache,
index_granularity_info.getMarksFilePath(*stream_name), index_granularity_info.getMarksFilePath(*stream_name),
index_granularity.getMarksCount(), index_granularity->getMarksCount(),
index_granularity_info, index_granularity_info,
/*save_marks_in_cache=*/ true, /*save_marks_in_cache=*/ true,
read_settings, read_settings,

View File

@ -55,8 +55,11 @@ public:
protected: protected:
static void loadIndexGranularityImpl( static void loadIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, MergeTreeIndexGranularityPtr & index_granularity_ptr,
const IDataPartStorage & data_part_storage_, const std::string & any_column_file_name); MergeTreeIndexGranularityInfo & index_granularity_info_,
const IDataPartStorage & data_part_storage_,
const std::string & any_column_file_name,
const MergeTreeSettings & storage_settings);
void doCheckConsistency(bool require_part_metadata) const override; void doCheckConsistency(bool require_part_metadata) const override;

View File

@ -25,13 +25,13 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact(
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_, const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_) MergeTreeIndexGranularityPtr index_granularity_)
: MergeTreeDataPartWriterOnDisk( : MergeTreeDataPartWriterOnDisk(
data_part_name_, logger_name_, serializations_, data_part_name_, logger_name_, serializations_,
data_part_storage_, index_granularity_info_, storage_settings_, data_part_storage_, index_granularity_info_, storage_settings_,
columns_list_, metadata_snapshot_, virtual_columns_, columns_list_, metadata_snapshot_, virtual_columns_,
indices_to_recalc_, stats_to_recalc, marks_file_extension_, indices_to_recalc_, stats_to_recalc, marks_file_extension_,
default_codec_, settings_, index_granularity_) default_codec_, settings_, std::move(index_granularity_))
, plain_file(getDataPartStorage().writeFile( , plain_file(getDataPartStorage().writeFile(
MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION,
settings.max_compress_block_size, settings.max_compress_block_size,
@ -189,13 +189,13 @@ void MergeTreeDataPartWriterCompact::write(const Block & block, const IColumn::P
header = result_block.cloneEmpty(); header = result_block.cloneEmpty();
columns_buffer.add(result_block.mutateColumns()); columns_buffer.add(result_block.mutateColumns());
size_t current_mark_rows = index_granularity.getMarkRows(getCurrentMark()); size_t current_mark_rows = index_granularity->getMarkRows(getCurrentMark());
size_t rows_in_buffer = columns_buffer.size(); size_t rows_in_buffer = columns_buffer.size();
if (rows_in_buffer >= current_mark_rows) if (rows_in_buffer >= current_mark_rows)
{ {
Block flushed_block = header.cloneWithColumns(columns_buffer.releaseColumns()); Block flushed_block = header.cloneWithColumns(columns_buffer.releaseColumns());
auto granules_to_write = getGranulesToWrite(index_granularity, flushed_block.rows(), getCurrentMark(), /* last_block = */ false); auto granules_to_write = getGranulesToWrite(*index_granularity, flushed_block.rows(), getCurrentMark(), /* last_block = */ false);
writeDataBlockPrimaryIndexAndSkipIndices(flushed_block, granules_to_write); writeDataBlockPrimaryIndexAndSkipIndices(flushed_block, granules_to_write);
setCurrentMark(getCurrentMark() + granules_to_write.size()); setCurrentMark(getCurrentMark() + granules_to_write.size());
calculateAndSerializeStatistics(flushed_block); calculateAndSerializeStatistics(flushed_block);
@ -274,12 +274,11 @@ void MergeTreeDataPartWriterCompact::fillDataChecksums(MergeTreeDataPartChecksum
if (columns_buffer.size() != 0) if (columns_buffer.size() != 0)
{ {
auto block = header.cloneWithColumns(columns_buffer.releaseColumns()); auto block = header.cloneWithColumns(columns_buffer.releaseColumns());
auto granules_to_write = getGranulesToWrite(index_granularity, block.rows(), getCurrentMark(), /* last_block = */ true); auto granules_to_write = getGranulesToWrite(*index_granularity, block.rows(), getCurrentMark(), /*last_block=*/ true);
if (!granules_to_write.back().is_complete) if (!granules_to_write.back().is_complete)
{ {
/// Correct last mark as it should contain exact amount of rows. /// Correct last mark as it should contain exact amount of rows.
index_granularity.popMark(); index_granularity->adjustLastMark(granules_to_write.back().rows_to_write);
index_granularity.appendMark(granules_to_write.back().rows_to_write);
} }
writeDataBlockPrimaryIndexAndSkipIndices(block, granules_to_write); writeDataBlockPrimaryIndexAndSkipIndices(block, granules_to_write);
} }
@ -375,11 +374,11 @@ static void fillIndexGranularityImpl(
void MergeTreeDataPartWriterCompact::fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) void MergeTreeDataPartWriterCompact::fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block)
{ {
size_t index_offset = 0; size_t index_offset = 0;
if (index_granularity.getMarksCount() > getCurrentMark()) if (index_granularity->getMarksCount() > getCurrentMark())
index_offset = index_granularity.getMarkRows(getCurrentMark()) - columns_buffer.size(); index_offset = index_granularity->getMarkRows(getCurrentMark()) - columns_buffer.size();
fillIndexGranularityImpl( fillIndexGranularityImpl(
index_granularity, *index_granularity,
index_offset, index_offset,
index_granularity_for_block, index_granularity_for_block,
rows_in_block); rows_in_block);

View File

@ -25,7 +25,7 @@ public:
const String & marks_file_extension, const String & marks_file_extension,
const CompressionCodecPtr & default_codec, const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings, const MergeTreeWriterSettings & settings,
const MergeTreeIndexGranularity & index_granularity); MergeTreeIndexGranularityPtr index_granularity_);
void write(const Block & block, const IColumn::Permutation * permutation) override; void write(const Block & block, const IColumn::Permutation * permutation) override;

View File

@ -162,20 +162,20 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk(
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_, const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_) MergeTreeIndexGranularityPtr index_granularity_)
: IMergeTreeDataPartWriter( : IMergeTreeDataPartWriter(
data_part_name_, serializations_, data_part_storage_, index_granularity_info_, data_part_name_, serializations_, data_part_storage_, index_granularity_info_,
storage_settings_, columns_list_, metadata_snapshot_, virtual_columns_, settings_, index_granularity_) storage_settings_, columns_list_, metadata_snapshot_, virtual_columns_, settings_, std::move(index_granularity_))
, skip_indices(indices_to_recalc_) , skip_indices(indices_to_recalc_)
, stats(stats_to_recalc_) , stats(stats_to_recalc_)
, marks_file_extension(marks_file_extension_) , marks_file_extension(marks_file_extension_)
, default_codec(default_codec_) , default_codec(default_codec_)
, compute_granularity(index_granularity.empty()) , compute_granularity(index_granularity->empty())
, compress_primary_key(settings.compress_primary_key) , compress_primary_key(settings.compress_primary_key)
, execution_stats(skip_indices.size(), stats.size()) , execution_stats(skip_indices.size(), stats.size())
, log(getLogger(logger_name_ + " (DataPartWriter)")) , log(getLogger(logger_name_ + " (DataPartWriter)"))
{ {
if (settings.blocks_are_granules_size && !index_granularity.empty()) if (settings.blocks_are_granules_size && !index_granularity->empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Can't take information about index granularity from blocks, when non empty index_granularity array specified"); "Can't take information about index granularity from blocks, when non empty index_granularity array specified");
@ -189,63 +189,15 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk(
initStatistics(); initStatistics();
} }
// Implementation is split into static functions for ability
/// of making unit tests without creation instance of IMergeTreeDataPartWriter,
/// which requires a lot of dependencies and access to filesystem.
static size_t computeIndexGranularityImpl(
const Block & block,
size_t index_granularity_bytes,
size_t fixed_index_granularity_rows,
bool blocks_are_granules,
bool can_use_adaptive_index_granularity)
{
size_t rows_in_block = block.rows();
size_t index_granularity_for_block;
if (!can_use_adaptive_index_granularity)
{
index_granularity_for_block = fixed_index_granularity_rows;
}
else
{
size_t block_size_in_memory = block.bytes();
if (blocks_are_granules)
{
index_granularity_for_block = rows_in_block;
}
else if (block_size_in_memory >= index_granularity_bytes)
{
size_t granules_in_block = block_size_in_memory / index_granularity_bytes;
index_granularity_for_block = rows_in_block / granules_in_block;
}
else
{
size_t size_of_row_in_bytes = std::max(block_size_in_memory / rows_in_block, 1UL);
index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes;
}
}
/// We should be less or equal than fixed index granularity.
/// But if block size is a granule size then do not adjust it.
/// Granularity greater than fixed granularity might come from compact part.
if (!blocks_are_granules)
index_granularity_for_block = std::min(fixed_index_granularity_rows, index_granularity_for_block);
/// Very rare case when index granularity bytes less than single row.
if (index_granularity_for_block == 0)
index_granularity_for_block = 1;
return index_granularity_for_block;
}
size_t MergeTreeDataPartWriterOnDisk::computeIndexGranularity(const Block & block) const size_t MergeTreeDataPartWriterOnDisk::computeIndexGranularity(const Block & block) const
{ {
return computeIndexGranularityImpl( return DB::computeIndexGranularity(
block, block.rows(),
(*storage_settings)[MergeTreeSetting::index_granularity_bytes], block.bytes(),
(*storage_settings)[MergeTreeSetting::index_granularity], (*storage_settings)[MergeTreeSetting::index_granularity_bytes],
settings.blocks_are_granules_size, (*storage_settings)[MergeTreeSetting::index_granularity],
settings.can_use_adaptive_granularity); settings.blocks_are_granules_size,
settings.can_use_adaptive_granularity);
} }
void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() void MergeTreeDataPartWriterOnDisk::initPrimaryIndex()
@ -433,7 +385,7 @@ void MergeTreeDataPartWriterOnDisk::fillPrimaryIndexChecksums(MergeTreeData::Dat
{ {
bool write_final_mark = (with_final_mark && data_written); bool write_final_mark = (with_final_mark && data_written);
if (write_final_mark && compute_granularity) if (write_final_mark && compute_granularity)
index_granularity.appendMark(0); index_granularity->appendMark(0);
if (index_file_hashing_stream) if (index_file_hashing_stream)
{ {

View File

@ -116,7 +116,7 @@ public:
const String & marks_file_extension, const String & marks_file_extension,
const CompressionCodecPtr & default_codec, const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings, const MergeTreeWriterSettings & settings,
const MergeTreeIndexGranularity & index_granularity); MergeTreeIndexGranularityPtr index_granularity_);
void setWrittenOffsetColumns(WrittenOffsetColumns * written_offset_columns_) void setWrittenOffsetColumns(WrittenOffsetColumns * written_offset_columns_)
{ {

View File

@ -99,13 +99,13 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide(
const String & marks_file_extension_, const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_, const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_, const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_) MergeTreeIndexGranularityPtr index_granularity_)
: MergeTreeDataPartWriterOnDisk( : MergeTreeDataPartWriterOnDisk(
data_part_name_, logger_name_, serializations_, data_part_name_, logger_name_, serializations_,
data_part_storage_, index_granularity_info_, storage_settings_, data_part_storage_, index_granularity_info_, storage_settings_,
columns_list_, metadata_snapshot_, virtual_columns_, columns_list_, metadata_snapshot_, virtual_columns_,
indices_to_recalc_, stats_to_recalc_, marks_file_extension_, indices_to_recalc_, stats_to_recalc_, marks_file_extension_,
default_codec_, settings_, index_granularity_) default_codec_, settings_, std::move(index_granularity_))
{ {
if (settings.save_marks_in_cache) if (settings.save_marks_in_cache)
{ {
@ -238,8 +238,8 @@ void MergeTreeDataPartWriterWide::shiftCurrentMark(const Granules & granules_wri
if (settings.can_use_adaptive_granularity && settings.blocks_are_granules_size) if (settings.can_use_adaptive_granularity && settings.blocks_are_granules_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incomplete granules are not allowed while blocks are granules size. " throw Exception(ErrorCodes::LOGICAL_ERROR, "Incomplete granules are not allowed while blocks are granules size. "
"Mark number {} (rows {}), rows written in last mark {}, rows to write in last mark from block {} (from row {}), " "Mark number {} (rows {}), rows written in last mark {}, rows to write in last mark from block {} (from row {}), "
"total marks currently {}", last_granule.mark_number, index_granularity.getMarkRows(last_granule.mark_number), "total marks currently {}", last_granule.mark_number, index_granularity->getMarkRows(last_granule.mark_number),
rows_written_in_last_mark, last_granule.rows_to_write, last_granule.start_row, index_granularity.getMarksCount()); rows_written_in_last_mark, last_granule.rows_to_write, last_granule.start_row, index_granularity->getMarksCount());
/// Shift forward except last granule /// Shift forward except last granule
setCurrentMark(getCurrentMark() + granules_written.size() - 1); setCurrentMark(getCurrentMark() + granules_written.size() - 1);
@ -273,10 +273,15 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm
/// but not in case of vertical part of vertical merge) /// but not in case of vertical part of vertical merge)
if (compute_granularity) if (compute_granularity)
{ {
size_t index_granularity_for_block = computeIndexGranularity(block_to_write); size_t index_granularity_for_block;
if (auto constant_granularity = index_granularity->getConstantGranularity())
index_granularity_for_block = *constant_granularity;
else
index_granularity_for_block = computeIndexGranularity(block_to_write);
if (rows_written_in_last_mark > 0) if (rows_written_in_last_mark > 0)
{ {
size_t rows_left_in_last_mark = index_granularity.getMarkRows(getCurrentMark()) - rows_written_in_last_mark; size_t rows_left_in_last_mark = index_granularity->getMarkRows(getCurrentMark()) - rows_written_in_last_mark;
/// Previous granularity was much bigger than our new block's /// Previous granularity was much bigger than our new block's
/// granularity let's adjust it, because we want add new /// granularity let's adjust it, because we want add new
/// heavy-weight blocks into small old granule. /// heavy-weight blocks into small old granule.
@ -294,7 +299,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm
fillIndexGranularity(index_granularity_for_block, block_to_write.rows()); fillIndexGranularity(index_granularity_for_block, block_to_write.rows());
} }
auto granules_to_write = getGranulesToWrite(index_granularity, block_to_write.rows(), getCurrentMark(), rows_written_in_last_mark); auto granules_to_write = getGranulesToWrite(*index_granularity, block_to_write.rows(), getCurrentMark(), rows_written_in_last_mark);
auto offset_columns = written_offset_columns ? *written_offset_columns : WrittenOffsetColumns{}; auto offset_columns = written_offset_columns ? *written_offset_columns : WrittenOffsetColumns{};
Block primary_key_block; Block primary_key_block;
@ -482,7 +487,7 @@ void MergeTreeDataPartWriterWide::writeColumn(
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"We have to add new mark for column, but already have non written mark. " "We have to add new mark for column, but already have non written mark. "
"Current mark {}, total marks {}, offset {}", "Current mark {}, total marks {}, offset {}",
getCurrentMark(), index_granularity.getMarksCount(), rows_written_in_last_mark); getCurrentMark(), index_granularity->getMarksCount(), rows_written_in_last_mark);
last_non_written_marks[name] = getCurrentMarksForColumn(name_and_type, column.getPtr(), offset_columns); last_non_written_marks[name] = getCurrentMarksForColumn(name_and_type, column.getPtr(), offset_columns);
} }
@ -502,7 +507,7 @@ void MergeTreeDataPartWriterWide::writeColumn(
throw Exception(ErrorCodes::LOGICAL_ERROR, "No mark was saved for incomplete granule for column {}", backQuoteIfNeed(name)); throw Exception(ErrorCodes::LOGICAL_ERROR, "No mark was saved for incomplete granule for column {}", backQuoteIfNeed(name));
for (const auto & mark : marks_it->second) for (const auto & mark : marks_it->second)
flushMarkToFile(mark, index_granularity.getMarkRows(granule.mark_number)); flushMarkToFile(mark, index_granularity->getMarkRows(granule.mark_number));
last_non_written_marks.erase(marks_it); last_non_written_marks.erase(marks_it);
} }
} }
@ -549,10 +554,10 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
for (mark_num = 0; !mrk_in->eof(); ++mark_num) for (mark_num = 0; !mrk_in->eof(); ++mark_num)
{ {
if (mark_num > index_granularity.getMarksCount()) if (mark_num > index_granularity->getMarksCount())
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Incorrect number of marks in memory {}, on disk (at least) {}", "Incorrect number of marks in memory {}, on disk (at least) {}",
index_granularity.getMarksCount(), mark_num + 1); index_granularity->getMarksCount(), mark_num + 1);
readBinaryLittleEndian(offset_in_compressed_file, *mrk_in); readBinaryLittleEndian(offset_in_compressed_file, *mrk_in);
readBinaryLittleEndian(offset_in_decompressed_block, *mrk_in); readBinaryLittleEndian(offset_in_decompressed_block, *mrk_in);
@ -583,10 +588,10 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Still have {} rows in bin stream, last mark #{}" "Still have {} rows in bin stream, last mark #{}"
" index granularity size {}, last rows {}", " index granularity size {}, last rows {}",
column->size(), mark_num, index_granularity.getMarksCount(), index_granularity_rows); column->size(), mark_num, index_granularity->getMarksCount(), index_granularity_rows);
} }
if (index_granularity_rows != index_granularity.getMarkRows(mark_num)) if (index_granularity_rows != index_granularity->getMarkRows(mark_num))
{ {
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
@ -594,8 +599,8 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
" (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}", " (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}",
getDataPartStorage().getFullPath(), getDataPartStorage().getFullPath(),
mark_num, offset_in_compressed_file, offset_in_decompressed_block, mark_num, offset_in_compressed_file, offset_in_decompressed_block,
index_granularity.getMarkRows(mark_num), index_granularity_rows, index_granularity->getMarkRows(mark_num), index_granularity_rows,
index_granularity.getMarksCount()); index_granularity->getMarksCount());
} }
auto column = type->createColumn(); auto column = type->createColumn();
@ -630,7 +635,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for mark #{} (compressed offset {}, decompressed offset {}), " ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for mark #{} (compressed offset {}, decompressed offset {}), "
"actually in bin file {}, in mrk file {}, total marks {}", "actually in bin file {}, in mrk file {}, total marks {}",
mark_num, offset_in_compressed_file, offset_in_decompressed_block, column->size(), mark_num, offset_in_compressed_file, offset_in_decompressed_block, column->size(),
index_granularity.getMarkRows(mark_num), index_granularity.getMarksCount()); index_granularity->getMarkRows(mark_num), index_granularity->getMarksCount());
} }
} }
@ -638,7 +643,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Still have something in marks stream, last mark #{}" "Still have something in marks stream, last mark #{}"
" index granularity size {}, last rows {}", " index granularity size {}, last rows {}",
mark_num, index_granularity.getMarksCount(), index_granularity_rows); mark_num, index_granularity->getMarksCount(), index_granularity_rows);
if (!bin_in.eof()) if (!bin_in.eof())
{ {
auto column = type->createColumn(); auto column = type->createColumn();
@ -648,7 +653,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Still have {} rows in bin stream, last mark #{}" "Still have {} rows in bin stream, last mark #{}"
" index granularity size {}, last rows {}", " index granularity size {}, last rows {}",
column->size(), mark_num, index_granularity.getMarksCount(), index_granularity_rows); column->size(), mark_num, index_granularity->getMarksCount(), index_granularity_rows);
} }
} }
@ -665,8 +670,8 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(MergeTreeDataPartChecksums &
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Incomplete granule is not allowed while blocks are granules size even for last granule. " "Incomplete granule is not allowed while blocks are granules size even for last granule. "
"Mark number {} (rows {}), rows written for last mark {}, total marks {}", "Mark number {} (rows {}), rows written for last mark {}, total marks {}",
getCurrentMark(), index_granularity.getMarkRows(getCurrentMark()), getCurrentMark(), index_granularity->getMarkRows(getCurrentMark()),
rows_written_in_last_mark, index_granularity.getMarksCount()); rows_written_in_last_mark, index_granularity->getMarksCount());
adjustLastMarkIfNeedAndFlushToDisk(rows_written_in_last_mark); adjustLastMarkIfNeedAndFlushToDisk(rows_written_in_last_mark);
} }
@ -785,16 +790,16 @@ static void fillIndexGranularityImpl(
void MergeTreeDataPartWriterWide::fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) void MergeTreeDataPartWriterWide::fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block)
{ {
if (getCurrentMark() < index_granularity.getMarksCount() && getCurrentMark() != index_granularity.getMarksCount() - 1) if (getCurrentMark() < index_granularity->getMarksCount() && getCurrentMark() != index_granularity->getMarksCount() - 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to add marks, while current mark {}, but total marks {}", throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to add marks, while current mark {}, but total marks {}",
getCurrentMark(), index_granularity.getMarksCount()); getCurrentMark(), index_granularity->getMarksCount());
size_t index_offset = 0; size_t index_offset = 0;
if (rows_written_in_last_mark != 0) if (rows_written_in_last_mark != 0)
index_offset = index_granularity.getLastMarkRows() - rows_written_in_last_mark; index_offset = index_granularity->getLastMarkRows() - rows_written_in_last_mark;
fillIndexGranularityImpl( fillIndexGranularityImpl(
index_granularity, *index_granularity,
index_offset, index_offset,
index_granularity_for_block, index_granularity_for_block,
rows_in_block); rows_in_block);
@ -813,27 +818,26 @@ void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_
/// other columns /// other columns
if (compute_granularity && settings.can_use_adaptive_granularity) if (compute_granularity && settings.can_use_adaptive_granularity)
{ {
if (getCurrentMark() != index_granularity.getMarksCount() - 1) if (getCurrentMark() != index_granularity->getMarksCount() - 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Non last mark {} (with {} rows) having rows offset {}, total marks {}", "Non last mark {} (with {} rows) having rows offset {}, total marks {}",
getCurrentMark(), index_granularity.getMarkRows(getCurrentMark()), getCurrentMark(), index_granularity->getMarkRows(getCurrentMark()),
rows_written_in_last_mark, index_granularity.getMarksCount()); rows_written_in_last_mark, index_granularity->getMarksCount());
index_granularity.popMark(); index_granularity->adjustLastMark(new_rows_in_last_mark);
index_granularity.appendMark(new_rows_in_last_mark);
} }
/// Last mark should be filled, otherwise it's a bug /// Last mark should be filled, otherwise it's a bug
if (last_non_written_marks.empty()) if (last_non_written_marks.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "No saved marks for last mark {} having rows offset {}, total marks {}", throw Exception(ErrorCodes::LOGICAL_ERROR, "No saved marks for last mark {} having rows offset {}, total marks {}",
getCurrentMark(), rows_written_in_last_mark, index_granularity.getMarksCount()); getCurrentMark(), rows_written_in_last_mark, index_granularity->getMarksCount());
if (rows_written_in_last_mark == new_rows_in_last_mark) if (rows_written_in_last_mark == new_rows_in_last_mark)
{ {
for (const auto & [name, marks] : last_non_written_marks) for (const auto & [name, marks] : last_non_written_marks)
{ {
for (const auto & mark : marks) for (const auto & mark : marks)
flushMarkToFile(mark, index_granularity.getMarkRows(getCurrentMark())); flushMarkToFile(mark, index_granularity->getMarkRows(getCurrentMark()));
} }
last_non_written_marks.clear(); last_non_written_marks.clear();

View File

@ -35,7 +35,7 @@ public:
const String & marks_file_extension, const String & marks_file_extension,
const CompressionCodecPtr & default_codec, const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings, const MergeTreeWriterSettings & settings,
const MergeTreeIndexGranularity & index_granularity); MergeTreeIndexGranularityPtr index_granularity_);
void write(const Block & block, const IColumn::Permutation * permutation) override; void write(const Block & block, const IColumn::Permutation * permutation) override;

View File

@ -129,7 +129,7 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead(
{ {
MarkRanges part_ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, {}, &exact_ranges, settings, log); MarkRanges part_ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, {}, &exact_ranges, settings, log);
for (const auto & range : part_ranges) for (const auto & range : part_ranges)
rows_count += part->index_granularity.getRowsCountInRange(range); rows_count += part->index_granularity->getRowsCountInRange(range);
} }
UNUSED(exact_ranges); UNUSED(exact_ranges);
@ -688,7 +688,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
auto & part = parts[part_index]; auto & part = parts[part_index];
RangesInDataPart ranges(part, part_index); RangesInDataPart ranges(part, part_index);
size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal(); size_t total_marks_count = part->index_granularity->getMarksCountWithoutFinal();
if (metadata_snapshot->hasPrimaryKey() || part_offset_condition) if (metadata_snapshot->hasPrimaryKey() || part_offset_condition)
{ {
@ -1044,11 +1044,11 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
{ {
MarkRanges res; MarkRanges res;
size_t marks_count = part->index_granularity.getMarksCount(); size_t marks_count = part->index_granularity->getMarksCount();
if (marks_count == 0) if (marks_count == 0)
return res; return res;
bool has_final_mark = part->index_granularity.hasFinalMark(); bool has_final_mark = part->index_granularity->hasFinalMark();
bool key_condition_useful = !key_condition.alwaysUnknownOrTrue(); bool key_condition_useful = !key_condition.alwaysUnknownOrTrue();
bool part_offset_condition_useful = part_offset_condition && !part_offset_condition->alwaysUnknownOrTrue(); bool part_offset_condition_useful = part_offset_condition && !part_offset_condition->alwaysUnknownOrTrue();
@ -1160,16 +1160,16 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
auto check_part_offset_condition = [&]() auto check_part_offset_condition = [&]()
{ {
auto begin = part->index_granularity.getMarkStartingRow(range.begin); auto begin = part->index_granularity->getMarkStartingRow(range.begin);
auto end = part->index_granularity.getMarkStartingRow(range.end) - 1; auto end = part->index_granularity->getMarkStartingRow(range.end) - 1;
if (begin > end) if (begin > end)
{ {
/// Empty mark (final mark) /// Empty mark (final mark)
return BoolMask(false, true); return BoolMask(false, true);
} }
part_offset_left[0] = part->index_granularity.getMarkStartingRow(range.begin); part_offset_left[0] = part->index_granularity->getMarkStartingRow(range.begin);
part_offset_right[0] = part->index_granularity.getMarkStartingRow(range.end) - 1; part_offset_right[0] = part->index_granularity->getMarkStartingRow(range.end) - 1;
part_offset_left[1] = part->name; part_offset_left[1] = part->name;
part_offset_right[1] = part->name; part_offset_right[1] = part->name;
@ -1381,9 +1381,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
part->index_granularity_info.fixed_index_granularity, part->index_granularity_info.fixed_index_granularity,
part->index_granularity_info.index_granularity_bytes); part->index_granularity_info.index_granularity_bytes);
size_t marks_count = part->getMarksCount(); size_t marks_count = part->index_granularity->getMarksCountWithoutFinal();
size_t final_mark = part->index_granularity.hasFinalMark(); size_t index_marks_count = (marks_count + index_granularity - 1) / index_granularity;
size_t index_marks_count = (marks_count - final_mark + index_granularity - 1) / index_granularity;
MarkRanges index_ranges; MarkRanges index_ranges;
for (const auto & range : ranges) for (const auto & range : ranges)
@ -1431,8 +1430,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
for (auto row : rows) for (auto row : rows)
{ {
const MergeTreeIndexGranularity & merge_tree_index_granularity = part->index_granularity; size_t num_marks = part->index_granularity->countMarksForRows(index_mark * index_granularity, row);
size_t num_marks = merge_tree_index_granularity.countMarksForRows(index_mark * index_granularity, row);
MarkRange data_range( MarkRange data_range(
std::max(ranges[i].begin, (index_mark * index_granularity) + num_marks), std::max(ranges[i].begin, (index_mark * index_granularity) + num_marks),
@ -1505,9 +1503,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex(
part->index_granularity_info.fixed_index_granularity, part->index_granularity_info.fixed_index_granularity,
part->index_granularity_info.index_granularity_bytes); part->index_granularity_info.index_granularity_bytes);
size_t marks_count = part->getMarksCount(); size_t marks_count = part->index_granularity->getMarksCountWithoutFinal();
size_t final_mark = part->index_granularity.hasFinalMark(); size_t index_marks_count = (marks_count + index_granularity - 1) / index_granularity;
size_t index_marks_count = (marks_count - final_mark + index_granularity - 1) / index_granularity;
std::vector<std::unique_ptr<MergeTreeIndexReader>> readers; std::vector<std::unique_ptr<MergeTreeIndexReader>> readers;
for (const auto & index_helper : indices) for (const auto & index_helper : indices)
@ -1607,9 +1604,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead(
continue; continue;
} }
size_t num_granules = part->getMarksCount(); size_t num_granules = part->index_granularity->getMarksCountWithoutFinal();
if (num_granules && part->index_granularity.hasFinalMark())
--num_granules;
counters.num_initial_selected_parts += 1; counters.num_initial_selected_parts += 1;
counters.num_initial_selected_granules += num_granules; counters.num_initial_selected_granules += num_granules;
@ -1676,9 +1671,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
if (part->uuid != UUIDHelpers::Nil && ignored_part_uuids->has(part->uuid)) if (part->uuid != UUIDHelpers::Nil && ignored_part_uuids->has(part->uuid))
continue; continue;
size_t num_granules = part->getMarksCount(); size_t num_granules = part->index_granularity->getMarksCountWithoutFinal();
if (num_granules && part->index_granularity.hasFinalMark())
--num_granules;
counters.num_initial_selected_parts += 1; counters.num_initial_selected_parts += 1;
counters.num_initial_selected_granules += num_granules; counters.num_initial_selected_granules += num_granules;

View File

@ -687,6 +687,13 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0);
bool save_marks_in_cache = (*data_settings)[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache(); bool save_marks_in_cache = (*data_settings)[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache();
auto index_granularity_ptr = createMergeTreeIndexGranularity(
block.rows(),
block.bytes(),
*data.getSettings(),
new_data_part->index_granularity_info,
/*blocks_are_granules=*/ false);
auto out = std::make_unique<MergedBlockOutputStream>( auto out = std::make_unique<MergedBlockOutputStream>(
new_data_part, new_data_part,
metadata_snapshot, metadata_snapshot,
@ -694,6 +701,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
indices, indices,
statistics, statistics,
compression_codec, compression_codec,
std::move(index_granularity_ptr),
context->getCurrentTransaction() ? context->getCurrentTransaction()->tid : Tx::PrehistoricTID, context->getCurrentTransaction() ? context->getCurrentTransaction()->tid : Tx::PrehistoricTID,
/*reset_columns=*/ false, /*reset_columns=*/ false,
save_marks_in_cache, save_marks_in_cache,
@ -834,6 +842,13 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0);
bool save_marks_in_cache = (*data.getSettings())[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache(); bool save_marks_in_cache = (*data.getSettings())[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache();
auto index_granularity_ptr = createMergeTreeIndexGranularity(
block.rows(),
block.bytes(),
*data.getSettings(),
new_data_part->index_granularity_info,
/*blocks_are_granules=*/ false);
auto out = std::make_unique<MergedBlockOutputStream>( auto out = std::make_unique<MergedBlockOutputStream>(
new_data_part, new_data_part,
metadata_snapshot, metadata_snapshot,
@ -842,6 +857,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
/// TODO(hanfei): It should be helpful to write statistics for projection result. /// TODO(hanfei): It should be helpful to write statistics for projection result.
ColumnsStatistics{}, ColumnsStatistics{},
compression_codec, compression_codec,
std::move(index_granularity_ptr),
Tx::PrehistoricTID, Tx::PrehistoricTID,
/*reset_columns=*/ false, /*reset_columns=*/ false,
save_marks_in_cache, save_marks_in_cache,

View File

@ -1,77 +1,23 @@
#include <Storages/MergeTree/MergeTreeIndexGranularity.h> #include <Storages/MergeTree/MergeTreeIndexGranularity.h>
#include <Common/Exception.h> #include <Storages/MergeTree/MergeTreeIndexGranularityAdaptive.h>
#include <Storages/MergeTree/MergeTreeIndexGranularityConstant.h>
#include <Storages/MergeTree/MergeTreeIndexGranularityInfo.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
MergeTreeIndexGranularity::MergeTreeIndexGranularity(const std::vector<size_t> & marks_rows_partial_sums_) namespace MergeTreeSetting
: marks_rows_partial_sums(marks_rows_partial_sums_)
{ {
} extern const MergeTreeSettingsUInt64 index_granularity;
extern const MergeTreeSettingsUInt64 index_granularity_bytes;
/// Rows after mark to next mark extern const MergeTreeSettingsBool use_const_adaptive_granularity;
size_t MergeTreeIndexGranularity::getMarkRows(size_t mark_index) const
{
if (mark_index >= getMarksCount())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get non existing mark {}, while size is {}", mark_index, getMarksCount());
if (mark_index == 0)
return marks_rows_partial_sums[0];
return marks_rows_partial_sums[mark_index] - marks_rows_partial_sums[mark_index - 1];
}
size_t MergeTreeIndexGranularity::getMarkStartingRow(size_t mark_index) const
{
if (mark_index == 0)
return 0;
return marks_rows_partial_sums[mark_index - 1];
}
size_t MergeTreeIndexGranularity::getMarksCount() const
{
return marks_rows_partial_sums.size();
}
size_t MergeTreeIndexGranularity::getTotalRows() const
{
if (marks_rows_partial_sums.empty())
return 0;
return marks_rows_partial_sums.back();
}
void MergeTreeIndexGranularity::appendMark(size_t rows_count)
{
if (marks_rows_partial_sums.empty())
marks_rows_partial_sums.push_back(rows_count);
else
marks_rows_partial_sums.push_back(marks_rows_partial_sums.back() + rows_count);
}
void MergeTreeIndexGranularity::addRowsToLastMark(size_t rows_count)
{
if (marks_rows_partial_sums.empty())
marks_rows_partial_sums.push_back(rows_count);
else
marks_rows_partial_sums.back() += rows_count;
}
void MergeTreeIndexGranularity::popMark()
{
if (!marks_rows_partial_sums.empty())
marks_rows_partial_sums.pop_back();
}
size_t MergeTreeIndexGranularity::getRowsCountInRange(size_t begin, size_t end) const
{
size_t subtrahend = 0;
if (begin != 0)
subtrahend = marks_rows_partial_sums[begin - 1];
return marks_rows_partial_sums[end - 1] - subtrahend;
} }
size_t MergeTreeIndexGranularity::getRowsCountInRange(const MarkRange & range) const size_t MergeTreeIndexGranularity::getRowsCountInRange(const MarkRange & range) const
@ -87,55 +33,118 @@ size_t MergeTreeIndexGranularity::getRowsCountInRanges(const MarkRanges & ranges
return total; return total;
} }
size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t number_of_rows) const size_t MergeTreeIndexGranularity::getMarksCountWithoutFinal() const
{ {
size_t rows_before_mark = getMarkStartingRow(from_mark); size_t total = getMarksCount();
size_t last_row_pos = rows_before_mark + number_of_rows; if (total == 0)
auto it = std::upper_bound(marks_rows_partial_sums.begin(), marks_rows_partial_sums.end(), last_row_pos); return total;
size_t to_mark = it - marks_rows_partial_sums.begin(); return total - hasFinalMark();
return to_mark - from_mark;
} }
size_t MergeTreeIndexGranularity::countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const size_t MergeTreeIndexGranularity::getMarkStartingRow(size_t mark_index) const
{ {
size_t rows_before_mark = getMarkStartingRow(from_mark); return getRowsCountInRange(0, mark_index);
size_t last_row_pos = rows_before_mark + offset_in_rows + number_of_rows;
auto it = std::upper_bound(marks_rows_partial_sums.begin(), marks_rows_partial_sums.end(), last_row_pos);
size_t to_mark = it - marks_rows_partial_sums.begin();
return getRowsCountInRange(from_mark, std::max(1UL, to_mark)) - offset_in_rows;
} }
void MergeTreeIndexGranularity::resizeWithFixedGranularity(size_t size, size_t fixed_granularity) size_t MergeTreeIndexGranularity::getLastMarkRows() const
{ {
marks_rows_partial_sums.resize(size); return getMarkRows(getMarksCount() - 1);
}
size_t prev = 0; size_t MergeTreeIndexGranularity::getLastNonFinalMarkRows() const
for (size_t i = 0; i < size; ++i) {
size_t last_mark_rows = getMarkRows(getMarksCount() - 1);
if (last_mark_rows != 0)
return last_mark_rows;
return getMarkRows(getMarksCount() - 2);
}
void MergeTreeIndexGranularity::addRowsToLastMark(size_t rows_count)
{
if (hasFinalMark())
{ {
marks_rows_partial_sums[i] = fixed_granularity + prev; throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add rows to final mark");
prev = marks_rows_partial_sums[i]; }
else if (empty())
{
appendMark(rows_count);
}
else
{
adjustLastMark(getLastMarkRows() + rows_count);
} }
} }
std::string MergeTreeIndexGranularity::describe() const size_t computeIndexGranularity(
size_t rows,
size_t bytes_uncompressed,
size_t index_granularity_bytes,
size_t fixed_index_granularity_rows,
bool blocks_are_granules,
bool can_use_adaptive_index_granularity)
{ {
return fmt::format("initialized: {}, marks_rows_partial_sums: [{}]", initialized, fmt::join(marks_rows_partial_sums, ", ")); size_t index_granularity_for_block;
if (!can_use_adaptive_index_granularity)
{
index_granularity_for_block = fixed_index_granularity_rows;
}
else
{
if (blocks_are_granules)
{
index_granularity_for_block = rows;
}
else if (bytes_uncompressed >= index_granularity_bytes)
{
size_t granules_in_block = bytes_uncompressed / index_granularity_bytes;
index_granularity_for_block = rows / granules_in_block;
}
else
{
size_t size_of_row_in_bytes = std::max(bytes_uncompressed / rows, 1UL);
index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes;
}
}
/// We should be less or equal than fixed index granularity.
/// But if block size is a granule size then do not adjust it.
/// Granularity greater than fixed granularity might come from compact part.
if (!blocks_are_granules)
index_granularity_for_block = std::min(fixed_index_granularity_rows, index_granularity_for_block);
/// Very rare case when index granularity bytes less than single row.
if (index_granularity_for_block == 0)
index_granularity_for_block = 1;
return index_granularity_for_block;
} }
void MergeTreeIndexGranularity::shrinkToFitInMemory() MergeTreeIndexGranularityPtr createMergeTreeIndexGranularity(
size_t rows,
size_t bytes_uncompressed,
const MergeTreeSettings & settings,
const MergeTreeIndexGranularityInfo & info,
bool blocks_are_granules)
{ {
marks_rows_partial_sums.shrink_to_fit(); bool use_adaptive_granularity = info.mark_type.adaptive;
bool use_const_adaptive_granularity = settings[MergeTreeSetting::use_const_adaptive_granularity];
bool is_compact_part = info.mark_type.part_type == MergeTreeDataPartType::Compact;
/// Compact parts cannot work without adaptive granularity.
/// If part is empty create adaptive granularity because constant granularity doesn't support this corner case.
if (rows == 0 || blocks_are_granules || is_compact_part || (use_adaptive_granularity && !use_const_adaptive_granularity))
return std::make_shared<MergeTreeIndexGranularityAdaptive>();
size_t computed_granularity = computeIndexGranularity(
rows,
bytes_uncompressed,
settings[MergeTreeSetting::index_granularity_bytes],
settings[MergeTreeSetting::index_granularity],
blocks_are_granules,
use_adaptive_granularity);
return std::make_shared<MergeTreeIndexGranularityConstant>(computed_granularity);
} }
uint64_t MergeTreeIndexGranularity::getBytesSize() const
{
return marks_rows_partial_sums.size() * sizeof(size_t);
}
uint64_t MergeTreeIndexGranularity::getBytesAllocated() const
{
return marks_rows_partial_sums.capacity() * sizeof(size_t);
}
} }

View File

@ -1,35 +1,28 @@
#pragma once #pragma once
#include <vector> #include <optional>
#include <Storages/MergeTree/MarkRange.h> #include <Storages/MergeTree/MarkRange.h>
namespace DB namespace DB
{ {
/// Class contains information about index granularity in rows of IMergeTreeDataPart /// Class that contains information about index granularity in rows of IMergeTreeDataPart
/// Inside it contains vector of partial sums of rows after mark:
/// |-----|---|----|----|
/// | 5 | 8 | 12 | 16 |
/// If user doesn't specify setting index_granularity_bytes for MergeTree* table
/// all values in inner vector would have constant stride (default 8192).
class MergeTreeIndexGranularity class MergeTreeIndexGranularity
{ {
private:
std::vector<size_t> marks_rows_partial_sums;
bool initialized = false;
public: public:
MergeTreeIndexGranularity() = default; MergeTreeIndexGranularity() = default;
explicit MergeTreeIndexGranularity(const std::vector<size_t> & marks_rows_partial_sums_); virtual ~MergeTreeIndexGranularity() = default;
/// Returns granularity if it is constant for whole part (except last granule).
virtual std::optional<size_t> getConstantGranularity() const = 0;
/// Return count of rows between marks
virtual size_t getRowsCountInRange(size_t begin, size_t end) const = 0;
/// Return count of rows between marks /// Return count of rows between marks
size_t getRowsCountInRange(const MarkRange & range) const; size_t getRowsCountInRange(const MarkRange & range) const;
/// Return count of rows between marks
size_t getRowsCountInRange(size_t begin, size_t end) const;
/// Return sum of rows between all ranges /// Return sum of rows between all ranges
size_t getRowsCountInRanges(const MarkRanges & ranges) const; size_t getRowsCountInRanges(const MarkRanges & ranges) const;
/// Return number of marks, starting from `from_marks` that contain `number_of_rows` /// Return number of marks, starting from `from_marks` that contain `number_of_rows`
size_t countMarksForRows(size_t from_mark, size_t number_of_rows) const; virtual size_t countMarksForRows(size_t from_mark, size_t number_of_rows) const = 0;
/// Return number of rows, starting from `from_mark`, that contains amount of `number_of_rows` /// Return number of rows, starting from `from_mark`, that contains amount of `number_of_rows`
/// and possible some offset_in_rows from `from_mark` /// and possible some offset_in_rows from `from_mark`
@ -37,74 +30,65 @@ public:
/// |-----|---------------------------|----|----| /// |-----|---------------------------|----|----|
/// ^------------------------^-----------^ /// ^------------------------^-----------^
//// from_mark offset_in_rows number_of_rows //// from_mark offset_in_rows number_of_rows
size_t countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const; virtual size_t countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const = 0;
/// Total marks /// Total marks
size_t getMarksCount() const; virtual size_t getMarksCount() const = 0;
/// Total rows /// Total rows
size_t getTotalRows() const; virtual size_t getTotalRows() const = 0;
/// Total number marks without final mark if it exists /// Total number marks without final mark if it exists
size_t getMarksCountWithoutFinal() const { return getMarksCount() - hasFinalMark(); } size_t getMarksCountWithoutFinal() const;
/// Rows after mark to next mark /// Rows after mark to next mark
size_t getMarkRows(size_t mark_index) const; virtual size_t getMarkRows(size_t mark_index) const = 0;
/// Return amount of rows before mark /// Return amount of rows before mark
size_t getMarkStartingRow(size_t mark_index) const; size_t getMarkStartingRow(size_t mark_index) const;
/// Amount of rows after last mark /// Amount of rows after last mark
size_t getLastMarkRows() const size_t getLastMarkRows() const;
{
size_t last = marks_rows_partial_sums.size() - 1;
return getMarkRows(last);
}
size_t getLastNonFinalMarkRows() const /// Amount of rows after last non-final mark
{ size_t getLastNonFinalMarkRows() const;
size_t last_mark_rows = getLastMarkRows();
if (last_mark_rows != 0)
return last_mark_rows;
return getMarkRows(marks_rows_partial_sums.size() - 2);
}
bool hasFinalMark() const virtual bool hasFinalMark() const = 0;
{ bool empty() const { return getMarksCount() == 0; }
return getLastMarkRows() == 0;
}
bool empty() const /// Add new mark with rows_count.
{ virtual void appendMark(size_t rows_count) = 0;
return marks_rows_partial_sums.empty();
}
bool isInitialized() const /// Sets last mark equal to rows_count.
{ virtual void adjustLastMark(size_t rows_count) = 0;
return initialized;
}
void setInitialized()
{
initialized = true;
}
/// Add new mark with rows_count
void appendMark(size_t rows_count);
/// Extends last mark by rows_count.
void addRowsToLastMark(size_t rows_count); void addRowsToLastMark(size_t rows_count);
/// Drops last mark if any exists. virtual uint64_t getBytesSize() const = 0;
void popMark(); virtual uint64_t getBytesAllocated() const = 0;
/// Add `size` of marks with `fixed_granularity` rows /// Possibly optimizes values in memory (for example, to constant value).
void resizeWithFixedGranularity(size_t size, size_t fixed_granularity); /// Returns new optimized index granularity structure or nullptr if no optimization is not applicable.
virtual std::shared_ptr<MergeTreeIndexGranularity> optimize() = 0;
std::string describe() const; virtual std::string describe() const = 0;
void shrinkToFitInMemory();
uint64_t getBytesSize() const;
uint64_t getBytesAllocated() const;
}; };
using MergeTreeIndexGranularityPtr = std::shared_ptr<MergeTreeIndexGranularity>;
size_t computeIndexGranularity(
size_t rows,
size_t bytes_uncompressed,
size_t index_granularity_bytes,
size_t fixed_index_granularity_rows,
bool blocks_are_granules,
bool can_use_adaptive_index_granularity);
struct MergeTreeSettings;
struct MergeTreeIndexGranularityInfo;
MergeTreeIndexGranularityPtr createMergeTreeIndexGranularity(
size_t rows,
size_t bytes_uncompressed,
const MergeTreeSettings & settings,
const MergeTreeIndexGranularityInfo & info,
bool blocks_are_granules);
} }

View File

@ -0,0 +1,152 @@
#include <Storages/MergeTree/MergeTreeIndexGranularityAdaptive.h>
#include <Storages/MergeTree/MergeTreeIndexGranularityConstant.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
MergeTreeIndexGranularityAdaptive::MergeTreeIndexGranularityAdaptive(const std::vector<size_t> & marks_rows_partial_sums_)
: marks_rows_partial_sums(marks_rows_partial_sums_)
{
}
/// Rows after mark to next mark
size_t MergeTreeIndexGranularityAdaptive::getMarkRows(size_t mark_index) const
{
if (mark_index >= getMarksCount())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get non existing mark {}, while size is {}", mark_index, getMarksCount());
if (mark_index == 0)
return marks_rows_partial_sums[0];
return marks_rows_partial_sums[mark_index] - marks_rows_partial_sums[mark_index - 1];
}
bool MergeTreeIndexGranularityAdaptive::hasFinalMark() const
{
if (marks_rows_partial_sums.empty())
return false;
return getLastMarkRows() == 0;
}
size_t MergeTreeIndexGranularityAdaptive::getMarksCount() const
{
return marks_rows_partial_sums.size();
}
size_t MergeTreeIndexGranularityAdaptive::getTotalRows() const
{
if (marks_rows_partial_sums.empty())
return 0;
return marks_rows_partial_sums.back();
}
void MergeTreeIndexGranularityAdaptive::appendMark(size_t rows_count)
{
if (hasFinalMark())
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot append mark after final");
}
else if (marks_rows_partial_sums.empty())
{
marks_rows_partial_sums.push_back(rows_count);
}
else
{
marks_rows_partial_sums.push_back(marks_rows_partial_sums.back() + rows_count);
}
}
void MergeTreeIndexGranularityAdaptive::adjustLastMark(size_t rows_count)
{
if (hasFinalMark())
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot adjust final mark");
}
else if (marks_rows_partial_sums.empty())
{
marks_rows_partial_sums.push_back(rows_count);
}
else
{
marks_rows_partial_sums.pop_back();
appendMark(rows_count);
}
}
size_t MergeTreeIndexGranularityAdaptive::getRowsCountInRange(size_t begin, size_t end) const
{
if (end > getMarksCount())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get marks in range [{}; {}), while size is {}", begin, end, getMarksCount());
if (end == 0)
return 0;
size_t subtrahend = 0;
if (begin != 0)
subtrahend = marks_rows_partial_sums[begin - 1];
return marks_rows_partial_sums[end - 1] - subtrahend;
}
size_t MergeTreeIndexGranularityAdaptive::countMarksForRows(size_t from_mark, size_t number_of_rows) const
{
size_t rows_before_mark = getMarkStartingRow(from_mark);
size_t last_row_pos = rows_before_mark + number_of_rows;
auto it = std::upper_bound(marks_rows_partial_sums.begin(), marks_rows_partial_sums.end(), last_row_pos);
size_t to_mark = it - marks_rows_partial_sums.begin();
return to_mark - from_mark;
}
size_t MergeTreeIndexGranularityAdaptive::countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const
{
size_t rows_before_mark = getMarkStartingRow(from_mark);
size_t last_row_pos = rows_before_mark + offset_in_rows + number_of_rows;
auto it = std::upper_bound(marks_rows_partial_sums.begin(), marks_rows_partial_sums.end(), last_row_pos);
size_t to_mark = it - marks_rows_partial_sums.begin();
return getRowsCountInRange(from_mark, std::max(1UL, to_mark)) - offset_in_rows;
}
uint64_t MergeTreeIndexGranularityAdaptive::getBytesSize() const
{
return marks_rows_partial_sums.size() * sizeof(size_t);
}
uint64_t MergeTreeIndexGranularityAdaptive::getBytesAllocated() const
{
return marks_rows_partial_sums.capacity() * sizeof(size_t);
}
std::shared_ptr<MergeTreeIndexGranularity> MergeTreeIndexGranularityAdaptive::optimize()
{
size_t marks_count = getMarksCountWithoutFinal();
if (marks_count == 0)
return nullptr;
size_t first_mark = getMarkRows(0);
for (size_t i = 1; i < marks_count - 1; ++i)
{
if (getMarkRows(i) != first_mark)
{
/// We cannot optimize to constant but at least optimize memory usage.
marks_rows_partial_sums.shrink_to_fit();
return nullptr;
}
}
size_t last_mark = getMarkRows(marks_count - 1);
return std::make_shared<MergeTreeIndexGranularityConstant>(first_mark, last_mark, marks_count, hasFinalMark());
}
std::string MergeTreeIndexGranularityAdaptive::describe() const
{
return fmt::format("Adaptive(marks_rows_partial_sums: [{}])", fmt::join(marks_rows_partial_sums, ", "));
}
}

View File

@ -0,0 +1,42 @@
#pragma once
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
namespace DB
{
/// Class that stores adaptive index granularity.
/// Inside it contains vector of partial sums of rows after mark:
/// |-----|---|----|----|
/// | 5 | 8 | 12 | 16 |
class MergeTreeIndexGranularityAdaptive final : public MergeTreeIndexGranularity
{
public:
MergeTreeIndexGranularityAdaptive() = default;
explicit MergeTreeIndexGranularityAdaptive(const std::vector<size_t> & marks_rows_partial_sums_);
std::optional<size_t> getConstantGranularity() const override { return {}; }
size_t getRowsCountInRange(size_t begin, size_t end) const override;
size_t countMarksForRows(size_t from_mark, size_t number_of_rows) const override;
size_t countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const override;
size_t getMarksCount() const override;
size_t getTotalRows() const override;
size_t getMarkRows(size_t mark_index) const override;
bool hasFinalMark() const override;
void appendMark(size_t rows_count) override;
void adjustLastMark(size_t rows_count) override;
uint64_t getBytesSize() const override;
uint64_t getBytesAllocated() const override;
std::shared_ptr<MergeTreeIndexGranularity> optimize() override;
std::string describe() const override;
private:
std::vector<size_t> marks_rows_partial_sums;
};
}

View File

@ -0,0 +1,143 @@
#include <Storages/MergeTree/MergeTreeIndexGranularityConstant.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
MergeTreeIndexGranularityConstant::MergeTreeIndexGranularityConstant(size_t constant_granularity_)
: constant_granularity(constant_granularity_)
, last_mark_granularity(constant_granularity_)
{
}
MergeTreeIndexGranularityConstant::MergeTreeIndexGranularityConstant(size_t constant_granularity_, size_t last_mark_granularity_, size_t num_marks_without_final_, bool has_final_mark_)
: constant_granularity(constant_granularity_)
, last_mark_granularity(last_mark_granularity_)
, num_marks_without_final(num_marks_without_final_)
, has_final_mark(has_final_mark_)
{
}
/// Rows after mark to next mark
size_t MergeTreeIndexGranularityConstant::getMarkRows(size_t mark_index) const
{
if (mark_index >= getMarksCount())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get non existing mark {}, while size is {}", mark_index, getMarksCount());
if (mark_index + 1 < num_marks_without_final)
return constant_granularity;
if (mark_index + 1 == num_marks_without_final)
return last_mark_granularity;
return 0; // Final mark.
}
size_t MergeTreeIndexGranularityConstant::getMarksCount() const
{
return num_marks_without_final + has_final_mark;
}
size_t MergeTreeIndexGranularityConstant::getTotalRows() const
{
if (num_marks_without_final == 0)
return 0;
return constant_granularity * (num_marks_without_final - 1) + last_mark_granularity;
}
void MergeTreeIndexGranularityConstant::appendMark(size_t rows_count)
{
if (has_final_mark)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot append mark after final");
}
else if (rows_count == 0)
{
has_final_mark = true;
}
else if (rows_count != constant_granularity)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot append mark with {} rows. Granularity is constant ({})", rows_count, constant_granularity);
}
else
{
++num_marks_without_final;
}
}
void MergeTreeIndexGranularityConstant::adjustLastMark(size_t rows_count)
{
if (has_final_mark)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot adjust final mark");
}
else
{
if (num_marks_without_final == 0)
++num_marks_without_final;
last_mark_granularity = rows_count;
}
}
size_t MergeTreeIndexGranularityConstant::getRowsCountInRange(size_t begin, size_t end) const
{
if (end > getMarksCount())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get marks in range [{}; {}), while size is {}", begin, end, getMarksCount());
if (end == 0)
return 0;
size_t total_rows = 0;
if (end >= num_marks_without_final)
{
total_rows += last_mark_granularity;
end = num_marks_without_final - 1;
}
total_rows += constant_granularity * (end - begin);
return total_rows;
}
size_t MergeTreeIndexGranularityConstant::getMarkUpperBoundForRow(size_t row_index) const
{
size_t num_rows_with_constant_granularity = (num_marks_without_final - 1) * constant_granularity;
if (row_index >= getTotalRows())
return getMarksCount();
if (row_index >= num_rows_with_constant_granularity)
return num_marks_without_final - 1;
return row_index / constant_granularity;
}
size_t MergeTreeIndexGranularityConstant::countMarksForRows(size_t from_mark, size_t number_of_rows) const
{
size_t rows_before_mark = getMarkStartingRow(from_mark);
size_t last_row_pos = rows_before_mark + number_of_rows;
return getMarkUpperBoundForRow(last_row_pos) - from_mark;
}
size_t MergeTreeIndexGranularityConstant::countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const
{
size_t rows_before_mark = getMarkStartingRow(from_mark);
size_t last_row_pos = rows_before_mark + offset_in_rows + number_of_rows;
return getRowsCountInRange(from_mark, std::max(1UL, getMarkUpperBoundForRow(last_row_pos))) - offset_in_rows;
}
std::string MergeTreeIndexGranularityConstant::describe() const
{
return fmt::format(
"Constant(constant_granularity: {}, last_mark_granularity: {}, num_marks_without_final: {}, has_final_mark: {})",
constant_granularity, last_mark_granularity, num_marks_without_final, has_final_mark);
}
}

View File

@ -0,0 +1,47 @@
#pragma once
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
namespace DB
{
/// Class that stores constant index granularity for whole part, except
/// last non-zero granule and final granule which always has zero rows.
class MergeTreeIndexGranularityConstant final : public MergeTreeIndexGranularity
{
private:
size_t constant_granularity;
size_t last_mark_granularity;
size_t num_marks_without_final = 0;
bool has_final_mark = false;
size_t getMarkUpperBoundForRow(size_t row_index) const;
public:
MergeTreeIndexGranularityConstant() = default;
explicit MergeTreeIndexGranularityConstant(size_t constant_granularity_);
MergeTreeIndexGranularityConstant(size_t constant_granularity_, size_t last_mark_granularity_, size_t num_marks_without_final_, bool has_final_mark_);
std::optional<size_t> getConstantGranularity() const override { return constant_granularity; }
size_t getRowsCountInRange(size_t begin, size_t end) const override;
size_t countMarksForRows(size_t from_mark, size_t number_of_rows) const override;
size_t countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const override;
size_t getMarksCount() const override;
size_t getTotalRows() const override;
size_t getMarkRows(size_t mark_index) const override;
bool hasFinalMark() const override { return has_final_mark; }
void appendMark(size_t rows_count) override;
void adjustLastMark(size_t rows_count) override;
uint64_t getBytesSize() const override { return sizeof(size_t) * 3 + sizeof(bool); }
uint64_t getBytesAllocated() const override { return getBytesSize(); }
std::shared_ptr<MergeTreeIndexGranularity> optimize() override { return nullptr; }
std::string describe() const override;
};
}

View File

@ -4,12 +4,12 @@
#include <base/types.h> #include <base/types.h>
#include <Storages/MergeTree/MergeTreeDataPartType.h> #include <Storages/MergeTree/MergeTreeDataPartType.h>
#include <Disks/IDisk.h> #include <Disks/IDisk.h>
#include <Storages/MergeTree/IDataPartStorage.h>
namespace DB namespace DB
{ {
class MergeTreeData; class MergeTreeData;
class IDataPartStorage;
/** Various types of mark files are stored in files with various extensions: /** Various types of mark files are stored in files with various extensions:

View File

@ -151,7 +151,7 @@ UInt64 MergeTreeReadTask::estimateNumRows() const
return rows_to_read; return rows_to_read;
const auto & index_granularity = info->data_part->index_granularity; const auto & index_granularity = info->data_part->index_granularity;
return index_granularity.countRowsForRows(range_readers.main.currentMark(), rows_to_read, range_readers.main.numReadRowsInCurrentGranule()); return index_granularity->countRowsForRows(range_readers.main.currentMark(), rows_to_read, range_readers.main.numReadRowsInCurrentGranule());
} }
MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read() MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read()

View File

@ -230,7 +230,7 @@ try
if (!isCancelled() && current_row < data_part->rows_count) if (!isCancelled() && current_row < data_part->rows_count)
{ {
size_t rows_to_read = data_part->index_granularity.getMarkRows(current_mark); size_t rows_to_read = data_part->index_granularity->getMarkRows(current_mark);
bool continue_reading = (current_mark != 0); bool continue_reading = (current_mark != 0);
const auto & sample = reader->getColumns(); const auto & sample = reader->getColumns();

View File

@ -187,6 +187,8 @@ namespace ErrorCodes
DECLARE(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ DECLARE(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \
DECLARE(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ DECLARE(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \
DECLARE(UInt64, min_index_granularity_bytes, 1024, "Minimum amount of bytes in single granule.", 1024) \ DECLARE(UInt64, min_index_granularity_bytes, 1024, "Minimum amount of bytes in single granule.", 1024) \
DECLARE(Bool, use_const_adaptive_granularity, false, "Always use constant granularity for whole part. It allows to compress in memory values of index granularity. It can be useful in extremely large workloads with thin tables.", 0) \
DECLARE(Bool, enable_index_granularity_compression, true, "Compress in memory values of index granularity if it is possible", 0) \
DECLARE(Int64, merge_with_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with delete TTL can be repeated.", 0) \ DECLARE(Int64, merge_with_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with delete TTL can be repeated.", 0) \
DECLARE(Int64, merge_with_recompression_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with recompression TTL can be repeated.", 0) \ DECLARE(Int64, merge_with_recompression_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with recompression TTL can be repeated.", 0) \
DECLARE(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ DECLARE(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \

View File

@ -1,9 +1,9 @@
#include <Storages/MergeTree/MergedBlockOutputStream.h> #include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <IO/HashingWriteBuffer.h> #include <IO/HashingWriteBuffer.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/MergeTreeTransaction.h> #include <Interpreters/MergeTreeTransaction.h>
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>
#include <Common/logger_useful.h>
#include <Core/Settings.h> #include <Core/Settings.h>
@ -15,6 +15,10 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
namespace MergeTreeSetting
{
extern const MergeTreeSettingsBool enable_index_granularity_compression;
}
MergedBlockOutputStream::MergedBlockOutputStream( MergedBlockOutputStream::MergedBlockOutputStream(
const MergeTreeMutableDataPartPtr & data_part, const MergeTreeMutableDataPartPtr & data_part,
@ -23,12 +27,12 @@ MergedBlockOutputStream::MergedBlockOutputStream(
const MergeTreeIndices & skip_indices, const MergeTreeIndices & skip_indices,
const ColumnsStatistics & statistics, const ColumnsStatistics & statistics,
CompressionCodecPtr default_codec_, CompressionCodecPtr default_codec_,
MergeTreeIndexGranularityPtr index_granularity_ptr,
TransactionID tid, TransactionID tid,
bool reset_columns_, bool reset_columns_,
bool save_marks_in_cache, bool save_marks_in_cache,
bool blocks_are_granules_size, bool blocks_are_granules_size,
const WriteSettings & write_settings_, const WriteSettings & write_settings_)
const MergeTreeIndexGranularity & computed_index_granularity)
: IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, reset_columns_) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, reset_columns_)
, columns_list(columns_list_) , columns_list(columns_list_)
, default_codec(default_codec_) , default_codec(default_codec_)
@ -53,11 +57,22 @@ MergedBlockOutputStream::MergedBlockOutputStream(
data_part->storeVersionMetadata(); data_part->storeVersionMetadata();
writer = createMergeTreeDataPartWriter(data_part->getType(), writer = createMergeTreeDataPartWriter(data_part->getType(),
data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), data_part->name,
data_part_storage, data_part->index_granularity_info, data_part->storage.getLogName(),
storage_settings, data_part->getSerializations(),
columns_list, data_part->getColumnPositions(), metadata_snapshot, data_part->storage.getVirtualsPtr(), data_part_storage,
skip_indices, statistics, data_part->getMarksFileExtension(), default_codec, writer_settings, computed_index_granularity); data_part->index_granularity_info,
storage_settings,
columns_list,
data_part->getColumnPositions(),
metadata_snapshot,
data_part->storage.getVirtualsPtr(),
skip_indices,
statistics,
data_part->getMarksFileExtension(),
default_codec,
writer_settings,
std::move(index_granularity_ptr));
} }
/// If data is pre-sorted. /// If data is pre-sorted.
@ -207,10 +222,14 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync(
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
new_part->setBytesUncompressedOnDisk(checksums.getTotalSizeUncompressedOnDisk()); new_part->setBytesUncompressedOnDisk(checksums.getTotalSizeUncompressedOnDisk());
new_part->index_granularity = writer->getIndexGranularity(); new_part->index_granularity = writer->getIndexGranularity();
/// Just in case
new_part->index_granularity.shrinkToFitInMemory();
new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(writer->getColumnsSample()); new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(writer->getColumnsSample());
if ((*new_part->storage.getSettings())[MergeTreeSetting::enable_index_granularity_compression])
{
if (auto new_index_granularity = new_part->index_granularity->optimize())
new_part->index_granularity = std::move(new_index_granularity);
}
/// In mutation, existing_rows_count is already calculated in PartMergerWriter /// In mutation, existing_rows_count is already calculated in PartMergerWriter
/// In merge situation, lightweight deleted rows was physically deleted, existing_rows_count equals rows_count /// In merge situation, lightweight deleted rows was physically deleted, existing_rows_count equals rows_count
if (!new_part->existing_rows_count.has_value()) if (!new_part->existing_rows_count.has_value())

View File

@ -22,12 +22,12 @@ public:
const MergeTreeIndices & skip_indices, const MergeTreeIndices & skip_indices,
const ColumnsStatistics & statistics, const ColumnsStatistics & statistics,
CompressionCodecPtr default_codec_, CompressionCodecPtr default_codec_,
MergeTreeIndexGranularityPtr index_granularity_ptr,
TransactionID tid, TransactionID tid,
bool reset_columns_ = false, bool reset_columns_ = false,
bool save_marks_in_cache = false, bool save_marks_in_cache = false,
bool blocks_are_granules_size = false, bool blocks_are_granules_size = false,
const WriteSettings & write_settings = {}, const WriteSettings & write_settings = {});
const MergeTreeIndexGranularity & computed_index_granularity = {});
Block getHeader() const { return metadata_snapshot->getSampleBlock(); } Block getHeader() const { return metadata_snapshot->getSampleBlock(); }

View File

@ -15,25 +15,25 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
const MergeTreeMutableDataPartPtr & data_part, const MergeTreeMutableDataPartPtr & data_part,
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const NamesAndTypesList & columns_list_, const NamesAndTypesList & columns_list_,
CompressionCodecPtr default_codec,
const MergeTreeIndices & indices_to_recalc, const MergeTreeIndices & indices_to_recalc,
const ColumnsStatistics & stats_to_recalc_, const ColumnsStatistics & stats_to_recalc,
WrittenOffsetColumns * offset_columns_, CompressionCodecPtr default_codec,
bool save_marks_in_cache, MergeTreeIndexGranularityPtr index_granularity_ptr,
const MergeTreeIndexGranularity & index_granularity, WrittenOffsetColumns * offset_columns,
const MergeTreeIndexGranularityInfo * index_granularity_info) bool save_marks_in_cache)
: IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true)
{ {
const auto & global_settings = data_part->storage.getContext()->getSettingsRef(); const auto & global_settings = data_part->storage.getContext()->getSettingsRef();
/// Granularity is never recomputed while writing only columns.
MergeTreeWriterSettings writer_settings( MergeTreeWriterSettings writer_settings(
global_settings, global_settings,
data_part->storage.getContext()->getWriteSettings(), data_part->storage.getContext()->getWriteSettings(),
storage_settings, storage_settings,
index_granularity_info ? index_granularity_info->mark_type.adaptive : data_part->storage.canUseAdaptiveGranularity(), data_part->index_granularity_info.mark_type.adaptive,
/* rewrite_primary_key = */ false, /*rewrite_primary_key=*/ false,
save_marks_in_cache, save_marks_in_cache,
/* blocks_are_granules_size = */ false); /*blocks_are_granules_size=*/ false);
writer = createMergeTreeDataPartWriter( writer = createMergeTreeDataPartWriter(
data_part->getType(), data_part->getType(),
@ -45,17 +45,17 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
metadata_snapshot_, metadata_snapshot_,
data_part->storage.getVirtualsPtr(), data_part->storage.getVirtualsPtr(),
indices_to_recalc, indices_to_recalc,
stats_to_recalc_, stats_to_recalc,
data_part->getMarksFileExtension(), data_part->getMarksFileExtension(),
default_codec, default_codec,
writer_settings, writer_settings,
index_granularity); std::move(index_granularity_ptr));
auto * writer_on_disk = dynamic_cast<MergeTreeDataPartWriterOnDisk *>(writer.get()); auto * writer_on_disk = dynamic_cast<MergeTreeDataPartWriterOnDisk *>(writer.get());
if (!writer_on_disk) if (!writer_on_disk)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergedColumnOnlyOutputStream supports only parts stored on disk"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergedColumnOnlyOutputStream supports only parts stored on disk");
writer_on_disk->setWrittenOffsetColumns(offset_columns_); writer_on_disk->setWrittenOffsetColumns(offset_columns);
} }
void MergedColumnOnlyOutputStream::write(const Block & block) void MergedColumnOnlyOutputStream::write(const Block & block)

View File

@ -18,13 +18,12 @@ public:
const MergeTreeMutableDataPartPtr & data_part, const MergeTreeMutableDataPartPtr & data_part,
const StorageMetadataPtr & metadata_snapshot_, const StorageMetadataPtr & metadata_snapshot_,
const NamesAndTypesList & columns_list_, const NamesAndTypesList & columns_list_,
CompressionCodecPtr default_codec_, const MergeTreeIndices & indices_to_recalc,
const MergeTreeIndices & indices_to_recalc_, const ColumnsStatistics & stats_to_recalc,
const ColumnsStatistics & stats_to_recalc_, CompressionCodecPtr default_codec,
WrittenOffsetColumns * offset_columns_ = nullptr, MergeTreeIndexGranularityPtr index_granularity_ptr,
bool save_marks_in_cache = false, WrittenOffsetColumns * offset_columns = nullptr,
const MergeTreeIndexGranularity & index_granularity = {}, bool save_marks_in_cache = false);
const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr);
void write(const Block & block) override; void write(const Block & block) override;

View File

@ -74,6 +74,7 @@ namespace MergeTreeSetting
extern const MergeTreeSettingsFloat ratio_of_defaults_for_sparse_serialization; extern const MergeTreeSettingsFloat ratio_of_defaults_for_sparse_serialization;
extern const MergeTreeSettingsBool replace_long_file_name_to_hash; extern const MergeTreeSettingsBool replace_long_file_name_to_hash;
extern const MergeTreeSettingsBool ttl_only_drop_parts; extern const MergeTreeSettingsBool ttl_only_drop_parts;
extern const MergeTreeSettingsBool enable_index_granularity_compression;
} }
namespace ErrorCodes namespace ErrorCodes
@ -984,12 +985,16 @@ void finalizeMutatedPart(
new_data_part->rows_count = source_part->rows_count; new_data_part->rows_count = source_part->rows_count;
new_data_part->index_granularity = source_part->index_granularity; new_data_part->index_granularity = source_part->index_granularity;
/// Just in case
new_data_part->index_granularity.shrinkToFitInMemory();
new_data_part->setIndex(*source_part->getIndex()); new_data_part->setIndex(*source_part->getIndex());
new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->minmax_idx = source_part->minmax_idx;
new_data_part->modification_time = time(nullptr); new_data_part->modification_time = time(nullptr);
if ((*new_data_part->storage.getSettings())[MergeTreeSetting::enable_index_granularity_compression])
{
if (auto new_index_granularity = new_data_part->index_granularity->optimize())
new_data_part->index_granularity = std::move(new_index_granularity);
}
/// Load rest projections which are hardlinked /// Load rest projections which are hardlinked
bool noop; bool noop;
new_data_part->loadProjections(false, false, noop, true /* if_not_loaded */); new_data_part->loadProjections(false, false, noop, true /* if_not_loaded */);
@ -1599,7 +1604,6 @@ private:
ctx->minmax_idx = std::make_shared<IMergeTreeDataPart::MinMaxIndex>(); ctx->minmax_idx = std::make_shared<IMergeTreeDataPart::MinMaxIndex>();
MergeTreeIndexGranularity computed_granularity;
bool has_delete = false; bool has_delete = false;
for (auto & command_for_interpreter : ctx->for_interpreter) for (auto & command_for_interpreter : ctx->for_interpreter)
@ -1612,9 +1616,21 @@ private:
} }
} }
MergeTreeIndexGranularityPtr index_granularity_ptr;
/// Reuse source part granularity if mutation does not change number of rows /// Reuse source part granularity if mutation does not change number of rows
if (!has_delete && ctx->execute_ttl_type == ExecuteTTLType::NONE) if (!has_delete && ctx->execute_ttl_type == ExecuteTTLType::NONE)
computed_granularity = ctx->source_part->index_granularity; {
index_granularity_ptr = ctx->source_part->index_granularity;
}
else
{
index_granularity_ptr = createMergeTreeIndexGranularity(
ctx->new_data_part->rows_count,
ctx->new_data_part->getBytesUncompressedOnDisk(),
*ctx->data->getSettings(),
ctx->new_data_part->index_granularity_info,
/*blocks_are_granules=*/ false);
}
ctx->out = std::make_shared<MergedBlockOutputStream>( ctx->out = std::make_shared<MergedBlockOutputStream>(
ctx->new_data_part, ctx->new_data_part,
@ -1623,12 +1639,12 @@ private:
skip_indices, skip_indices,
stats_to_rewrite, stats_to_rewrite,
ctx->compression_codec, ctx->compression_codec,
std::move(index_granularity_ptr),
ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID, ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID,
/*reset_columns=*/ true, /*reset_columns=*/ true,
/*save_marks_in_cache=*/ false, /*save_marks_in_cache=*/ false,
/*blocks_are_granules_size=*/ false, /*blocks_are_granules_size=*/ false,
ctx->context->getWriteSettings(), ctx->context->getWriteSettings());
computed_granularity);
ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback); ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback);
@ -1850,14 +1866,10 @@ private:
ctx->new_data_part, ctx->new_data_part,
ctx->metadata_snapshot, ctx->metadata_snapshot,
ctx->updated_header.getNamesAndTypesList(), ctx->updated_header.getNamesAndTypesList(),
ctx->compression_codec,
std::vector<MergeTreeIndexPtr>(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), std::vector<MergeTreeIndexPtr>(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()),
ColumnsStatistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()), ColumnsStatistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()),
nullptr, ctx->compression_codec,
/*save_marks_in_cache=*/ false, ctx->source_part->index_granularity);
ctx->source_part->index_granularity,
&ctx->source_part->index_granularity_info
);
ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback); ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback);

View File

@ -99,7 +99,7 @@ size_t RangesInDataPart::getMarksCount() const
size_t RangesInDataPart::getRowsCount() const size_t RangesInDataPart::getRowsCount() const
{ {
return data_part->index_granularity.getRowsCountInRanges(ranges); return data_part->index_granularity->getRowsCountInRanges(ranges);
} }

View File

@ -63,7 +63,7 @@ protected:
marks_loader = createMarksLoader(part, MergeTreeDataPartCompact::DATA_FILE_NAME, part->getColumns().size()); marks_loader = createMarksLoader(part, MergeTreeDataPartCompact::DATA_FILE_NAME, part->getColumns().size());
size_t num_columns = header.columns(); size_t num_columns = header.columns();
size_t num_rows = index_granularity.getMarksCount(); size_t num_rows = index_granularity->getMarksCount();
const auto & part_name_column = StorageMergeTreeIndex::part_name_column; const auto & part_name_column = StorageMergeTreeIndex::part_name_column;
const auto & mark_number_column = StorageMergeTreeIndex::mark_number_column; const auto & mark_number_column = StorageMergeTreeIndex::mark_number_column;
@ -115,7 +115,7 @@ protected:
data.resize(num_rows); data.resize(num_rows);
for (size_t i = 0; i < num_rows; ++i) for (size_t i = 0; i < num_rows; ++i)
data[i] = index_granularity.getMarkRows(i); data[i] = index_granularity->getMarkRows(i);
result_columns[pos] = std::move(column); result_columns[pos] = std::move(column);
} }
@ -159,7 +159,7 @@ private:
{ {
size_t col_idx = 0; size_t col_idx = 0;
bool has_marks_in_part = false; bool has_marks_in_part = false;
size_t num_rows = part->index_granularity.getMarksCount(); size_t num_rows = part->index_granularity->getMarksCount();
if (isWidePart(part)) if (isWidePart(part))
{ {

View File

@ -1,12 +1,15 @@
#include <gtest/gtest.h> #include <gtest/gtest.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <DataTypes/DataTypesNumber.h>
// I know that inclusion of .cpp is not good at all // I know that inclusion of .cpp is not good at all
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp> // NOLINT
#include <Storages/MergeTree/MergeTreeDataPartWriterWide.cpp> // NOLINT #include <Storages/MergeTree/MergeTreeDataPartWriterWide.cpp> // NOLINT
#include <Storages/MergeTree/MergeTreeIndexGranularityAdaptive.h>
#include <Storages/MergeTree/MergeTreeIndexGranularityConstant.h>
using namespace DB; using namespace DB;
static Block getBlockWithSize(size_t required_size_in_bytes, size_t size_of_row_in_bytes) static Block getBlockWithSize(size_t required_size_in_bytes, size_t size_of_row_in_bytes)
{ {
@ -25,16 +28,16 @@ TEST(AdaptiveIndexGranularity, FillGranularityToyTests)
auto block1 = getBlockWithSize(80, 8); auto block1 = getBlockWithSize(80, 8);
EXPECT_EQ(block1.bytes(), 80); EXPECT_EQ(block1.bytes(), 80);
{ /// Granularity bytes are not set. Take default index_granularity. { /// Granularity bytes are not set. Take default index_granularity.
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
auto granularity = computeIndexGranularityImpl(block1, 0, 100, false, false); auto granularity = computeIndexGranularity(block1.rows(), block1.bytes(), 0, 100, false, false);
fillIndexGranularityImpl(index_granularity, 0, granularity, block1.rows()); fillIndexGranularityImpl(index_granularity, 0, granularity, block1.rows());
EXPECT_EQ(index_granularity.getMarksCount(), 1); EXPECT_EQ(index_granularity.getMarksCount(), 1);
EXPECT_EQ(index_granularity.getMarkRows(0), 100); EXPECT_EQ(index_granularity.getMarkRows(0), 100);
} }
{ /// Granule size is less than block size. Block contains multiple granules. { /// Granule size is less than block size. Block contains multiple granules.
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
auto granularity = computeIndexGranularityImpl(block1, 16, 100, false, true); auto granularity = computeIndexGranularity(block1.rows(), block1.bytes(), 16, 100, false, true);
fillIndexGranularityImpl(index_granularity, 0, granularity, block1.rows()); fillIndexGranularityImpl(index_granularity, 0, granularity, block1.rows());
EXPECT_EQ(index_granularity.getMarksCount(), 5); /// First granule with 8 rows, and second with 1 row EXPECT_EQ(index_granularity.getMarksCount(), 5); /// First granule with 8 rows, and second with 1 row
for (size_t i = 0; i < index_granularity.getMarksCount(); ++i) for (size_t i = 0; i < index_granularity.getMarksCount(); ++i)
@ -43,8 +46,8 @@ TEST(AdaptiveIndexGranularity, FillGranularityToyTests)
{ /// Granule size is more than block size. Whole block (and maybe more) can be placed in single granule. { /// Granule size is more than block size. Whole block (and maybe more) can be placed in single granule.
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
auto granularity = computeIndexGranularityImpl(block1, 512, 100, false, true); auto granularity = computeIndexGranularity(block1.rows(), block1.bytes(), 512, 100, false, true);
fillIndexGranularityImpl(index_granularity, 0, granularity, block1.rows()); fillIndexGranularityImpl(index_granularity, 0, granularity, block1.rows());
EXPECT_EQ(index_granularity.getMarksCount(), 1); EXPECT_EQ(index_granularity.getMarksCount(), 1);
for (size_t i = 0; i < index_granularity.getMarksCount(); ++i) for (size_t i = 0; i < index_granularity.getMarksCount(); ++i)
@ -53,8 +56,8 @@ TEST(AdaptiveIndexGranularity, FillGranularityToyTests)
{ /// Blocks with granule size { /// Blocks with granule size
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
auto granularity = computeIndexGranularityImpl(block1, 1, 100, true, true); auto granularity = computeIndexGranularity(block1.rows(), block1.bytes(), 1, 100, true, true);
fillIndexGranularityImpl(index_granularity, 0, granularity, block1.rows()); fillIndexGranularityImpl(index_granularity, 0, granularity, block1.rows());
EXPECT_EQ(index_granularity.getMarksCount(), 1); EXPECT_EQ(index_granularity.getMarksCount(), 1);
for (size_t i = 0; i < index_granularity.getMarksCount(); ++i) for (size_t i = 0; i < index_granularity.getMarksCount(); ++i)
@ -62,8 +65,8 @@ TEST(AdaptiveIndexGranularity, FillGranularityToyTests)
} }
{ /// Shift in index offset { /// Shift in index offset
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
auto granularity = computeIndexGranularityImpl(block1, 16, 100, false, true); auto granularity = computeIndexGranularity(block1.rows(), block1.bytes(), 16, 100, false, true);
fillIndexGranularityImpl(index_granularity, 6, granularity, block1.rows()); fillIndexGranularityImpl(index_granularity, 6, granularity, block1.rows());
EXPECT_EQ(index_granularity.getMarksCount(), 2); EXPECT_EQ(index_granularity.getMarksCount(), 2);
for (size_t i = 0; i < index_granularity.getMarksCount(); ++i) for (size_t i = 0; i < index_granularity.getMarksCount(); ++i)
@ -78,10 +81,10 @@ TEST(AdaptiveIndexGranularity, FillGranularitySequenceOfBlocks)
auto block1 = getBlockWithSize(65536, 8); auto block1 = getBlockWithSize(65536, 8);
auto block2 = getBlockWithSize(65536, 8); auto block2 = getBlockWithSize(65536, 8);
auto block3 = getBlockWithSize(65536, 8); auto block3 = getBlockWithSize(65536, 8);
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
for (const auto & block : {block1, block2, block3}) for (const auto & block : {block1, block2, block3})
{ {
auto granularity = computeIndexGranularityImpl(block, 1024, 8192, false, true); auto granularity = computeIndexGranularity(block.rows(), block.bytes(), 1024, 8192, false, true);
fillIndexGranularityImpl(index_granularity, 0, granularity, block.rows()); fillIndexGranularityImpl(index_granularity, 0, granularity, block.rows());
} }
@ -94,10 +97,10 @@ TEST(AdaptiveIndexGranularity, FillGranularitySequenceOfBlocks)
auto block2 = getBlockWithSize(32768, 32); auto block2 = getBlockWithSize(32768, 32);
auto block3 = getBlockWithSize(2048, 32); auto block3 = getBlockWithSize(2048, 32);
EXPECT_EQ(block1.rows() + block2.rows() + block3.rows(), 3136); EXPECT_EQ(block1.rows() + block2.rows() + block3.rows(), 3136);
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
for (const auto & block : {block1, block2, block3}) for (const auto & block : {block1, block2, block3})
{ {
auto granularity = computeIndexGranularityImpl(block, 1024, 8192, false, true); auto granularity = computeIndexGranularity(block.rows(), block.bytes(), 1024, 8192, false, true);
fillIndexGranularityImpl(index_granularity, 0, granularity, block.rows()); fillIndexGranularityImpl(index_granularity, 0, granularity, block.rows());
} }
@ -113,11 +116,11 @@ TEST(AdaptiveIndexGranularity, FillGranularitySequenceOfBlocks)
EXPECT_EQ(block1.rows() + block2.rows() + block3.rows(), (2048 + 4096 + 8192) / 32); EXPECT_EQ(block1.rows() + block2.rows() + block3.rows(), (2048 + 4096 + 8192) / 32);
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
size_t index_offset = 0; size_t index_offset = 0;
for (const auto & block : {block1, block2, block3}) for (const auto & block : {block1, block2, block3})
{ {
auto granularity = computeIndexGranularityImpl(block, 16384, 8192, false, true); auto granularity = computeIndexGranularity(block.rows(), block.bytes(), 16384, 8192, false, true);
fillIndexGranularityImpl(index_granularity, index_offset, granularity, block.rows()); fillIndexGranularityImpl(index_granularity, index_offset, granularity, block.rows());
index_offset = index_granularity.getLastMarkRows() - block.rows(); index_offset = index_granularity.getLastMarkRows() - block.rows();
} }
@ -128,10 +131,10 @@ TEST(AdaptiveIndexGranularity, FillGranularitySequenceOfBlocks)
} }
TEST(AdaptiveIndexGranularity, TestIndexGranularityClass) TEST(AdaptiveIndexGranularity, TestIndexGranularityAdaptive)
{ {
{ {
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
size_t sum_rows = 0; size_t sum_rows = 0;
size_t sum_marks = 0; size_t sum_marks = 0;
for (size_t i = 10; i <= 100; i+=10) for (size_t i = 10; i <= 100; i+=10)
@ -148,11 +151,70 @@ TEST(AdaptiveIndexGranularity, TestIndexGranularityClass)
EXPECT_EQ(index_granularity.getMarkStartingRow(2), 30); EXPECT_EQ(index_granularity.getMarkStartingRow(2), 30);
EXPECT_EQ(index_granularity.getMarkStartingRow(3), 60); EXPECT_EQ(index_granularity.getMarkStartingRow(3), 60);
EXPECT_EQ(index_granularity.getRowsCountInRange({0, 10}), sum_rows); EXPECT_EQ(index_granularity.getRowsCountInRange(0, 10), sum_rows);
EXPECT_EQ(index_granularity.getRowsCountInRange({0, 1}), 10); EXPECT_EQ(index_granularity.getRowsCountInRange(0, 1), 10);
EXPECT_EQ(index_granularity.getRowsCountInRange({2, 5}), 30 + 40 + 50); EXPECT_EQ(index_granularity.getRowsCountInRange(2, 5), 30 + 40 + 50);
EXPECT_EQ(index_granularity.getRowsCountInRanges({{2, 5}, {0, 1}, {0, 10}}), 10 + 30 + 40 + 50 + sum_rows); EXPECT_EQ(index_granularity.getRowsCountInRanges({{2, 5}, {0, 1}, {0, 10}}), 10 + 30 + 40 + 50 + sum_rows);
} }
} }
TEST(AdaptiveIndexGranularity, TestIndexGranularityConstant)
{
auto test = [](MergeTreeIndexGranularity & index_granularity, size_t granularity_rows)
{
size_t sum_marks = 10;
size_t sum_rows = granularity_rows * sum_marks;
for (size_t i = 0; i < 10; ++i)
index_granularity.appendMark(granularity_rows);
size_t new_granularity_rows = granularity_rows / 2;
index_granularity.adjustLastMark(new_granularity_rows);
sum_rows -= (granularity_rows - new_granularity_rows);
index_granularity.appendMark(0);
++sum_marks;
EXPECT_EQ(index_granularity.getMarksCount(), sum_marks);
EXPECT_EQ(index_granularity.getMarksCountWithoutFinal(), sum_marks - 1);
EXPECT_EQ(index_granularity.hasFinalMark(), true);
EXPECT_EQ(index_granularity.getTotalRows(), sum_rows);
EXPECT_EQ(index_granularity.getTotalRows(), sum_rows);
EXPECT_EQ(index_granularity.getLastMarkRows(), 0);
EXPECT_EQ(index_granularity.getLastNonFinalMarkRows(), granularity_rows / 2);
EXPECT_EQ(index_granularity.getMarkStartingRow(0), 0);
EXPECT_EQ(index_granularity.getMarkStartingRow(3), 30);
EXPECT_EQ(index_granularity.getMarkStartingRow(9), 90);
EXPECT_EQ(index_granularity.getMarkStartingRow(10), sum_rows);
EXPECT_EQ(index_granularity.getMarkStartingRow(11), sum_rows);
EXPECT_EQ(index_granularity.getRowsCountInRange(0, 10), sum_rows);
EXPECT_EQ(index_granularity.getRowsCountInRange(0, 11), sum_rows);
EXPECT_EQ(index_granularity.getRowsCountInRange(0, 1), 10);
EXPECT_EQ(index_granularity.getRowsCountInRange(2, 5), 30);
EXPECT_EQ(index_granularity.getRowsCountInRange(3, 9), 60);
EXPECT_EQ(index_granularity.getRowsCountInRange(5, 10), 45);
EXPECT_EQ(index_granularity.getRowsCountInRange(5, 11), 45);
EXPECT_EQ(index_granularity.countMarksForRows(0, 35), 3);
EXPECT_EQ(index_granularity.countMarksForRows(5, 29), 2);
EXPECT_EQ(index_granularity.countMarksForRows(0, 89), 8);
EXPECT_EQ(index_granularity.countMarksForRows(0, 90), 9);
EXPECT_EQ(index_granularity.countMarksForRows(0, 92), 9);
EXPECT_EQ(index_granularity.countMarksForRows(0, 95), sum_marks);
EXPECT_EQ(index_granularity.countMarksForRows(0, 99), sum_marks);
};
const size_t granularity_rows = 10;
{
MergeTreeIndexGranularityConstant index_granularity(granularity_rows);
test(index_granularity, granularity_rows);
}
{
MergeTreeIndexGranularityAdaptive index_granularity;
test(index_granularity, granularity_rows);
}
}

View File

@ -4,6 +4,7 @@
// I know that inclusion of .cpp is not good at all // I know that inclusion of .cpp is not good at all
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp> // NOLINT #include <Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp> // NOLINT
#include <Storages/MergeTree/MergeTreeIndexGranularityAdaptive.h>
using namespace DB; using namespace DB;
@ -13,7 +14,7 @@ TEST(IndexGranularityCompactParts, FillGranularitySequenceOfBlocks)
size_t rows = 8; size_t rows = 8;
size_t granularity = 32; size_t granularity = 32;
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
size_t index_offset = 0; size_t index_offset = 0;
size_t rows_written = 0; size_t rows_written = 0;
for (size_t i = 0; i < 3; ++i) for (size_t i = 0; i < 3; ++i)
@ -34,7 +35,7 @@ TEST(IndexGranularityCompactParts, FillGranularitySequenceOfBlocks)
size_t rows2 = 8; size_t rows2 = 8;
size_t granularity = 32; size_t granularity = 32;
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
size_t index_offset = 0; size_t index_offset = 0;
fillIndexGranularityImpl(index_granularity, index_offset, granularity, rows1); fillIndexGranularityImpl(index_granularity, index_offset, granularity, rows1);
@ -51,7 +52,7 @@ TEST(IndexGranularityCompactParts, FillGranularitySequenceOfBlocks)
size_t rows2 = 25; size_t rows2 = 25;
size_t granularity = 32; size_t granularity = 32;
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
size_t index_offset = 0; size_t index_offset = 0;
fillIndexGranularityImpl(index_granularity, index_offset, granularity, rows1); fillIndexGranularityImpl(index_granularity, index_offset, granularity, rows1);
@ -68,7 +69,7 @@ TEST(IndexGranularityCompactParts, FillGranularitySequenceOfBlocks)
size_t rows = 40; size_t rows = 40;
size_t granularity = 32; size_t granularity = 32;
MergeTreeIndexGranularity index_granularity; MergeTreeIndexGranularityAdaptive index_granularity;
size_t index_offset = 0; size_t index_offset = 0;
for (size_t i = 0; i < 3; ++i) for (size_t i = 0; i < 3; ++i)

View File

@ -981,6 +981,8 @@ class MergeTreeSettingsRandomizer:
"cache_populated_by_fetch": lambda: random.randint(0, 1), "cache_populated_by_fetch": lambda: random.randint(0, 1),
"concurrent_part_removal_threshold": threshold_generator(0.2, 0.3, 0, 100), "concurrent_part_removal_threshold": threshold_generator(0.2, 0.3, 0, 100),
"old_parts_lifetime": threshold_generator(0.2, 0.3, 10, 8 * 60), "old_parts_lifetime": threshold_generator(0.2, 0.3, 10, 8 * 60),
"use_const_adaptive_granularity": lambda: random.randint(0, 1),
"enable_index_granularity_compression": lambda: random.randint(0, 1),
} }
@staticmethod @staticmethod

View File

@ -667,7 +667,7 @@ def test_long_disconnection_stops_backup():
# A backup is expected to fail, but it isn't expected to fail too soon. # A backup is expected to fail, but it isn't expected to fail too soon.
print(f"Backup failed after {time_to_fail} seconds disconnection") print(f"Backup failed after {time_to_fail} seconds disconnection")
assert time_to_fail > 3 assert time_to_fail > 3
assert time_to_fail < 35 assert time_to_fail < 45
# A backup must NOT be stopped if Zookeeper is disconnected shorter than `failure_after_host_disconnected_for_seconds`. # A backup must NOT be stopped if Zookeeper is disconnected shorter than `failure_after_host_disconnected_for_seconds`.

View File

@ -5,11 +5,12 @@
<fill_query>INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000)</fill_query> <fill_query>INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000)</fill_query>
<fill_query>INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000)</fill_query> <fill_query>INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000)</fill_query>
<query tag='INNER'>SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b</query> <query tag='INNER'>SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b settings join_algorithm='hash'</query>
<query tag='INNER'>SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b settings join_algorithm='parallel_hash'</query>
<query tag='LEFT'>SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b</query> <query tag='LEFT'>SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b</query>
<query tag='RIGHT'>SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b</query> <query tag='RIGHT'>SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b</query>
<query tag='FULL'>SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b</query> <query tag='FULL'>SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b</query>
<drop_query>DROP TABLE IF EXISTS test</drop_query> <drop_query>DROP TABLE IF EXISTS test</drop_query>
<drop_query>DROP TABLE IF EXISTS test1</drop_query> <drop_query>DROP TABLE IF EXISTS test1</drop_query>
</test> </test>

View File

@ -9,6 +9,7 @@
<name>settings</name> <name>settings</name>
<values> <values>
<value>join_algorithm='hash'</value> <value>join_algorithm='hash'</value>
<value>join_algorithm='parallel_hash'</value>
<value>join_algorithm='grace_hash'</value> <value>join_algorithm='grace_hash'</value>
</values> </values>
</substitution> </substitution>

View File

@ -1,3 +1,4 @@
<test> <test>
<query tag='INNER'>SELECT count(c) FROM numbers_mt(100000000) AS a INNER JOIN (SELECT number, toString(number) AS c FROM numbers(2000000)) AS b ON (a.number % 10000000) = b.number</query> <query tag='INNER'>SELECT count(c) FROM numbers_mt(100000000) AS a INNER JOIN (SELECT number, toString(number) AS c FROM numbers(2000000)) AS b ON (a.number % 10000000) = b.number settings join_algorithm='hash'</query>
<query tag='INNER'>SELECT count(c) FROM numbers_mt(100000000) AS a INNER JOIN (SELECT number, toString(number) AS c FROM numbers(2000000)) AS b ON (a.number % 10000000) = b.number settings join_algorithm='parallel_hash'</query>
</test> </test>

View File

@ -13,7 +13,8 @@
<query short='1' tag='ANY LEFT IN'>SELECT COUNT() FROM ints l ANY LEFT JOIN ints r USING i64 WHERE i32 IN(42, 10042, 20042, 30042, 40042)</query> <query short='1' tag='ANY LEFT IN'>SELECT COUNT() FROM ints l ANY LEFT JOIN ints r USING i64 WHERE i32 IN(42, 10042, 20042, 30042, 40042)</query>
<query tag='INNER'>SELECT COUNT() FROM ints l INNER JOIN ints r USING i64 WHERE i32 = 20042</query> <query tag='INNER'>SELECT COUNT() FROM ints l INNER JOIN ints r USING i64 WHERE i32 = 20042</query>
<query tag='INNER KEY'>SELECT COUNT() FROM ints l INNER JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 20042 settings query_plan_filter_push_down = 0</query> <query tag='INNER KEY'>SELECT COUNT() FROM ints l INNER JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 20042 settings query_plan_filter_push_down = 0 settings join_algorithm='hash'</query>
<query tag='INNER KEY'>SELECT COUNT() FROM ints l INNER JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 20042 settings query_plan_filter_push_down = 0 settings join_algorithm='parallel_hash'</query>
<query tag='INNER ON'>SELECT COUNT() FROM ints l INNER JOIN ints r ON l.i64 = r.i64 WHERE i32 = 20042</query> <query tag='INNER ON'>SELECT COUNT() FROM ints l INNER JOIN ints r ON l.i64 = r.i64 WHERE i32 = 20042</query>
<query tag='INNER IN'>SELECT COUNT() FROM ints l INNER JOIN ints r USING i64 WHERE i32 IN(42, 10042, 20042, 30042, 40042)</query> <query tag='INNER IN'>SELECT COUNT() FROM ints l INNER JOIN ints r USING i64 WHERE i32 IN(42, 10042, 20042, 30042, 40042)</query>

File diff suppressed because one or more lines are too long

View File

@ -478,6 +478,8 @@ for query_index in queries_to_run:
client_seconds = time.perf_counter() - start_seconds client_seconds = time.perf_counter() - start_seconds
print(f"client-time\t{query_index}\t{client_seconds}\t{server_seconds}") print(f"client-time\t{query_index}\t{client_seconds}\t{server_seconds}")
median = [statistics.median(t) for t in all_server_times]
print(f"median\t{query_index}\t{median[0]}")
# Run additional profiling queries to collect profile data, but only if test times appeared to be different. # Run additional profiling queries to collect profile data, but only if test times appeared to be different.
# We have to do it after normal runs because otherwise it will affect test statistics too much # We have to do it after normal runs because otherwise it will affect test statistics too much
@ -491,7 +493,6 @@ for query_index in queries_to_run:
pvalue = stats.ttest_ind( pvalue = stats.ttest_ind(
all_server_times[0], all_server_times[1], equal_var=False all_server_times[0], all_server_times[1], equal_var=False
).pvalue ).pvalue
median = [statistics.median(t) for t in all_server_times]
# Keep this consistent with the value used in report. Should eventually move # Keep this consistent with the value used in report. Should eventually move
# to (median[1] - median[0]) / min(median), which is compatible with "times" # to (median[1] - median[0]) / min(median), which is compatible with "times"
# difference we use in report (max(median) / min(median)). # difference we use in report (max(median) / min(median)).

View File

@ -9,7 +9,8 @@
<max_threads>1</max_threads> <max_threads>1</max_threads>
</settings> </settings>
<query>SELECT 1 FROM hits_10m_words AS l ANY LEFT JOIN hits_10m_words AS r USING (word) FORMAT Null</query> <query>SELECT 1 FROM hits_10m_words AS l ANY LEFT JOIN hits_10m_words AS r USING (word) FORMAT Null settings join_algorithm='hash'</query>
<query>SELECT 1 FROM hits_10m_words AS l ANY LEFT JOIN hits_10m_words AS r USING (word) FORMAT Null settings join_algorithm='parallel_hash'</query>
<query>SELECT 1 FROM strings AS l ANY LEFT JOIN strings AS r USING (short) FORMAT Null</query> <query>SELECT 1 FROM strings AS l ANY LEFT JOIN strings AS r USING (short) FORMAT Null</query>
<query>SELECT 1 FROM strings AS l ANY LEFT JOIN strings AS r USING (long) FORMAT Null</query> <query>SELECT 1 FROM strings AS l ANY LEFT JOIN strings AS r USING (long) FORMAT Null</query>

View File

@ -1,2 +1,3 @@
SET min_joined_block_size_bytes = 0;
SET max_block_size = 6; SET max_block_size = 6;
SELECT blockSize() bs FROM (SELECT 1 s) js1 ALL RIGHT JOIN (SELECT arrayJoin([2, 2, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3]) s) js2 USING (s) GROUP BY bs ORDER BY bs; SELECT blockSize() bs FROM (SELECT 1 s) js1 ALL RIGHT JOIN (SELECT arrayJoin([2, 2, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3]) s) js2 USING (s) GROUP BY bs ORDER BY bs;

View File

@ -7,6 +7,7 @@ CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id;
INSERT INTO t1 VALUES (1), (2); INSERT INTO t1 VALUES (1), (2);
INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111); INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111);
SET min_joined_block_size_bytes = 0;
SET max_block_size = 100; SET max_block_size = 100;
SELECT count() == 2222 FROM t1 JOIN t2 ON 1 = 1; SELECT count() == 2222 FROM t1 JOIN t2 ON 1 = 1;

View File

@ -8,3 +8,7 @@
1 1
1 1
1 1
1
1
1
1

View File

@ -20,7 +20,9 @@ $CLICKHOUSE_CLIENT -q "
INSERT INTO t2 SELECT number, number FROM numbers_mt(1e6); INSERT INTO t2 SELECT number, number FROM numbers_mt(1e6);
" "
# list of query_id-s that expected to be executed without preallocation
queries_without_preallocation=() queries_without_preallocation=()
# list of query_id-s that expected to be executed with preallocation
queries_with_preallocation=() queries_with_preallocation=()
run_new_query() { run_new_query() {
@ -51,6 +53,9 @@ $CLICKHOUSE_CLIENT "${opts[@]}" --query_id="$query_id" -q "SELECT * FROM t1 AS x
# now t1 is the right table # now t1 is the right table
run_new_query "SELECT * FROM t2 AS x INNER JOIN t1 AS y ON x.a = y.a" run_new_query "SELECT * FROM t2 AS x INNER JOIN t1 AS y ON x.a = y.a"
run_new_query "SELECT * FROM t1 AS x INNER JOIN t2 AS y ON x.a = y.a WHERE a < 200_000"
run_new_query "SELECT * FROM t1 AS x INNER JOIN t2 AS y ON x.a = y.a WHERE a >= 200_000"
################################## ##################################
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"

View File

@ -274,18 +274,21 @@ select parseDateTime('08 13, 2022, 07:58:32', '%c %e, %G, %k:%i:%s', 'UTC');
set session_timezone = 'UTC'; -- don't randomize the session timezone set session_timezone = 'UTC'; -- don't randomize the session timezone
select parseDateTime('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34'); select parseDateTime('2021-01-04 23:12:34') = toDateTime('2021-01-04 23:12:34');
1 1
-- The following is test of parseDateTime64 select parseDateTime(''); -- { serverError NOT_ENOUGH_SPACE }
-- -------------------------------------------------------------------------------------------------------------------------
-- Tests for parseDateTime64, these are not systematic
select parseDateTime64(''); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime64(''); -- { serverError NOT_ENOUGH_SPACE }
select parseDateTime64('2021-01-04 23:12:34.118'); -- { serverError NOT_ENOUGH_SPACE }
select parseDateTime64('2177-10-09 10:30:10.123'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2177-10-09 10:30:10.123'); -- { serverError CANNOT_PARSE_DATETIME }
select parseDateTime64('2021-01-04 23:12:34.118112') = toDateTime64('2021-01-04 23:12:34.118112', 6); select parseDateTime64('2021-01-04 23:12:34.118112') = toDateTime64('2021-01-04 23:12:34.118112', 6);
1 1
select parseDateTime64('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2021-01-04 23:12:34.118112', 6); select parseDateTime64('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2021-01-04 23:12:34.118112', 6);
1 1
select parseDateTime64('2021-01-04 23:12:34.118'); -- { serverError NOT_ENOUGH_SPACE }
select parseDateTime64('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime64('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError NOT_ENOUGH_SPACE }
select parseDateTime64('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME }
select parseDateTime64('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s'); -- { serverError CANNOT_PARSE_DATETIME }
--leap years and non-leap years -- leap vs non-leap years
select parseDateTime64('2024-02-29 11:23:34.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2024-02-29 11:23:34.123433', 6); select parseDateTime64('2024-02-29 11:23:34.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2024-02-29 11:23:34.123433', 6);
1 1
select parseDateTime64('2023-02-29 11:22:33.123433', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime64('2023-02-29 11:22:33.123433', '%Y-%m-%d %H:%i:%s.%f'); -- { serverError CANNOT_PARSE_DATETIME }
@ -293,7 +296,7 @@ select parseDateTime64('2024-02-28 23:22:33.123433', '%Y-%m-%d %H:%i:%s.%f') = t
1 1
select parseDateTime64('2023-02-28 23:22:33.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2023-02-28 23:22:33.123433', 6); select parseDateTime64('2023-02-28 23:22:33.123433', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('2023-02-28 23:22:33.123433', 6);
1 1
-- Test of parseDateTime64OrNull -- parseDateTime64OrNull
select parseDateTime64OrNull('2021-01-04 23:12:34.118') IS NULL; select parseDateTime64OrNull('2021-01-04 23:12:34.118') IS NULL;
1 1
select parseDateTime64OrNull('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f') IS NULL; select parseDateTime64OrNull('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f') IS NULL;
@ -302,7 +305,7 @@ select parseDateTime64OrNull('2021-01-04 23:12:34.118112', '%Y-%m-%d %H:%i:%s')
1 1
select parseDateTime64OrNull('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f') IS NULL; select parseDateTime64OrNull('2021-01-04 23:12:34.11811235', '%Y-%m-%d %H:%i:%s.%f') IS NULL;
1 1
-- Test of parseDateTime64OrZero -- parseDateTime64OrZero
select parseDateTime64OrZero('2021-01-04 23:12:34.118') = toDateTime64('1970-01-01 00:00:00', 6); select parseDateTime64OrZero('2021-01-04 23:12:34.118') = toDateTime64('1970-01-01 00:00:00', 6);
1 1
select parseDateTime64OrZero('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('1970-01-01 00:00:00', 6); select parseDateTime64OrZero('2021-01-04 23:12:34.118', '%Y-%m-%d %H:%i:%s.%f') = toDateTime64('1970-01-01 00:00:00', 6);

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