Merge remote-tracking branch 'upstream/master' into client-complete-error-codes

This commit is contained in:
Azat Khuzhin 2021-02-02 00:14:53 +03:00
commit 37797fdf5b
185 changed files with 4158 additions and 1278 deletions

View File

@ -116,8 +116,8 @@ void Connection::connect(const char* db,
if (!mysql_real_connect(driver.get(), server, user, password, db, port, ifNotEmpty(socket), driver->client_flag))
throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get()));
/// Sets UTF-8 as default encoding.
if (mysql_set_character_set(driver.get(), "UTF8"))
/// Sets UTF-8 as default encoding. See https://mariadb.com/kb/en/mysql_set_character_set/
if (mysql_set_character_set(driver.get(), "utf8mb4"))
throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get()));
is_connected = true;

View File

@ -1,9 +1,9 @@
# This strings autochanged from release_lib.sh:
SET(VERSION_REVISION 54447)
SET(VERSION_REVISION 54448)
SET(VERSION_MAJOR 21)
SET(VERSION_MINOR 2)
SET(VERSION_MINOR 3)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH 53d0c9fa7255aa1dc48991d19f4246ff71cc2fd7)
SET(VERSION_DESCRIBE v21.2.1.1-prestable)
SET(VERSION_STRING 21.2.1.1)
SET(VERSION_GITHASH ef72ba7349f230321750c13ee63b49a11a7c0adc)
SET(VERSION_DESCRIBE v21.3.1.1-prestable)
SET(VERSION_STRING 21.3.1.1)
# end of autochange

2
contrib/hyperscan vendored

@ -1 +1 @@
Subproject commit 3907fd00ee8b2538739768fa9533f8635a276531
Subproject commit e9f08df0213fc637aac0a5bbde9beeaeba2fe9fa

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit 2c32e17c7dfee1f8bf24227b697cdef5fddf0823
Subproject commit e11f3c971570cf6a31006cd21cadf41a259c360a

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (21.2.1.1) unstable; urgency=low
clickhouse (21.3.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Mon, 11 Jan 2021 11:12:08 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Mon, 01 Feb 2021 12:50:53 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.2.1.*
ARG version=21.3.1.*
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \

View File

@ -1,7 +1,7 @@
FROM ubuntu:20.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.2.1.*
ARG version=21.3.1.*
ARG gosu_ver=1.10
# user/group precreated explicitly with fixed uid/gid on purpose.

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.2.1.*
ARG version=21.3.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -251,8 +251,12 @@ function run_tests
00701_rollup
00834_cancel_http_readonly_queries_on_client_close
00911_tautological_compare
# Hyperscan
00926_multimatch
00929_multi_match_edit_distance
01681_hyperscan_debug_assertion
01031_mutations_interpreter_and_context
01053_ssd_dictionary # this test mistakenly requires acces to /var/lib/clickhouse -- can't run this locally, disabled
01083_expressions_in_engine_arguments

View File

@ -1,12 +1,16 @@
# docker build -t yandex/clickhouse-style-test .
FROM ubuntu:20.04
RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes shellcheck libxml2-utils git python3-pip python3-pytest && pip3 install codespell
RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes shellcheck libxml2-utils git python3-pip pylint && pip3 install codespell
# For |& syntax
SHELL ["bash", "-c"]
CMD cd /ClickHouse/utils/check-style && \
./check-style -n | tee /test_output/style_output.txt && \
./check-typos | tee /test_output/typos_output.txt && \
./check-whitespaces -n | tee /test_output/whitespaces_output.txt && \
./check-duplicate-includes.sh | tee /test_output/duplicate_output.txt && \
./shellcheck-run.sh | tee /test_output/shellcheck_output.txt
./check-style -n |& tee /test_output/style_output.txt && \
./check-typos |& tee /test_output/typos_output.txt && \
./check-whitespaces -n |& tee /test_output/whitespaces_output.txt && \
./check-duplicate-includes.sh |& tee /test_output/duplicate_output.txt && \
./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt && \
true

View File

@ -40,7 +40,7 @@ $ cd ClickHouse
``` bash
$ mkdir build
$ cd build
$ cmake ..-DCMAKE_C_COMPILER=`brew --prefix llvm`/bin/clang -DCMAKE_CXX_COMPILER=`brew --prefix llvm`/bin/clang++ -DCMAKE_PREFIX_PATH=`brew --prefix llvm`
$ cmake .. -DCMAKE_C_COMPILER=`brew --prefix llvm`/bin/clang -DCMAKE_CXX_COMPILER=`brew --prefix llvm`/bin/clang++ -DCMAKE_PREFIX_PATH=`brew --prefix llvm`
$ ninja
$ cd ..
```

View File

@ -1944,6 +1944,21 @@ Possible values:
Default value: 16.
## background_message_broker_schedule_pool_size {#background_message_broker_schedule_pool_size}
Sets the number of threads performing background tasks for message streaming. This setting is applied at the ClickHouse server start and cant be changed in a user session.
Possible values:
- Any positive integer.
Default value: 16.
**See Also**
- [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) engine
- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine) engine
## validate_polygons {#validate_polygons}
Enables or disables throwing an exception in the [pointInPolygon](../../sql-reference/functions/geo/index.md#pointinpolygon) function, if the polygon is self-intersecting or self-tangent.

View File

@ -1,22 +1,21 @@
# system.distributed_ddl_queue {#system_tables-distributed_ddl_queue}
Contains information about distributed ddl queries (ON CLUSTER queries) that were executed on a cluster.
Contains information about [distributed ddl queries (ON CLUSTER clause)](../../sql-reference/distributed-ddl.md) that were executed on a cluster.
Columns:
- `entry` ([String](../../sql-reference/data-types/string.md)) - Query id.
- `host_name` ([String](../../sql-reference/data-types/string.md)) - Hostname.
- `host_address` ([String](../../sql-reference/data-types/string.md)) - IP address that the Hostname resolves to.
- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) - Host Port.
- `status` ([Enum](../../sql-reference/data-types/enum.md)) - Stats of the query.
- `cluster` ([String](../../sql-reference/data-types/string.md)) - Cluster name.
- `query` ([String](../../sql-reference/data-types/string.md)) - Query executed.
- `initiator` ([String](../../sql-reference/data-types/string.md)) - Nod that executed the query.
- `query_start_time` ([Date](../../sql-reference/data-types/date.md)) — Query start time.
- `query_finish_time` ([Date](../../sql-reference/data-types/date.md)) — Query finish time.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/datetime64.md)) — Duration of query execution in milliseconds.
- `exception_code` ([Enum](../../sql-reference/data-types/enum.md)) - Exception code from ZooKeeper.
- `entry` ([String](../../sql-reference/data-types/string.md)) — Query id.
- `host_name` ([String](../../sql-reference/data-types/string.md)) — Hostname.
- `host_address` ([String](../../sql-reference/data-types/string.md)) — IP address that the Hostname resolves to.
- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — Host Port.
- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — Status of the query.
- `cluster` ([String](../../sql-reference/data-types/string.md)) — Cluster name.
- `query` ([String](../../sql-reference/data-types/string.md)) — Query executed.
- `initiator` ([String](../../sql-reference/data-types/string.md)) — Node that executed the query.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query start time.
- `query_finish_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query finish time.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/datetime64.md)) — Duration of query execution (in milliseconds).
- `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — Exception code from [ZooKeeper](../../operations/tips.md#zookeeper).
**Example**
@ -62,6 +61,5 @@ exception_code: ZOK
2 rows in set. Elapsed: 0.025 sec.
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/distributed_ddl_queuedistributed_ddl_queue.md) <!--hide-->

View File

@ -115,9 +115,20 @@ LIMIT 10
## IPv6StringToNum(s) {#ipv6stringtonums}
The reverse function of IPv6NumToString. If the IPv6 address has an invalid format, it returns a string of null bytes.
The reverse function of IPv6NumToString. If the IPv6 address has an invalid format, it returns a string of null bytes.
If the IP address is a valid IPv4 address then the IPv6 equivalent of the IPv4 address is returned.
HEX can be uppercase or lowercase.
``` sql
SELECT cutIPv6(IPv6StringToNum('127.0.0.1'), 0, 0);
```
``` text
┌─cutIPv6(IPv6StringToNum('127.0.0.1'), 0, 0)─┐
│ ::ffff:127.0.0.1 │
└─────────────────────────────────────────────┘
```
## IPv4ToIPv6(x) {#ipv4toipv6x}
Takes a `UInt32` number. Interprets it as an IPv4 address in [big endian](https://en.wikipedia.org/wiki/Endianness). Returns a `FixedString(16)` value containing the IPv6 address in binary format. Examples:
@ -214,6 +225,7 @@ SELECT
## toIPv6(string) {#toipv6string}
An alias to `IPv6StringToNum()` that takes a string form of IPv6 address and returns value of [IPv6](../../sql-reference/data-types/domains/ipv6.md) type, which is binary equal to value returned by `IPv6StringToNum()`.
If the IP address is a valid IPv4 address then the IPv6 equivalent of the IPv4 address is returned.
``` sql
WITH
@ -243,6 +255,15 @@ SELECT
└───────────────────────────────────┴──────────────────────────────────┘
```
``` sql
SELECT toIPv6('127.0.0.1')
```
``` text
┌─toIPv6('127.0.0.1')─┐
│ ::ffff:127.0.0.1 │
└─────────────────────┘
```
## isIPv4String

View File

@ -13,7 +13,7 @@ Basic query format:
INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ...
```
You can specify a list of columns to insert using the `(c1, c2, c3)`. You can also use an expression with column [matcher](../../sql-reference/statements/select/index.md#asterisk) such as `*` and/or [modifiers](../../sql-reference/statements/select/index.md#select-modifiers) such as [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#apply-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier).
You can specify a list of columns to insert using the `(c1, c2, c3)`. You can also use an expression with column [matcher](../../sql-reference/statements/select/index.md#asterisk) such as `*` and/or [modifiers](../../sql-reference/statements/select/index.md#select-modifiers) such as [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier).
For example, consider the table:

View File

@ -278,5 +278,4 @@ Other ways to make settings see [here](../../../operations/settings/index.md).
SELECT * FROM some_table SETTINGS optimize_read_in_order=1, cast_keep_nullable=1;
```
[Original article](https://clickhouse.tech/docs/en/sql-reference/statements/select/)
<!--hide-->
[Original article](https://clickhouse.tech/docs/en/sql-reference/statements/select/)<!--hide-->

View File

@ -0,0 +1,65 @@
# system.distributed_ddl_queue {#system_tables-distributed_ddl_queue}
Содержит информацию о [распределенных ddl запросах (секция ON CLUSTER)](../../sql-reference/distributed-ddl.md), которые были выполнены на кластере.
Столбцы:
- `entry` ([String](../../sql-reference/data-types/string.md)) — идентификатор запроса.
- `host_name` ([String](../../sql-reference/data-types/string.md)) — имя хоста.
- `host_address` ([String](../../sql-reference/data-types/string.md)) — IP-адрес хоста.
- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — порт для соединения с сервером.
- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — состояние запроса.
- `cluster` ([String](../../sql-reference/data-types/string.md)) — имя кластера.
- `query` ([String](../../sql-reference/data-types/string.md)) — выполненный запрос.
- `initiator` ([String](../../sql-reference/data-types/string.md)) — узел, выполнивший запрос.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время начала запроса.
- `query_finish_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время окончания запроса.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/datetime64.md)) — продолжительность выполнения запроса (в миллисекундах).
- `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — код исключения из [ZooKeeper](../../operations/tips.md#zookeeper).
**Пример**
``` sql
SELECT *
FROM system.distributed_ddl_queue
WHERE cluster = 'test_cluster'
LIMIT 2
FORMAT Vertical
Query id: f544e72a-6641-43f1-836b-24baa1c9632a
Row 1:
──────
entry: query-0000000000
host_name: clickhouse01
host_address: 172.23.0.11
port: 9000
status: Finished
cluster: test_cluster
query: CREATE DATABASE test_db UUID '4a82697e-c85e-4e5b-a01e-a36f2a758456' ON CLUSTER test_cluster
initiator: clickhouse01:9000
query_start_time: 2020-12-30 13:07:51
query_finish_time: 2020-12-30 13:07:51
query_duration_ms: 6
exception_code: ZOK
Row 2:
──────
entry: query-0000000000
host_name: clickhouse02
host_address: 172.23.0.12
port: 9000
status: Finished
cluster: test_cluster
query: CREATE DATABASE test_db UUID '4a82697e-c85e-4e5b-a01e-a36f2a758456' ON CLUSTER test_cluster
initiator: clickhouse01:9000
query_start_time: 2020-12-30 13:07:51
query_finish_time: 2020-12-30 13:07:51
query_duration_ms: 6
exception_code: ZOK
2 rows in set. Elapsed: 0.025 sec.
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/distributed_ddl_queuedistributed_ddl_queue.md) <!--hide-->

View File

@ -13,7 +13,7 @@ toc_title: INSERT INTO
INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ...
```
Вы можете указать список столбцов для вставки, используя синтаксис `(c1, c2, c3)`. Также можно использовать выражение cо [звездочкой](../../sql-reference/statements/select/index.md#asterisk) и/или модификаторами, такими как `APPLY`, `EXCEPT`, `REPLACE`.
Вы можете указать список столбцов для вставки, используя синтаксис `(c1, c2, c3)`. Также можно использовать выражение cо [звездочкой](../../sql-reference/statements/select/index.md#asterisk) и/или модификаторами, такими как [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier).
В качестве примера рассмотрим таблицу:

View File

@ -162,6 +162,112 @@ Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of
Подробнее смотрите в разделе «Настройки». Присутствует возможность использовать внешнюю сортировку (с сохранением временных данных на диск) и внешнюю агрегацию.
## Модификаторы запроса SELECT {#select-modifiers}
Вы можете использовать следующие модификаторы в запросах `SELECT`.
### APPLY {#apply-modifier}
Вызывает указанную функцию для каждой строки, возвращаемой внешним табличным выражением запроса.
**Синтаксис:**
``` sql
SELECT <expr> APPLY( <func> ) FROM [db.]table_name
```
**Пример:**
``` sql
CREATE TABLE columns_transformers (i Int64, j Int16, k Int64) ENGINE = MergeTree ORDER by (i);
INSERT INTO columns_transformers VALUES (100, 10, 324), (120, 8, 23);
SELECT * APPLY(sum) FROM columns_transformers;
```
```
┌─sum(i)─┬─sum(j)─┬─sum(k)─┐
│ 220 │ 18 │ 347 │
└────────┴────────┴────────┘
```
### EXCEPT {#except-modifier}
Исключает из результата запроса один или несколько столбцов.
**Синтаксис:**
``` sql
SELECT <expr> EXCEPT ( col_name1 [, col_name2, col_name3, ...] ) FROM [db.]table_name
```
**Пример:**
``` sql
SELECT * EXCEPT (i) from columns_transformers;
```
```
┌──j─┬───k─┐
│ 10 │ 324 │
│ 8 │ 23 │
└────┴─────┘
```
### REPLACE {#replace-modifier}
Определяет одно или несколько [выражений алиасов](../../../sql-reference/syntax.md#syntax-expression_aliases). Каждый алиас должен соответствовать имени столбца из запроса `SELECT *`. В списке столбцов результата запроса имя столбца, соответствующее алиасу, заменяется выражением в модификаторе `REPLACE`.
Этот модификатор не изменяет имена или порядок столбцов. Однако он может изменить значение и тип значения.
**Синтаксис:**
``` sql
SELECT <expr> REPLACE( <expr> AS col_name) from [db.]table_name
```
**Пример:**
``` sql
SELECT * REPLACE(i + 1 AS i) from columns_transformers;
```
```
┌───i─┬──j─┬───k─┐
│ 101 │ 10 │ 324 │
│ 121 │ 8 │ 23 │
└─────┴────┴─────┘
```
### Комбинации модификаторов {#modifier-combinations}
Вы можете использовать каждый модификатор отдельно или комбинировать их.
**Примеры:**
Использование одного и того же модификатора несколько раз.
``` sql
SELECT COLUMNS('[jk]') APPLY(toString) APPLY(length) APPLY(max) from columns_transformers;
```
```
┌─max(length(toString(j)))─┬─max(length(toString(k)))─┐
│ 2 │ 3 │
└──────────────────────────┴──────────────────────────┘
```
Использование нескольких модификаторов в одном запросе.
``` sql
SELECT * REPLACE(i + 1 AS i) EXCEPT (j) APPLY(sum) from columns_transformers;
```
```
┌─sum(plus(i, 1))─┬─sum(k)─┐
│ 222 │ 347 │
└─────────────────┴────────┘
```
## SETTINGS в запросе SELECT {#settings-in-select}
Вы можете задать значения необходимых настроек непосредственно в запросе `SELECT` в секции `SETTINGS`. Эти настройки действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию.
@ -174,5 +280,4 @@ Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of
SELECT * FROM some_table SETTINGS optimize_read_in_order=1, cast_keep_nullable=1;
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/select/)
<!--hide-->
[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/select/)<!--hide-->

View File

@ -62,12 +62,12 @@ public:
bool randomize_, size_t max_iterations_, double max_time_,
const String & json_path_, size_t confidence_,
const String & query_id_, const String & query_to_execute_, bool continue_on_errors_,
bool print_stacktrace_, const Settings & settings_)
bool reconnect_, bool print_stacktrace_, const Settings & settings_)
:
concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_),
cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_),
json_path(json_path_), confidence(confidence_), query_id(query_id_),
query_to_execute(query_to_execute_), continue_on_errors(continue_on_errors_),
query_to_execute(query_to_execute_), continue_on_errors(continue_on_errors_), reconnect(reconnect_),
print_stacktrace(print_stacktrace_), settings(settings_),
shared_context(Context::createShared()), global_context(Context::createGlobal(shared_context.get())),
pool(concurrency)
@ -155,6 +155,7 @@ private:
String query_id;
String query_to_execute;
bool continue_on_errors;
bool reconnect;
bool print_stacktrace;
const Settings & settings;
SharedContextHolder shared_context;
@ -404,9 +405,14 @@ private:
void execute(EntryPtrs & connection_entries, Query & query, size_t connection_index)
{
Stopwatch watch;
Connection & connection = **connection_entries[connection_index];
if (reconnect)
connection.disconnect();
RemoteBlockInputStream stream(
*(*connection_entries[connection_index]),
query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage);
connection, query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage);
if (!query_id.empty())
stream.setQueryId(query_id);
@ -589,6 +595,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
("confidence", value<size_t>()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)")
("query_id", value<std::string>()->default_value(""), "")
("continue_on_errors", "continue testing even if a query fails")
("reconnect", "establish new connection for every query")
;
Settings settings;
@ -638,7 +645,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
options["confidence"].as<size_t>(),
options["query_id"].as<std::string>(),
options["query"].as<std::string>(),
options.count("continue_on_errors") > 0,
options.count("continue_on_errors"),
options.count("reconnect"),
print_stacktrace,
settings);
return benchmark.run();

View File

@ -513,7 +513,7 @@ private:
}
protected:
void extractColumns(const IColumn ** columns, const IColumn ** aggr_columns) const
ssize_t extractColumns(const IColumn ** columns, const IColumn ** aggr_columns, ssize_t if_argument_pos) const
{
if (tuple_argument)
{
@ -526,6 +526,13 @@ protected:
for (size_t i = 0; i < args_count; ++i)
columns[i] = aggr_columns[i];
}
if (if_argument_pos >= 0)
{
columns[args_count] = aggr_columns[if_argument_pos];
return args_count;
}
else
return -1;
}
bool tuple_argument;
@ -551,8 +558,8 @@ public:
Arena * arena,
ssize_t if_argument_pos = -1) const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
const IColumn * ex_columns[args_count + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
Base::addBatch(batch_size, places, place_offset, ex_columns, arena, if_argument_pos);
}
@ -560,8 +567,8 @@ public:
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
const IColumn * ex_columns[args_count + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
Base::addBatchSinglePlace(batch_size, place, ex_columns, arena, if_argument_pos);
}
@ -574,8 +581,8 @@ public:
Arena * arena,
ssize_t if_argument_pos = -1) const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
const IColumn * ex_columns[args_count + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
Base::addBatchSinglePlaceNotNull(batch_size, place, ex_columns, null_map, arena, if_argument_pos);
}
@ -584,8 +591,8 @@ public:
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1)
const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
const IColumn * ex_columns[args_count + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
Base::addBatchSinglePlaceFromInterval(batch_begin, batch_end, place, ex_columns, arena, if_argument_pos);
}
@ -595,7 +602,7 @@ public:
const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
extractColumns(ex_columns, columns, -1);
Base::addBatchArray(batch_size, places, place_offset, ex_columns, offsets, arena);
}
@ -610,7 +617,7 @@ public:
Arena * arena) const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
extractColumns(ex_columns, columns, -1);
Base::addBatchLookupTable8(batch_size, map, place_offset, init, key, ex_columns, arena);
}

View File

@ -7,8 +7,10 @@
#include <atomic>
#include <Poco/Net/StreamSocket.h>
#include <Common/Exception.h>
#include <Common/Stopwatch.h>
#include <Common/ShellCommand.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/copyData.h>
/** In a loop it connects to the server and immediately breaks the connection.
@ -18,22 +20,26 @@
int main(int argc, char ** argv)
try
{
using namespace DB;
size_t num_iterations = 1;
size_t num_threads = 1;
std::string host = "localhost";
uint16_t port = 9000;
if (argc >= 2)
num_iterations = DB::parse<size_t>(argv[1]);
num_iterations = parse<size_t>(argv[1]);
if (argc >= 3)
num_threads = DB::parse<size_t>(argv[2]);
num_threads = parse<size_t>(argv[2]);
if (argc >= 4)
host = argv[3];
if (argc >= 5)
port = DB::parse<uint16_t>(argv[4]);
port = parse<uint16_t>(argv[4]);
WriteBufferFromFileDescriptor out(STDERR_FILENO);
std::atomic_bool cancel{false};
std::vector<std::thread> threads(num_threads);
@ -45,44 +51,32 @@ try
{
std::cerr << ".";
Poco::Net::SocketAddress address(host, port);
int fd = socket(PF_INET, SOCK_STREAM, IPPROTO_IP);
if (fd < 0)
DB::throwFromErrno("Cannot create socket", 0);
linger linger_value;
linger_value.l_onoff = 1;
linger_value.l_linger = 0;
if (0 != setsockopt(fd, SOL_SOCKET, SO_LINGER, &linger_value, sizeof(linger_value)))
DB::throwFromErrno("Cannot set linger", 0);
try
{
Stopwatch watch;
Poco::Net::SocketAddress address(host, port);
Poco::Net::StreamSocket socket;
//socket.setLinger(1, 0);
int res = connect(fd, address.addr(), address.length());
if (res != 0 && errno != EINPROGRESS && errno != EWOULDBLOCK)
socket.connectNB(address);
if (!socket.poll(Poco::Timespan(1000000),
Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_WRITE | Poco::Net::Socket::SELECT_ERROR))
{
close(fd);
DB::throwFromErrno("Cannot connect", 0);
}
/// Allow to debug the server.
/* auto command = ShellCommand::execute("kill -STOP $(pidof clickhouse-server)");
copyData(command->err, out);
copyData(command->out, out);
command->wait();*/
close(fd);
if (watch.elapsedSeconds() > 0.1)
{
std::cerr << watch.elapsedSeconds() << "\n";
cancel = true;
break;
std::cerr << "Timeout\n";
/* cancel = true;
break;*/
}
}
catch (const Poco::Exception & e)
{
std::cerr << e.displayText() << "\n";
cancel = true;
break;
}
}
});

View File

@ -119,6 +119,13 @@ void tryLogCurrentException(const char * log_name, const std::string & start_of_
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message)
{
/// Under high memory pressure, any new allocation will definitelly lead
/// to MEMORY_LIMIT_EXCEEDED exception.
///
/// And in this case the exception will not be logged, so let's block the
/// MemoryTracker until the exception will be logged.
MemoryTracker::LockExceptionInThread lock_memory_tracker;
try
{
if (start_of_message.empty())

View File

@ -53,7 +53,7 @@ public:
if constexpr (std::is_arithmetic_v<U>)
{
ReadBufferFromString in(l);
T parsed;
U parsed;
readText(parsed, in);
return operator()(parsed, r);
}
@ -113,7 +113,7 @@ public:
if constexpr (std::is_arithmetic_v<U>)
{
ReadBufferFromString in(l);
T parsed;
U parsed;
readText(parsed, in);
return operator()(parsed, r);
}

View File

@ -98,14 +98,31 @@ public:
}
else
{
const auto first_u32 = UTF8::convert(needle);
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
auto first_u32 = UTF8::convertUTF8ToCodePoint(needle, needle_size);
/// Invalid UTF-8
if (!first_u32)
{
/// Process it verbatim as a sequence of bytes.
size_t src_len = UTF8::seqLength(*needle);
memcpy(l_seq, needle, src_len);
memcpy(u_seq, needle, src_len);
}
else
{
uint32_t first_l_u32 = Poco::Unicode::toLower(*first_u32);
uint32_t first_u_u32 = Poco::Unicode::toUpper(*first_u32);
/// lower and uppercase variants of the first octet of the first character in `needle`
size_t length_l = UTF8::convertCodePointToUTF8(first_l_u32, l_seq, sizeof(l_seq));
size_t length_r = UTF8::convertCodePointToUTF8(first_u_u32, u_seq, sizeof(u_seq));
if (length_l != length_r)
throw Exception{"UTF8 sequences with different lowercase and uppercase lengths are not supported", ErrorCodes::UNSUPPORTED_PARAMETER};
}
/// lower and uppercase variants of the first octet of the first character in `needle`
UTF8::convert(first_l_u32, l_seq, sizeof(l_seq));
l = l_seq[0];
UTF8::convert(first_u_u32, u_seq, sizeof(u_seq));
u = u_seq[0];
}
@ -128,18 +145,21 @@ public:
continue;
}
const auto src_len = UTF8::seqLength(*needle_pos);
const auto c_u32 = UTF8::convert(needle_pos);
size_t src_len = std::min<size_t>(needle_end - needle_pos, UTF8::seqLength(*needle_pos));
auto c_u32 = UTF8::convertUTF8ToCodePoint(needle_pos, src_len);
const auto c_l_u32 = Poco::Unicode::toLower(c_u32);
const auto c_u_u32 = Poco::Unicode::toUpper(c_u32);
if (c_u32)
{
int c_l_u32 = Poco::Unicode::toLower(*c_u32);
int c_u_u32 = Poco::Unicode::toUpper(*c_u32);
const auto dst_l_len = static_cast<uint8_t>(UTF8::convert(c_l_u32, l_seq, sizeof(l_seq)));
const auto dst_u_len = static_cast<uint8_t>(UTF8::convert(c_u_u32, u_seq, sizeof(u_seq)));
uint8_t dst_l_len = static_cast<uint8_t>(UTF8::convertCodePointToUTF8(c_l_u32, l_seq, sizeof(l_seq)));
uint8_t dst_u_len = static_cast<uint8_t>(UTF8::convertCodePointToUTF8(c_u_u32, u_seq, sizeof(u_seq)));
/// @note Unicode standard states it is a rare but possible occasion
if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
throw Exception{"UTF8 sequences with different lowercase and uppercase lengths are not supported", ErrorCodes::UNSUPPORTED_PARAMETER};
/// @note Unicode standard states it is a rare but possible occasion
if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
throw Exception{"UTF8 sequences with different lowercase and uppercase lengths are not supported", ErrorCodes::UNSUPPORTED_PARAMETER};
}
cache_actual_len += src_len;
if (cache_actual_len < n)
@ -164,7 +184,7 @@ public:
}
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
ALWAYS_INLINE bool compare(const CharT * /*haystack*/, const CharT * /*haystack_end*/, const CharT * pos) const
ALWAYS_INLINE bool compare(const CharT * /*haystack*/, const CharT * haystack_end, const CharT * pos) const
{
#ifdef __SSE4_1__
@ -183,11 +203,20 @@ public:
pos += cache_valid_len;
auto needle_pos = needle + cache_valid_len;
while (needle_pos < needle_end &&
Poco::Unicode::toLower(UTF8::convert(pos)) ==
Poco::Unicode::toLower(UTF8::convert(needle_pos)))
while (needle_pos < needle_end)
{
/// @note assuming sequences for lowercase and uppercase have exact same length
auto haystack_code_point = UTF8::convertUTF8ToCodePoint(pos, haystack_end - pos);
auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
/// Invalid UTF-8, should not compare equals
if (!haystack_code_point || !needle_code_point)
break;
/// Not equals case insensitive.
if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
break;
/// @note assuming sequences for lowercase and uppercase have exact same length (that is not always true)
const auto len = UTF8::seqLength(*pos);
pos += len;
needle_pos += len;
@ -209,10 +238,19 @@ public:
pos += first_needle_symbol_is_ascii;
auto needle_pos = needle + first_needle_symbol_is_ascii;
while (needle_pos < needle_end &&
Poco::Unicode::toLower(UTF8::convert(pos)) ==
Poco::Unicode::toLower(UTF8::convert(needle_pos)))
while (needle_pos < needle_end)
{
auto haystack_code_point = UTF8::convertUTF8ToCodePoint(pos, haystack_end - pos);
auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
/// Invalid UTF-8, should not compare equals
if (!haystack_code_point || !needle_code_point)
break;
/// Not equals case insensitive.
if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
break;
const auto len = UTF8::seqLength(*pos);
pos += len;
needle_pos += len;
@ -270,11 +308,20 @@ public:
auto haystack_pos = haystack + cache_valid_len;
auto needle_pos = needle + cache_valid_len;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
Poco::Unicode::toLower(UTF8::convert(haystack_pos)) ==
Poco::Unicode::toLower(UTF8::convert(needle_pos)))
while (haystack_pos < haystack_end && needle_pos < needle_end)
{
/// @note assuming sequences for lowercase and uppercase have exact same length
auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos);
auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
/// Invalid UTF-8, should not compare equals
if (!haystack_code_point || !needle_code_point)
break;
/// Not equals case insensitive.
if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
break;
/// @note assuming sequences for lowercase and uppercase have exact same length (that is not always true)
const auto len = UTF8::seqLength(*haystack_pos);
haystack_pos += len;
needle_pos += len;
@ -302,10 +349,19 @@ public:
auto haystack_pos = haystack + first_needle_symbol_is_ascii;
auto needle_pos = needle + first_needle_symbol_is_ascii;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
Poco::Unicode::toLower(UTF8::convert(haystack_pos)) ==
Poco::Unicode::toLower(UTF8::convert(needle_pos)))
while (haystack_pos < haystack_end && needle_pos < needle_end)
{
auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos);
auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
/// Invalid UTF-8, should not compare equals
if (!haystack_code_point || !needle_code_point)
break;
/// Not equals case insensitive.
if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
break;
const auto len = UTF8::seqLength(*haystack_pos);
haystack_pos += len;
needle_pos += len;

View File

@ -1,5 +1,6 @@
#pragma once
#include <optional>
#include <common/types.h>
#include <Common/BitHelpers.h>
#include <Poco/UTF8Encoding.h>
@ -73,26 +74,27 @@ inline size_t countCodePoints(const UInt8 * data, size_t size)
return res;
}
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
int convert(const CharT * bytes)
size_t convertCodePointToUTF8(uint32_t code_point, CharT * out_bytes, size_t out_length)
{
static const Poco::UTF8Encoding utf8;
return utf8.convert(reinterpret_cast<const uint8_t *>(bytes));
int res = utf8.convert(code_point, reinterpret_cast<uint8_t *>(out_bytes), out_length);
assert(res >= 0);
return res;
}
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
int convert(int ch, CharT * bytes, int length)
std::optional<uint32_t> convertUTF8ToCodePoint(const CharT * in_bytes, size_t in_length)
{
static const Poco::UTF8Encoding utf8;
return utf8.convert(ch, reinterpret_cast<uint8_t *>(bytes), length);
int res = utf8.queryConvert(reinterpret_cast<const uint8_t *>(in_bytes), in_length);
if (res >= 0)
return res;
return {};
}
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
int queryConvert(const CharT * bytes, int length)
{
static const Poco::UTF8Encoding utf8;
return utf8.queryConvert(reinterpret_cast<const uint8_t *>(bytes), length);
}
/// returns UTF-8 wcswidth. Invalid sequence is treated as zero width character.
/// `prefix` is used to compute the `\t` width which extends the string before

View File

@ -60,7 +60,7 @@ namespace VolnitskyTraits
static inline Ngram toNGram(const UInt8 * const pos) { return unalignedLoad<Ngram>(pos); }
template <typename Callback>
static inline void putNGramASCIICaseInsensitive(const UInt8 * const pos, const int offset, const Callback & putNGramBase)
static inline void putNGramASCIICaseInsensitive(const UInt8 * pos, int offset, Callback && putNGramBase)
{
struct Chars
{
@ -109,199 +109,234 @@ namespace VolnitskyTraits
putNGramBase(n, offset);
}
template <bool CaseSensitive, bool ASCII, typename Callback>
static inline void putNGram(const UInt8 * const pos, const int offset, [[maybe_unused]] const UInt8 * const begin, const Callback & putNGramBase)
template <typename Callback>
static inline void putNGramUTF8CaseInsensitive(
const UInt8 * pos, int offset, const UInt8 * begin, size_t size, Callback && putNGramBase)
{
if constexpr (CaseSensitive)
const UInt8 * end = begin + size;
struct Chars
{
putNGramBase(toNGram(pos), offset);
UInt8 c0;
UInt8 c1;
};
union
{
VolnitskyTraits::Ngram n;
Chars chars;
};
n = toNGram(pos);
if (isascii(chars.c0) && isascii(chars.c1))
{
putNGramASCIICaseInsensitive(pos, offset, putNGramBase);
}
else
{
if constexpr (ASCII)
/** n-gram (in the case of n = 2)
* can be entirely located within one code point,
* or intersect with two code points.
*
* In the first case, you need to consider up to two alternatives - this code point in upper and lower case,
* and in the second case - up to four alternatives - fragments of two code points in all combinations of cases.
*
* It does not take into account the dependence of the case-transformation from the locale (for example - Turkish `Ii`)
* as well as composition / decomposition and other features.
*
* It also does not work if characters with lower and upper cases are represented by different number of bytes or code points.
*/
using Seq = UInt8[6];
if (UTF8::isContinuationOctet(chars.c1))
{
putNGramASCIICaseInsensitive(pos, offset, putNGramBase);
}
else
{
struct Chars
/// ngram is inside a sequence
auto seq_pos = pos;
UTF8::syncBackward(seq_pos, begin);
auto u32 = UTF8::convertUTF8ToCodePoint(seq_pos, end - seq_pos);
/// Invalid UTF-8
if (!u32)
{
UInt8 c0;
UInt8 c1;
};
union
{
VolnitskyTraits::Ngram n;
Chars chars;
};
n = toNGram(pos);
if (isascii(chars.c0) && isascii(chars.c1))
putNGramASCIICaseInsensitive(pos, offset, putNGramBase);
putNGramBase(n, offset);
}
else
{
/** n-gram (in the case of n = 2)
* can be entirely located within one code point,
* or intersect with two code points.
*
* In the first case, you need to consider up to two alternatives - this code point in upper and lower case,
* and in the second case - up to four alternatives - fragments of two code points in all combinations of cases.
*
* It does not take into account the dependence of the case-transformation from the locale (for example - Turkish `Ii`)
* as well as composition / decomposition and other features.
*
* It also does not work if characters with lower and upper cases are represented by different number of bytes or code points.
*/
int l_u32 = Poco::Unicode::toLower(*u32);
int u_u32 = Poco::Unicode::toUpper(*u32);
using Seq = UInt8[6];
if (UTF8::isContinuationOctet(chars.c1))
/// symbol is case-independent
if (l_u32 == u_u32)
{
/// ngram is inside a sequence
auto seq_pos = pos;
UTF8::syncBackward(seq_pos, begin);
const auto u32 = UTF8::convert(seq_pos);
const auto l_u32 = Poco::Unicode::toLower(u32);
const auto u_u32 = Poco::Unicode::toUpper(u32);
/// symbol is case-independent
if (l_u32 == u_u32)
putNGramBase(n, offset);
else
{
/// where is the given ngram in respect to the start of UTF-8 sequence?
const auto seq_ngram_offset = pos - seq_pos;
Seq seq;
/// put ngram for lowercase
UTF8::convert(l_u32, seq, sizeof(seq));
chars.c0 = seq[seq_ngram_offset];
chars.c1 = seq[seq_ngram_offset + 1];
putNGramBase(n, offset);
/// put ngram for uppercase
UTF8::convert(u_u32, seq, sizeof(seq));
chars.c0 = seq[seq_ngram_offset]; //-V519
chars.c1 = seq[seq_ngram_offset + 1]; //-V519
putNGramBase(n, offset);
}
putNGramBase(n, offset);
}
else
{
/// ngram is on the boundary of two sequences
/// first sequence may start before u_pos if it is not ASCII
auto first_seq_pos = pos;
UTF8::syncBackward(first_seq_pos, begin);
/// where is the given ngram in respect to the start of first UTF-8 sequence?
const auto seq_ngram_offset = pos - first_seq_pos;
/// where is the given ngram in respect to the start of UTF-8 sequence?
size_t seq_ngram_offset = pos - seq_pos;
const auto first_u32 = UTF8::convert(first_seq_pos);
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
Seq seq;
/// second sequence always start immediately after u_pos
auto second_seq_pos = pos + 1;
/// put ngram for lowercase
size_t length_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(l_u32, seq, sizeof(seq));
assert(length_l >= 2);
chars.c0 = seq[seq_ngram_offset];
chars.c1 = seq[seq_ngram_offset + 1];
putNGramBase(n, offset);
const auto second_u32 = UTF8::convert(second_seq_pos); /// TODO This assumes valid UTF-8 or zero byte after needle.
const auto second_l_u32 = Poco::Unicode::toLower(second_u32);
const auto second_u_u32 = Poco::Unicode::toUpper(second_u32);
/// put ngram for uppercase
size_t length_r [[maybe_unused]] = UTF8::convertCodePointToUTF8(u_u32, seq, sizeof(seq));
assert(length_r >= 2);
chars.c0 = seq[seq_ngram_offset]; //-V519
chars.c1 = seq[seq_ngram_offset + 1]; //-V519
putNGramBase(n, offset);
}
}
}
else
{
/// ngram is on the boundary of two sequences
/// first sequence may start before u_pos if it is not ASCII
auto first_seq_pos = pos;
UTF8::syncBackward(first_seq_pos, begin);
/// where is the given ngram in respect to the start of first UTF-8 sequence?
size_t seq_ngram_offset = pos - first_seq_pos;
/// both symbols are case-independent
if (first_l_u32 == first_u_u32 && second_l_u32 == second_u_u32)
{
putNGramBase(n, offset);
}
else if (first_l_u32 == first_u_u32)
{
/// first symbol is case-independent
Seq seq;
auto first_u32 = UTF8::convertUTF8ToCodePoint(first_seq_pos, end - first_seq_pos);
int first_l_u32 = 0;
int first_u_u32 = 0;
/// put ngram for lowercase
UTF8::convert(second_l_u32, seq, sizeof(seq));
chars.c1 = seq[0];
putNGramBase(n, offset);
if (first_u32)
{
first_l_u32 = Poco::Unicode::toLower(*first_u32);
first_u_u32 = Poco::Unicode::toUpper(*first_u32);
}
/// put ngram from uppercase, if it is different
UTF8::convert(second_u_u32, seq, sizeof(seq));
if (chars.c1 != seq[0])
{
chars.c1 = seq[0];
putNGramBase(n, offset);
}
}
else if (second_l_u32 == second_u_u32)
{
/// second symbol is case-independent
Seq seq;
/// second sequence always start immediately after u_pos
auto second_seq_pos = pos + 1;
/// put ngram for lowercase
UTF8::convert(first_l_u32, seq, sizeof(seq));
chars.c0 = seq[seq_ngram_offset];
putNGramBase(n, offset);
auto second_u32 = UTF8::convertUTF8ToCodePoint(second_seq_pos, end - second_seq_pos);
int second_l_u32 = 0;
int second_u_u32 = 0;
/// put ngram for uppercase, if it is different
UTF8::convert(first_u_u32, seq, sizeof(seq));
if (chars.c0 != seq[seq_ngram_offset])
{
chars.c0 = seq[seq_ngram_offset];
putNGramBase(n, offset);
}
}
else
{
Seq first_l_seq;
Seq first_u_seq;
Seq second_l_seq;
Seq second_u_seq;
if (second_u32)
{
second_l_u32 = Poco::Unicode::toLower(*second_u32);
second_u_u32 = Poco::Unicode::toUpper(*second_u32);
}
UTF8::convert(first_l_u32, first_l_seq, sizeof(first_l_seq));
UTF8::convert(first_u_u32, first_u_seq, sizeof(first_u_seq));
UTF8::convert(second_l_u32, second_l_seq, sizeof(second_l_seq));
UTF8::convert(second_u_u32, second_u_seq, sizeof(second_u_seq));
/// both symbols are case-independent
if (first_l_u32 == first_u_u32 && second_l_u32 == second_u_u32)
{
putNGramBase(n, offset);
}
else if (first_l_u32 == first_u_u32)
{
/// first symbol is case-independent
Seq seq;
auto c0l = first_l_seq[seq_ngram_offset];
auto c0u = first_u_seq[seq_ngram_offset];
auto c1l = second_l_seq[0];
auto c1u = second_u_seq[0];
/// put ngram for lowercase
size_t size_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(second_l_u32, seq, sizeof(seq));
assert(size_l >= 1);
chars.c1 = seq[0];
putNGramBase(n, offset);
/// ngram for ll
chars.c0 = c0l;
chars.c1 = c1l;
putNGramBase(n, offset);
/// put ngram from uppercase, if it is different
size_t size_u [[maybe_unused]] = UTF8::convertCodePointToUTF8(second_u_u32, seq, sizeof(seq));
assert(size_u >= 1);
if (chars.c1 != seq[0])
{
chars.c1 = seq[0];
putNGramBase(n, offset);
}
}
else if (second_l_u32 == second_u_u32)
{
/// second symbol is case-independent
Seq seq;
if (c0l != c0u)
{
/// ngram for Ul
chars.c0 = c0u;
chars.c1 = c1l;
putNGramBase(n, offset);
}
/// put ngram for lowercase
size_t size_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(first_l_u32, seq, sizeof(seq));
assert(size_l > seq_ngram_offset);
chars.c0 = seq[seq_ngram_offset];
putNGramBase(n, offset);
if (c1l != c1u)
{
/// ngram for lU
chars.c0 = c0l;
chars.c1 = c1u;
putNGramBase(n, offset);
}
/// put ngram for uppercase, if it is different
size_t size_u [[maybe_unused]] = UTF8::convertCodePointToUTF8(first_u_u32, seq, sizeof(seq));
assert(size_u > seq_ngram_offset);
if (chars.c0 != seq[seq_ngram_offset])
{
chars.c0 = seq[seq_ngram_offset];
putNGramBase(n, offset);
}
}
else
{
Seq first_l_seq;
Seq first_u_seq;
Seq second_l_seq;
Seq second_u_seq;
if (c0l != c0u && c1l != c1u)
{
/// ngram for UU
chars.c0 = c0u;
chars.c1 = c1u;
putNGramBase(n, offset);
}
}
size_t size_first_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(first_l_u32, first_l_seq, sizeof(first_l_seq));
size_t size_first_u [[maybe_unused]] = UTF8::convertCodePointToUTF8(first_u_u32, first_u_seq, sizeof(first_u_seq));
size_t size_second_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(second_l_u32, second_l_seq, sizeof(second_l_seq));
size_t size_second_u [[maybe_unused]] = UTF8::convertCodePointToUTF8(second_u_u32, second_u_seq, sizeof(second_u_seq));
assert(size_first_l > seq_ngram_offset);
assert(size_first_u > seq_ngram_offset);
assert(size_second_l > 0);
assert(size_second_u > 0);
auto c0l = first_l_seq[seq_ngram_offset];
auto c0u = first_u_seq[seq_ngram_offset];
auto c1l = second_l_seq[0];
auto c1u = second_u_seq[0];
/// ngram for ll
chars.c0 = c0l;
chars.c1 = c1l;
putNGramBase(n, offset);
if (c0l != c0u)
{
/// ngram for Ul
chars.c0 = c0u;
chars.c1 = c1l;
putNGramBase(n, offset);
}
if (c1l != c1u)
{
/// ngram for lU
chars.c0 = c0l;
chars.c1 = c1u;
putNGramBase(n, offset);
}
if (c0l != c0u && c1l != c1u)
{
/// ngram for UU
chars.c0 = c0u;
chars.c1 = c1u;
putNGramBase(n, offset);
}
}
}
}
}
template <bool CaseSensitive, bool ASCII, typename Callback>
static inline void putNGram(const UInt8 * pos, int offset, [[maybe_unused]] const UInt8 * begin, size_t size, Callback && putNGramBase)
{
if constexpr (CaseSensitive)
putNGramBase(toNGram(pos), offset);
else if constexpr (ASCII)
putNGramASCIICaseInsensitive(pos, offset, std::forward<Callback>(putNGramBase));
else
putNGramUTF8CaseInsensitive(pos, offset, begin, size, std::forward<Callback>(putNGramBase));
}
}
@ -310,17 +345,17 @@ template <bool CaseSensitive, bool ASCII, typename FallbackSearcher>
class VolnitskyBase
{
protected:
const UInt8 * const needle;
const size_t needle_size;
const UInt8 * const needle_end = needle + needle_size;
const UInt8 * needle;
size_t needle_size;
const UInt8 * needle_end = needle + needle_size;
/// For how long we move, if the n-gram from haystack is not found in the hash table.
const size_t step = needle_size - sizeof(VolnitskyTraits::Ngram) + 1;
size_t step = needle_size - sizeof(VolnitskyTraits::Ngram) + 1;
/** max needle length is 255, max distinct ngrams for case-sensitive is (255 - 1), case-insensitive is 4 * (255 - 1)
* storage of 64K ngrams (n = 2, 128 KB) should be large enough for both cases */
std::unique_ptr<VolnitskyTraits::Offset[]> hash; /// Hash table.
const bool fallback; /// Do we need to use the fallback algorithm.
bool fallback; /// Do we need to use the fallback algorithm.
FallbackSearcher fallback_searcher;
@ -346,7 +381,7 @@ public:
/// ssize_t is used here because unsigned can't be used with condition like `i >= 0`, unsigned always >= 0
/// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first.
for (auto i = static_cast<ssize_t>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i)
VolnitskyTraits::putNGram<CaseSensitive, ASCII>(this->needle + i, i + 1, this->needle, callback);
VolnitskyTraits::putNGram<CaseSensitive, ASCII>(needle + i, i + 1, needle, needle_size, callback);
}
@ -493,6 +528,7 @@ public:
reinterpret_cast<const UInt8 *>(cur_needle_data) + i,
i + 1,
reinterpret_cast<const UInt8 *>(cur_needle_data),
cur_needle_size,
callback);
}
}

View File

@ -409,6 +409,15 @@ Block Block::cloneWithoutColumns() const
return res;
}
Block Block::cloneWithCutColumns(size_t start, size_t length) const
{
Block copy = *this;
for (auto & column_to_cut : copy.data)
column_to_cut.column = column_to_cut.column->cut(start, length);
return copy;
}
Block Block::sortColumns() const
{

View File

@ -129,6 +129,7 @@ public:
void setColumns(const Columns & columns);
Block cloneWithColumns(const Columns & columns) const;
Block cloneWithoutColumns() const;
Block cloneWithCutColumns(size_t start, size_t length) const;
/** Get empty columns with the same types as in block. */
MutableColumns cloneEmptyColumns() const;

View File

@ -36,6 +36,7 @@
#define DEFAULT_MERGE_BLOCK_SIZE 8192
#define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5
#define DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC 60
#define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160)
#define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024

View File

@ -391,6 +391,7 @@ class IColumn;
M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
M(Seconds, periodic_live_view_refresh, DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC, "Interval after which periodically refreshed live view is forced to refresh.", 0) \
M(Bool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
M(Bool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \
M(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \

View File

@ -0,0 +1,65 @@
#include <DataStreams/ITTLAlgorithm.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnConst.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
ITTLAlgorithm::ITTLAlgorithm(
const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: description(description_)
, old_ttl_info(old_ttl_info_)
, current_time(current_time_)
, force(force_)
, date_lut(DateLUT::instance())
{
}
bool ITTLAlgorithm::isTTLExpired(time_t ttl) const
{
return (ttl && (ttl <= current_time));
}
ColumnPtr ITTLAlgorithm::executeExpressionAndGetColumn(
const ExpressionActionsPtr & expression, const Block & block, const String & result_column)
{
if (!expression)
return nullptr;
if (block.has(result_column))
return block.getByName(result_column).column;
Block block_copy;
for (const auto & column_name : expression->getRequiredColumns())
block_copy.insert(block.getByName(column_name));
/// Keep number of rows for const expression.
size_t num_rows = block.rows();
expression->execute(block_copy, num_rows);
return block_copy.getByName(result_column).column;
}
UInt32 ITTLAlgorithm::getTimestampByIndex(const IColumn * column, size_t index) const
{
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
return date_lut.fromDayNum(DayNum(column_date->getData()[index]));
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
return column_date_time->getData()[index];
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
{
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
return date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>()));
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
return column_const->getValue<UInt32>();
}
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -0,0 +1,54 @@
#pragma once
#include <Storages/TTLDescription.h>
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <common/DateLUT.h>
namespace DB
{
/**
* Represents the actions, which are required to do
* with data, when TTL is expired: delete, aggregate, etc.
*/
class ITTLAlgorithm
{
public:
using TTLInfo = IMergeTreeDataPart::TTLInfo;
using MutableDataPartPtr = MergeTreeMutableDataPartPtr;
ITTLAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
virtual ~ITTLAlgorithm() = default;
virtual void execute(Block & block) = 0;
/// Updates TTL metadata of the data_part.
virtual void finalize(const MutableDataPartPtr & data_part) const = 0;
bool isMinTTLExpired() const { return force || isTTLExpired(old_ttl_info.min); }
bool isMaxTTLExpired() const { return isTTLExpired(old_ttl_info.max); }
/** This function is needed to avoid a conflict between already calculated columns and columns that needed to execute TTL.
* If result column is absent in block, all required columns are copied to new block and expression is executed on new block.
*/
static ColumnPtr executeExpressionAndGetColumn(
const ExpressionActionsPtr & expression, const Block & block, const String & result_column);
protected:
bool isTTLExpired(time_t ttl) const;
UInt32 getTimestampByIndex(const IColumn * column, size_t index) const;
const TTLDescription description;
const TTLInfo old_ttl_info;
const time_t current_time;
const bool force;
TTLInfo new_ttl_info;
private:
const DateLUTImpl & date_lut;
};
using TTLAlgorithmPtr = std::unique_ptr<ITTLAlgorithm>;
}

View File

@ -0,0 +1,173 @@
#include <DataStreams/TTLAggregationAlgorithm.h>
namespace DB
{
TTLAggregationAlgorithm::TTLAggregationAlgorithm(
const TTLDescription & description_,
const TTLInfo & old_ttl_info_,
time_t current_time_,
bool force_,
const Block & header_,
const MergeTreeData & storage_)
: ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_)
, header(header_)
{
current_key_value.resize(description.group_by_keys.size());
ColumnNumbers keys;
for (const auto & key : description.group_by_keys)
keys.push_back(header.getPositionByName(key));
key_columns.resize(description.group_by_keys.size());
AggregateDescriptions aggregates = description.aggregate_descriptions;
for (auto & descr : aggregates)
if (descr.arguments.empty())
for (const auto & name : descr.argument_names)
descr.arguments.push_back(header.getPositionByName(name));
columns_for_aggregator.resize(description.aggregate_descriptions.size());
const Settings & settings = storage_.global_context.getSettingsRef();
Aggregator::Params params(header, keys, aggregates,
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0,
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
storage_.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
aggregator = std::make_unique<Aggregator>(params);
}
void TTLAggregationAlgorithm::execute(Block & block)
{
if (!block)
{
if (!aggregation_result.empty())
{
MutableColumns result_columns = header.cloneEmptyColumns();
finalizeAggregates(result_columns);
block = header.cloneWithColumns(std::move(result_columns));
}
return;
}
const auto & column_names = header.getNames();
MutableColumns result_columns = header.cloneEmptyColumns();
MutableColumns aggregate_columns = header.cloneEmptyColumns();
auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column);
auto where_column = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column);
size_t rows_aggregated = 0;
size_t current_key_start = 0;
size_t rows_with_current_key = 0;
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
bool where_filter_passed = !where_column || where_column->getBool(i);
bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed;
bool same_as_current = true;
for (size_t j = 0; j < description.group_by_keys.size(); ++j)
{
const String & key_column = description.group_by_keys[j];
const IColumn * values_column = block.getByName(key_column).column.get();
if (!same_as_current || (*values_column)[i] != current_key_value[j])
{
values_column->get(i, current_key_value[j]);
same_as_current = false;
}
}
if (!same_as_current)
{
if (rows_with_current_key)
calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key);
finalizeAggregates(result_columns);
current_key_start = rows_aggregated;
rows_with_current_key = 0;
}
if (ttl_expired)
{
++rows_with_current_key;
++rows_aggregated;
for (const auto & name : column_names)
{
const IColumn * values_column = block.getByName(name).column.get();
auto & column = aggregate_columns[header.getPositionByName(name)];
column->insertFrom(*values_column, i);
}
}
else
{
new_ttl_info.update(cur_ttl);
for (const auto & name : column_names)
{
const IColumn * values_column = block.getByName(name).column.get();
auto & column = result_columns[header.getPositionByName(name)];
column->insertFrom(*values_column, i);
}
}
}
if (rows_with_current_key)
calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key);
block = header.cloneWithColumns(std::move(result_columns));
}
void TTLAggregationAlgorithm::calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length)
{
Columns aggregate_chunk;
aggregate_chunk.reserve(aggregate_columns.size());
for (const auto & name : header.getNames())
{
const auto & column = aggregate_columns[header.getPositionByName(name)];
ColumnPtr chunk_column = column->cut(start_pos, length);
aggregate_chunk.emplace_back(std::move(chunk_column));
}
aggregator->executeOnBlock(aggregate_chunk, length, aggregation_result, key_columns,
columns_for_aggregator, no_more_keys);
}
void TTLAggregationAlgorithm::finalizeAggregates(MutableColumns & result_columns)
{
if (!aggregation_result.empty())
{
auto aggregated_res = aggregator->convertToBlocks(aggregation_result, true, 1);
for (auto & agg_block : aggregated_res)
{
for (const auto & it : description.set_parts)
it.expression->execute(agg_block);
for (const auto & name : description.group_by_keys)
{
const IColumn * values_column = agg_block.getByName(name).column.get();
auto & result_column = result_columns[header.getPositionByName(name)];
result_column->insertRangeFrom(*values_column, 0, agg_block.rows());
}
for (const auto & it : description.set_parts)
{
const IColumn * values_column = agg_block.getByName(it.expression_result_column_name).column.get();
auto & result_column = result_columns[header.getPositionByName(it.column_name)];
result_column->insertRangeFrom(*values_column, 0, agg_block.rows());
}
}
}
aggregation_result.invalidate();
}
void TTLAggregationAlgorithm::finalize(const MutableDataPartPtr & data_part) const
{
data_part->ttl_infos.group_by_ttl[description.result_column] = new_ttl_info;
data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max);
}
}

View File

@ -0,0 +1,42 @@
#pragma once
#include <DataStreams/ITTLAlgorithm.h>
#include <Interpreters/Aggregator.h>
#include <Storages/MergeTree/MergeTreeData.h>
namespace DB
{
/// Aggregates rows according to 'TTL expr GROUP BY key' description.
/// Aggregation key must be the prefix of the sorting key.
class TTLAggregationAlgorithm final : public ITTLAlgorithm
{
public:
TTLAggregationAlgorithm(
const TTLDescription & description_,
const TTLInfo & old_ttl_info_,
time_t current_time_,
bool force_,
const Block & header_,
const MergeTreeData & storage_);
void execute(Block & block) override;
void finalize(const MutableDataPartPtr & data_part) const override;
private:
// Calculate aggregates of aggregate_columns into aggregation_result
void calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length);
/// Finalize aggregation_result into result_columns
void finalizeAggregates(MutableColumns & result_columns);
const Block header;
std::unique_ptr<Aggregator> aggregator;
Row current_key_value;
AggregatedDataVariants aggregation_result;
ColumnRawPtrs key_columns;
Aggregator::AggregateColumns columns_for_aggregator;
bool no_more_keys = false;
};
}

View File

@ -8,15 +8,14 @@
#include <Storages/TTLMode.h>
#include <Interpreters/Context.h>
#include <DataStreams/TTLDeleteAlgorithm.h>
#include <DataStreams/TTLColumnAlgorithm.h>
#include <DataStreams/TTLAggregationAlgorithm.h>
#include <DataStreams/TTLUpdateInfoAlgorithm.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
TTLBlockInputStream::TTLBlockInputStream(
const BlockInputStreamPtr & input_,
const MergeTreeData & storage_,
@ -24,83 +23,69 @@ TTLBlockInputStream::TTLBlockInputStream(
const MergeTreeData::MutableDataPartPtr & data_part_,
time_t current_time_,
bool force_)
: storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, data_part(data_part_)
, current_time(current_time_)
, force(force_)
, old_ttl_infos(data_part->ttl_infos)
, log(&Poco::Logger::get(storage.getLogName() + " (TTLBlockInputStream)"))
, date_lut(DateLUT::instance())
: data_part(data_part_)
, log(&Poco::Logger::get(storage_.getLogName() + " (TTLBlockInputStream)"))
{
children.push_back(input_);
header = children.at(0)->getHeader();
auto old_ttl_infos = data_part->ttl_infos;
const auto & storage_columns = metadata_snapshot->getColumns();
const auto & column_defaults = storage_columns.getDefaults();
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
for (const auto & [name, _] : metadata_snapshot->getColumnTTLs())
if (metadata_snapshot_->hasRowsTTL())
{
auto it = column_defaults.find(name);
if (it != column_defaults.end())
const auto & rows_ttl = metadata_snapshot_->getRowsTTL();
auto algorithm = std::make_unique<TTLDeleteAlgorithm>(
rows_ttl, old_ttl_infos.table_ttl, current_time_, force_);
/// Skip all data if table ttl is expired for part
if (algorithm->isMaxTTLExpired() && !rows_ttl.where_expression)
all_data_dropped = true;
delete_algorithm = algorithm.get();
algorithms.emplace_back(std::move(algorithm));
}
for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs())
algorithms.emplace_back(std::make_unique<TTLDeleteAlgorithm>(
where_ttl, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_));
for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs())
algorithms.emplace_back(std::make_unique<TTLAggregationAlgorithm>(
group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, header, storage_));
if (metadata_snapshot_->hasAnyColumnTTL())
{
const auto & storage_columns = metadata_snapshot_->getColumns();
const auto & column_defaults = storage_columns.getDefaults();
for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs())
{
auto column = storage_columns.get(name);
auto expression = it->second.expression->clone();
default_expr_list->children.emplace_back(setAlias(addTypeConversionToAST(std::move(expression), column.type->getName()), it->first));
ExpressionActionsPtr default_expression;
String default_column_name;
auto it = column_defaults.find(name);
if (it != column_defaults.end())
{
const auto & column = storage_columns.get(name);
auto default_ast = it->second.expression->clone();
default_ast = addTypeConversionToAST(std::move(default_ast), column.type->getName());
auto syntax_result = TreeRewriter(storage_.global_context).analyze(default_ast, metadata_snapshot_->getColumns().getAllPhysical());
default_expression = ExpressionAnalyzer{default_ast, syntax_result, storage_.global_context}.getActions(true);
default_column_name = default_ast->getColumnName();
}
algorithms.emplace_back(std::make_unique<TTLColumnAlgorithm>(
description, old_ttl_infos.columns_ttl[name], current_time_,
force_, name, default_expression, default_column_name));
}
}
for (const auto & [name, ttl_info] : old_ttl_infos.columns_ttl)
{
if (force || isTTLExpired(ttl_info.min))
{
new_ttl_infos.columns_ttl.emplace(name, IMergeTreeDataPart::TTLInfo{});
empty_columns.emplace(name);
}
else
new_ttl_infos.columns_ttl.emplace(name, ttl_info);
}
for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs())
algorithms.emplace_back(std::make_unique<TTLMoveAlgorithm>(
move_ttl, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_));
if (!force && !isTTLExpired(old_ttl_infos.table_ttl.min))
new_ttl_infos.table_ttl = old_ttl_infos.table_ttl;
if (!default_expr_list->children.empty())
{
auto syntax_result = TreeRewriter(storage.global_context).analyze(default_expr_list, metadata_snapshot->getColumns().getAllPhysical());
defaults_expression = ExpressionAnalyzer{default_expr_list, syntax_result, storage.global_context}.getActions(true);
}
auto storage_rows_ttl = metadata_snapshot->getRowsTTL();
if (metadata_snapshot->hasRowsTTL() && storage_rows_ttl.mode == TTLMode::GROUP_BY)
{
current_key_value.resize(storage_rows_ttl.group_by_keys.size());
ColumnNumbers keys;
for (const auto & key : storage_rows_ttl.group_by_keys)
keys.push_back(header.getPositionByName(key));
agg_key_columns.resize(storage_rows_ttl.group_by_keys.size());
AggregateDescriptions aggregates = storage_rows_ttl.aggregate_descriptions;
for (auto & descr : aggregates)
if (descr.arguments.empty())
for (const auto & name : descr.argument_names)
descr.arguments.push_back(header.getPositionByName(name));
agg_aggregate_columns.resize(storage_rows_ttl.aggregate_descriptions.size());
const Settings & settings = storage.global_context.getSettingsRef();
Aggregator::Params params(header, keys, aggregates,
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0,
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
storage.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
aggregator = std::make_unique<Aggregator>(params);
}
}
bool TTLBlockInputStream::isTTLExpired(time_t ttl) const
{
return (ttl && (ttl <= current_time));
for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs())
algorithms.emplace_back(std::make_unique<TTLRecompressionAlgorithm>(
recompression_ttl, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_));
}
Block reorderColumns(Block block, const Block & header)
@ -114,321 +99,30 @@ Block reorderColumns(Block block, const Block & header)
Block TTLBlockInputStream::readImpl()
{
/// Skip all data if table ttl is expired for part
auto storage_rows_ttl = metadata_snapshot->getRowsTTL();
if (metadata_snapshot->hasRowsTTL() && !storage_rows_ttl.where_expression && storage_rows_ttl.mode != TTLMode::GROUP_BY
&& isTTLExpired(old_ttl_infos.table_ttl.max))
{
rows_removed = data_part->rows_count;
if (all_data_dropped)
return {};
}
auto block = children.at(0)->read();
for (const auto & algorithm : algorithms)
algorithm->execute(block);
Block block = children.at(0)->read();
if (!block)
{
if (aggregator && !agg_result.empty())
{
MutableColumns result_columns = header.cloneEmptyColumns();
finalizeAggregates(result_columns);
block = header.cloneWithColumns(std::move(result_columns));
}
return block;
}
if (metadata_snapshot->hasRowsTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min)))
removeRowsWithExpiredTableTTL(block);
removeValuesWithExpiredColumnTTL(block);
updateMovesTTL(block);
updateRecompressionTTL(block);
return reorderColumns(std::move(block), header);
}
void TTLBlockInputStream::readSuffixImpl()
{
for (const auto & elem : new_ttl_infos.columns_ttl)
new_ttl_infos.updatePartMinMaxTTL(elem.second.min, elem.second.max);
data_part->ttl_infos = {};
for (const auto & algorithm : algorithms)
algorithm->finalize(data_part);
new_ttl_infos.updatePartMinMaxTTL(new_ttl_infos.table_ttl.min, new_ttl_infos.table_ttl.max);
data_part->ttl_infos = std::move(new_ttl_infos);
data_part->expired_columns = std::move(empty_columns);
if (rows_removed)
if (delete_algorithm)
{
size_t rows_removed = all_data_dropped ? data_part->rows_count : delete_algorithm->getNumberOfRemovedRows();
LOG_DEBUG(log, "Removed {} rows with expired TTL from part {}", rows_removed, data_part->name);
}
void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block)
{
auto rows_ttl = metadata_snapshot->getRowsTTL();
rows_ttl.expression->execute(block);
if (rows_ttl.where_expression)
rows_ttl.where_expression->execute(block);
const IColumn * ttl_column =
block.getByName(rows_ttl.result_column).column.get();
const IColumn * where_result_column = rows_ttl.where_expression ?
block.getByName(rows_ttl.where_result_column).column.get() : nullptr;
const auto & column_names = header.getNames();
if (!aggregator)
{
MutableColumns result_columns;
result_columns.reserve(column_names.size());
for (auto it = column_names.begin(); it != column_names.end(); ++it)
{
const IColumn * values_column = block.getByName(*it).column.get();
MutableColumnPtr result_column = values_column->cloneEmpty();
result_column->reserve(block.rows());
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column, i);
bool where_filter_passed = !where_result_column || where_result_column->getBool(i);
if (!isTTLExpired(cur_ttl) || !where_filter_passed)
{
new_ttl_infos.table_ttl.update(cur_ttl);
result_column->insertFrom(*values_column, i);
}
else if (it == column_names.begin())
++rows_removed;
}
result_columns.emplace_back(std::move(result_column));
}
block = header.cloneWithColumns(std::move(result_columns));
}
else
{
MutableColumns result_columns = header.cloneEmptyColumns();
MutableColumns aggregate_columns = header.cloneEmptyColumns();
size_t rows_aggregated = 0;
size_t current_key_start = 0;
size_t rows_with_current_key = 0;
auto storage_rows_ttl = metadata_snapshot->getRowsTTL();
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column, i);
bool where_filter_passed = !where_result_column || where_result_column->getBool(i);
bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed;
bool same_as_current = true;
for (size_t j = 0; j < storage_rows_ttl.group_by_keys.size(); ++j)
{
const String & key_column = storage_rows_ttl.group_by_keys[j];
const IColumn * values_column = block.getByName(key_column).column.get();
if (!same_as_current || (*values_column)[i] != current_key_value[j])
{
values_column->get(i, current_key_value[j]);
same_as_current = false;
}
}
if (!same_as_current)
{
if (rows_with_current_key)
calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key);
finalizeAggregates(result_columns);
current_key_start = rows_aggregated;
rows_with_current_key = 0;
}
if (ttl_expired)
{
++rows_with_current_key;
++rows_aggregated;
for (const auto & name : column_names)
{
const IColumn * values_column = block.getByName(name).column.get();
auto & column = aggregate_columns[header.getPositionByName(name)];
column->insertFrom(*values_column, i);
}
}
else
{
new_ttl_infos.table_ttl.update(cur_ttl);
for (const auto & name : column_names)
{
const IColumn * values_column = block.getByName(name).column.get();
auto & column = result_columns[header.getPositionByName(name)];
column->insertFrom(*values_column, i);
}
}
}
if (rows_with_current_key)
calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key);
block = header.cloneWithColumns(std::move(result_columns));
}
}
void TTLBlockInputStream::calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length)
{
Columns aggregate_chunk;
aggregate_chunk.reserve(aggregate_columns.size());
for (const auto & name : header.getNames())
{
const auto & column = aggregate_columns[header.getPositionByName(name)];
ColumnPtr chunk_column = column->cut(start_pos, length);
aggregate_chunk.emplace_back(std::move(chunk_column));
}
aggregator->executeOnBlock(aggregate_chunk, length, agg_result, agg_key_columns,
agg_aggregate_columns, agg_no_more_keys);
}
void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns)
{
if (!agg_result.empty())
{
auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1);
auto storage_rows_ttl = metadata_snapshot->getRowsTTL();
for (auto & agg_block : aggregated_res)
{
for (const auto & it : storage_rows_ttl.set_parts)
it.expression->execute(agg_block);
for (const auto & name : storage_rows_ttl.group_by_keys)
{
const IColumn * values_column = agg_block.getByName(name).column.get();
auto & result_column = result_columns[header.getPositionByName(name)];
result_column->insertRangeFrom(*values_column, 0, agg_block.rows());
}
for (const auto & it : storage_rows_ttl.set_parts)
{
const IColumn * values_column = agg_block.getByName(it.expression_result_column_name).column.get();
auto & result_column = result_columns[header.getPositionByName(it.column_name)];
result_column->insertRangeFrom(*values_column, 0, agg_block.rows());
}
}
}
agg_result.invalidate();
}
void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block)
{
Block block_with_defaults;
if (defaults_expression)
{
block_with_defaults = block;
defaults_expression->execute(block_with_defaults);
}
std::vector<String> columns_to_remove;
for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs())
{
/// If we read not all table columns. E.g. while mutation.
if (!block.has(name))
continue;
const auto & old_ttl_info = old_ttl_infos.columns_ttl[name];
auto & new_ttl_info = new_ttl_infos.columns_ttl[name];
/// Nothing to do
if (!force && !isTTLExpired(old_ttl_info.min))
continue;
/// Later drop full column
if (isTTLExpired(old_ttl_info.max))
continue;
if (!block.has(ttl_entry.result_column))
{
columns_to_remove.push_back(ttl_entry.result_column);
ttl_entry.expression->execute(block);
}
ColumnPtr default_column = nullptr;
if (block_with_defaults.has(name))
default_column = block_with_defaults.getByName(name).column->convertToFullColumnIfConst();
auto & column_with_type = block.getByName(name);
const IColumn * values_column = column_with_type.column.get();
MutableColumnPtr result_column = values_column->cloneEmpty();
result_column->reserve(block.rows());
const IColumn * ttl_column = block.getByName(ttl_entry.result_column).column.get();
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column, i);
if (isTTLExpired(cur_ttl))
{
if (default_column)
result_column->insertFrom(*default_column, i);
else
result_column->insertDefault();
}
else
{
new_ttl_info.update(cur_ttl);
empty_columns.erase(name);
result_column->insertFrom(*values_column, i);
}
}
column_with_type.column = std::move(result_column);
}
for (const String & column : columns_to_remove)
block.erase(column);
}
void TTLBlockInputStream::updateTTLWithDescriptions(Block & block, const TTLDescriptions & descriptions, TTLInfoMap & ttl_info_map)
{
std::vector<String> columns_to_remove;
for (const auto & ttl_entry : descriptions)
{
auto & new_ttl_info = ttl_info_map[ttl_entry.result_column];
if (!block.has(ttl_entry.result_column))
{
columns_to_remove.push_back(ttl_entry.result_column);
ttl_entry.expression->execute(block);
}
const IColumn * ttl_column = block.getByName(ttl_entry.result_column).column.get();
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column, i);
new_ttl_info.update(cur_ttl);
}
}
for (const String & column : columns_to_remove)
block.erase(column);
}
void TTLBlockInputStream::updateMovesTTL(Block & block)
{
updateTTLWithDescriptions(block, metadata_snapshot->getMoveTTLs(), new_ttl_infos.moves_ttl);
}
void TTLBlockInputStream::updateRecompressionTTL(Block & block)
{
updateTTLWithDescriptions(block, metadata_snapshot->getRecompressionTTLs(), new_ttl_infos.recompression_ttl);
}
UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t ind)
{
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
return date_lut.fromDayNum(DayNum(column_date->getData()[ind]));
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
return column_date_time->getData()[ind];
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
{
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
return date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>()));
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
return column_const->getValue<UInt32>();
}
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -3,8 +3,9 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <Core/Block.h>
#include <Interpreters/Aggregator.h>
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
#include <DataStreams/ITTLAlgorithm.h>
#include <DataStreams/TTLDeleteAlgorithm.h>
#include <common/DateLUT.h>
@ -24,7 +25,6 @@ public:
);
String getName() const override { return "TTL"; }
Block getHeader() const override { return header; }
protected:
@ -34,60 +34,14 @@ protected:
void readSuffixImpl() override;
private:
const MergeTreeData & storage;
StorageMetadataPtr metadata_snapshot;
std::vector<TTLAlgorithmPtr> algorithms;
const TTLDeleteAlgorithm * delete_algorithm = nullptr;
bool all_data_dropped = false;
/// ttl_infos and empty_columns are updating while reading
const MergeTreeData::MutableDataPartPtr & data_part;
time_t current_time;
bool force;
std::unique_ptr<Aggregator> aggregator;
std::vector<Field> current_key_value;
AggregatedDataVariants agg_result;
ColumnRawPtrs agg_key_columns;
Aggregator::AggregateColumns agg_aggregate_columns;
bool agg_no_more_keys = false;
IMergeTreeDataPart::TTLInfos old_ttl_infos;
IMergeTreeDataPart::TTLInfos new_ttl_infos;
NameSet empty_columns;
size_t rows_removed = 0;
Poco::Logger * log;
const DateLUTImpl & date_lut;
/// TODO rewrite defaults logic to evaluteMissingDefaults
std::unordered_map<String, String> defaults_result_column;
ExpressionActionsPtr defaults_expression;
Block header;
private:
/// Removes values with expired ttl and computes new_ttl_infos and empty_columns for part
void removeValuesWithExpiredColumnTTL(Block & block);
/// Removes rows with expired table ttl and computes new ttl_infos for part
void removeRowsWithExpiredTableTTL(Block & block);
// Calculate aggregates of aggregate_columns into agg_result
void calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length);
/// Finalize agg_result into result_columns
void finalizeAggregates(MutableColumns & result_columns);
/// Execute description expressions on block and update ttl's in
/// ttl_info_map with expression results.
void updateTTLWithDescriptions(Block & block, const TTLDescriptions & descriptions, TTLInfoMap & ttl_info_map);
/// Updates TTL for moves
void updateMovesTTL(Block & block);
/// Update values for recompression TTL using data from block.
void updateRecompressionTTL(Block & block);
UInt32 getTimestampByIndex(const IColumn * column, size_t ind);
bool isTTLExpired(time_t ttl) const;
};
}

View File

@ -0,0 +1,83 @@
#include <DataStreams/TTLColumnAlgorithm.h>
namespace DB
{
TTLColumnAlgorithm::TTLColumnAlgorithm(
const TTLDescription & description_,
const TTLInfo & old_ttl_info_,
time_t current_time_,
bool force_,
const String & column_name_,
const ExpressionActionsPtr & default_expression_,
const String & default_column_name_)
: ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_)
, column_name(column_name_)
, default_expression(default_expression_)
, default_column_name(default_column_name_)
{
if (!isMinTTLExpired())
{
new_ttl_info = old_ttl_info;
is_fully_empty = false;
}
}
void TTLColumnAlgorithm::execute(Block & block)
{
if (!block)
return;
/// If we read not all table columns. E.g. while mutation.
if (!block.has(column_name))
return;
/// Nothing to do
if (!isMinTTLExpired())
return;
/// Later drop full column
if (isMaxTTLExpired())
return;
auto default_column = executeExpressionAndGetColumn(default_expression, block, default_column_name);
if (default_column)
default_column = default_column->convertToFullColumnIfConst();
auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column);
auto & column_with_type = block.getByName(column_name);
const IColumn * values_column = column_with_type.column.get();
MutableColumnPtr result_column = values_column->cloneEmpty();
result_column->reserve(block.rows());
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
if (isTTLExpired(cur_ttl))
{
if (default_column)
result_column->insertFrom(*default_column, i);
else
result_column->insertDefault();
}
else
{
new_ttl_info.update(cur_ttl);
is_fully_empty = false;
result_column->insertFrom(*values_column, i);
}
}
column_with_type.column = std::move(result_column);
}
void TTLColumnAlgorithm::finalize(const MutableDataPartPtr & data_part) const
{
data_part->ttl_infos.columns_ttl[column_name] = new_ttl_info;
data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max);
if (is_fully_empty)
data_part->expired_columns.insert(column_name);
}
}

View File

@ -0,0 +1,33 @@
#pragma once
#include <DataStreams/ITTLAlgorithm.h>
namespace DB
{
/// Deletes (replaces to default) values in column according to column's TTL description.
/// If all values in column are replaced with defaults, this column won't be written to part.
class TTLColumnAlgorithm final : public ITTLAlgorithm
{
public:
TTLColumnAlgorithm(
const TTLDescription & description_,
const TTLInfo & old_ttl_info_,
time_t current_time_,
bool force_,
const String & column_name_,
const ExpressionActionsPtr & default_expression_,
const String & default_column_name_);
void execute(Block & block) override;
void finalize(const MutableDataPartPtr & data_part) const override;
private:
const String column_name;
const ExpressionActionsPtr default_expression;
const String default_column_name;
bool is_fully_empty = true;
};
}

View File

@ -0,0 +1,62 @@
#include <DataStreams/TTLDeleteAlgorithm.h>
namespace DB
{
TTLDeleteAlgorithm::TTLDeleteAlgorithm(
const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_)
{
if (!isMinTTLExpired())
new_ttl_info = old_ttl_info;
}
void TTLDeleteAlgorithm::execute(Block & block)
{
if (!block || !isMinTTLExpired())
return;
auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column);
auto where_column = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column);
MutableColumns result_columns;
const auto & column_names = block.getNames();
result_columns.reserve(column_names.size());
for (auto it = column_names.begin(); it != column_names.end(); ++it)
{
const IColumn * values_column = block.getByName(*it).column.get();
MutableColumnPtr result_column = values_column->cloneEmpty();
result_column->reserve(block.rows());
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
bool where_filter_passed = !where_column || where_column->getBool(i);
if (!isTTLExpired(cur_ttl) || !where_filter_passed)
{
new_ttl_info.update(cur_ttl);
result_column->insertFrom(*values_column, i);
}
else if (it == column_names.begin())
++rows_removed;
}
result_columns.emplace_back(std::move(result_column));
}
block = block.cloneWithColumns(std::move(result_columns));
}
void TTLDeleteAlgorithm::finalize(const MutableDataPartPtr & data_part) const
{
if (description.where_expression)
data_part->ttl_infos.rows_where_ttl[description.result_column] = new_ttl_info;
else
data_part->ttl_infos.table_ttl = new_ttl_info;
data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max);
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <DataStreams/ITTLAlgorithm.h>
namespace DB
{
/// Deletes rows according to table TTL description with
/// possible optional condition in 'WHERE' clause.
class TTLDeleteAlgorithm final : public ITTLAlgorithm
{
public:
TTLDeleteAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
void execute(Block & block) override;
void finalize(const MutableDataPartPtr & data_part) const override;
size_t getNumberOfRemovedRows() const { return rows_removed; }
private:
size_t rows_removed = 0;
};
}

View File

@ -0,0 +1,47 @@
#include <DataStreams/TTLUpdateInfoAlgorithm.h>
namespace DB
{
TTLUpdateInfoAlgorithm::TTLUpdateInfoAlgorithm(
const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_)
{
}
void TTLUpdateInfoAlgorithm::execute(Block & block)
{
if (!block)
return;
auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column);
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i);
new_ttl_info.update(cur_ttl);
}
}
TTLMoveAlgorithm::TTLMoveAlgorithm(
const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: TTLUpdateInfoAlgorithm(description_, old_ttl_info_, current_time_, force_)
{
}
void TTLMoveAlgorithm::finalize(const MutableDataPartPtr & data_part) const
{
data_part->ttl_infos.moves_ttl[description.result_column] = new_ttl_info;
}
TTLRecompressionAlgorithm::TTLRecompressionAlgorithm(
const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: TTLUpdateInfoAlgorithm(description_, old_ttl_info_, current_time_, force_)
{
}
void TTLRecompressionAlgorithm::finalize(const MutableDataPartPtr & data_part) const
{
data_part->ttl_infos.recompression_ttl[description.result_column] = new_ttl_info;
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <DataStreams/ITTLAlgorithm.h>
namespace DB
{
/// Calculates new ttl_info and does nothing with data.
class TTLUpdateInfoAlgorithm : public ITTLAlgorithm
{
public:
TTLUpdateInfoAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
void execute(Block & block) override;
void finalize(const MutableDataPartPtr & data_part) const override = 0;
};
class TTLMoveAlgorithm final : public TTLUpdateInfoAlgorithm
{
public:
TTLMoveAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
void finalize(const MutableDataPartPtr & data_part) const override;
};
class TTLRecompressionAlgorithm final : public TTLUpdateInfoAlgorithm
{
public:
TTLRecompressionAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
void finalize(const MutableDataPartPtr & data_part) const override;
};
}

View File

@ -27,6 +27,7 @@ SRCS(
ExecutionSpeedLimits.cpp
ExpressionBlockInputStream.cpp
IBlockInputStream.cpp
ITTLAlgorithm.cpp
InputStreamFromASTInsertQuery.cpp
InternalTextLogsRowOutputStream.cpp
LimitBlockInputStream.cpp
@ -44,7 +45,11 @@ SRCS(
SquashingBlockInputStream.cpp
SquashingBlockOutputStream.cpp
SquashingTransform.cpp
TTLAggregationAlgorithm.cpp
TTLBlockInputStream.cpp
TTLColumnAlgorithm.cpp
TTLDeleteAlgorithm.cpp
TTLUpdateInfoAlgorithm.cpp
copyData.cpp
finalizeBlock.cpp
materializeBlock.cpp

View File

@ -1,6 +1,5 @@
#include "DictionarySourceHelpers.h"
#include <Columns/ColumnsNumber.h>
#include <Core/Block.h>
#include <Core/ColumnWithTypeAndName.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataTypes/DataTypesNumber.h>
@ -13,44 +12,54 @@
namespace DB
{
/// For simple key
void formatIDs(BlockOutputStreamPtr & out, const std::vector<UInt64> & ids)
void formatBlock(BlockOutputStreamPtr & out, const Block & block)
{
auto column = ColumnUInt64::create(ids.size());
memcpy(column->getData().data(), ids.data(), ids.size() * sizeof(ids.front()));
Block block{{std::move(column), std::make_shared<DataTypeUInt64>(), "id"}};
out->writePrefix();
out->write(block);
out->writeSuffix();
out->flush();
}
/// For composite key
void formatKeys(
/// For simple key
Block blockForIds(
const DictionaryStructure & dict_struct,
const std::vector<UInt64> & ids)
{
auto column = ColumnUInt64::create(ids.size());
memcpy(column->getData().data(), ids.data(), ids.size() * sizeof(ids.front()));
Block block{{std::move(column), std::make_shared<DataTypeUInt64>(), (*dict_struct.id).name}};
return block;
}
/// For composite key
Block blockForKeys(
const DictionaryStructure & dict_struct,
BlockOutputStreamPtr & out,
const Columns & key_columns,
const std::vector<size_t> & requested_rows)
{
Block block;
for (size_t i = 0, size = key_columns.size(); i < size; ++i)
{
const ColumnPtr & source_column = key_columns[i];
auto filtered_column = source_column->cloneEmpty();
filtered_column->reserve(requested_rows.size());
size_t column_rows_size = source_column->size();
PaddedPODArray<UInt8> filter(column_rows_size, false);
for (size_t idx : requested_rows)
filtered_column->insertFrom(*source_column, idx);
filter[idx] = true;
block.insert({std::move(filtered_column), (*dict_struct.key)[i].type, toString(i)});
auto filtered_column = source_column->filter(filter, requested_rows.size());
block.insert({std::move(filtered_column), (*dict_struct.key)[i].type, (*dict_struct.key)[i].name});
}
out->writePrefix();
out->write(block);
out->writeSuffix();
out->flush();
return block;
}
Context copyContextAndApplySettings(

View File

@ -1,11 +1,15 @@
#pragma once
#include <vector>
#include <Columns/IColumn.h>
#include <common/types.h>
#include <Poco/File.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Columns/IColumn.h>
#include <Core/Block.h>
namespace DB
{
class IBlockOutputStream;
@ -16,13 +20,18 @@ class Context;
/// Write keys to block output stream.
void formatBlock(BlockOutputStreamPtr & out, const Block & block);
/// For simple key
void formatIDs(BlockOutputStreamPtr & out, const std::vector<UInt64> & ids);
Block blockForIds(
const DictionaryStructure & dict_struct,
const std::vector<UInt64> & ids);
/// For composite key
void formatKeys(
Block blockForKeys(
const DictionaryStructure & dict_struct,
BlockOutputStreamPtr & out,
const Columns & key_columns,
const std::vector<size_t> & requested_rows);
@ -36,4 +45,5 @@ void applySettingsToContext(
const std::string & config_prefix,
Context & context,
const Poco::Util::AbstractConfiguration & config);
}

View File

@ -281,6 +281,21 @@ size_t DictionaryStructure::getKeySize() const
});
}
Strings DictionaryStructure::getKeysNames() const
{
if (id)
return { id->name };
const auto & key_attributes = *key;
Strings keys_names;
keys_names.reserve(key_attributes.size());
for (const auto & key_attribute : key_attributes)
keys_names.emplace_back(key_attribute.name);
return keys_names;
}
static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & keys)
{

View File

@ -158,6 +158,8 @@ struct DictionaryStructure final
std::string getKeyDescription() const;
bool isKeySizeFixed() const;
size_t getKeySize() const;
Strings getKeysNames() const;
private:
/// range_min and range_max have to be parsed before this function call
std::vector<DictionaryAttribute> getAttributes(

View File

@ -26,6 +26,8 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int DICTIONARY_ACCESS_DENIED;
extern const int UNSUPPORTED_METHOD;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
namespace
@ -65,18 +67,34 @@ ExecutableDictionarySource::ExecutableDictionarySource(
const Context & context_)
: log(&Poco::Logger::get("ExecutableDictionarySource"))
, dict_struct{dict_struct_}
, implicit_key{config.getBool(config_prefix + ".implicit_key", false)}
, command{config.getString(config_prefix + ".command")}
, update_field{config.getString(config_prefix + ".update_field", "")}
, format{config.getString(config_prefix + ".format")}
, sample_block{sample_block_}
, context(context_)
{
/// Remove keys from sample_block for implicit_key dictionary because
/// these columns will not be returned from source
/// Implicit key means that the source script will return only values,
/// and the correspondence to the requested keys is determined implicitly - by the order of rows in the result.
if (implicit_key)
{
auto keys_names = dict_struct.getKeysNames();
for (auto & key_name : keys_names)
{
size_t key_column_position_in_block = sample_block.getPositionByName(key_name);
sample_block.erase(key_column_position_in_block);
}
}
}
ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other)
: log(&Poco::Logger::get("ExecutableDictionarySource"))
, update_time{other.update_time}
, dict_struct{other.dict_struct}
, implicit_key{other.implicit_key}
, command{other.command}
, update_field{other.update_field}
, format{other.format}
@ -87,6 +105,9 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar
BlockInputStreamPtr ExecutableDictionarySource::loadAll()
{
if (implicit_key)
throw Exception("ExecutableDictionarySource with implicit_key does not support loadAll method", ErrorCodes::UNSUPPORTED_METHOD);
LOG_TRACE(log, "loadAll {}", toString());
auto process = ShellCommand::execute(command);
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
@ -95,6 +116,9 @@ BlockInputStreamPtr ExecutableDictionarySource::loadAll()
BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll()
{
if (implicit_key)
throw Exception("ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method", ErrorCodes::UNSUPPORTED_METHOD);
time_t new_update_time = time(nullptr);
SCOPE_EXIT(update_time = new_update_time);
@ -173,6 +197,77 @@ namespace
std::function<void(WriteBufferFromFile &)> send_data;
ThreadFromGlobalPool thread;
};
/** A stream, adds additional columns to each block that it will read from inner stream.
*
* block_to_add rows size must be equal to final sum rows size of all inner stream blocks.
*/
class BlockInputStreamWithAdditionalColumns final: public IBlockInputStream
{
public:
BlockInputStreamWithAdditionalColumns(
Block block_to_add_,
std::unique_ptr<IBlockInputStream>&& stream_)
: block_to_add(std::move(block_to_add_))
, stream(std::move(stream_))
{
}
Block getHeader() const override
{
auto header = stream->getHeader();
if (header)
{
for (Int64 i = static_cast<Int64>(block_to_add.columns() - 1); i >= 0; --i)
header.insert(0, block_to_add.getByPosition(i).cloneEmpty());
}
return header;
}
Block readImpl() override
{
auto block = stream->read();
if (block)
{
auto block_rows = block.rows();
auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, block_rows);
if (cut_block.rows() != block_rows)
throw Exception(
"Number of rows in block to add after cut must equal to number of rows in block from inner stream",
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
for (Int64 i = static_cast<Int64>(cut_block.columns() - 1); i >= 0; --i)
block.insert(0, cut_block.getByPosition(i));
current_range_index += block_rows;
}
return block;
}
void readPrefix() override
{
stream->readPrefix();
}
void readSuffix() override
{
stream->readSuffix();
}
String getName() const override { return "BlockInputStreamWithAdditionalColumns"; }
private:
Block block_to_add;
std::unique_ptr<IBlockInputStream> stream;
size_t current_range_index = 0;
};
}
@ -180,28 +275,44 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64
{
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
return std::make_shared<BlockInputStreamWithBackgroundThread>(
auto block = blockForIds(dict_struct, ids);
auto stream = std::make_unique<BlockInputStreamWithBackgroundThread>(
context, format, sample_block, command, log,
[&ids, this](WriteBufferFromFile & out) mutable
[block, this](WriteBufferFromFile & out) mutable
{
auto output_stream = context.getOutputStream(format, out, sample_block);
formatIDs(output_stream, ids);
auto output_stream = context.getOutputStream(format, out, block.cloneEmpty());
formatBlock(output_stream, block);
out.close();
});
if (implicit_key)
{
return std::make_shared<BlockInputStreamWithAdditionalColumns>(block, std::move(stream));
}
else
return std::shared_ptr<BlockInputStreamWithBackgroundThread>(stream.release());
}
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
return std::make_shared<BlockInputStreamWithBackgroundThread>(
auto block = blockForKeys(dict_struct, key_columns, requested_rows);
auto stream = std::make_unique<BlockInputStreamWithBackgroundThread>(
context, format, sample_block, command, log,
[key_columns, &requested_rows, this](WriteBufferFromFile & out) mutable
[block, this](WriteBufferFromFile & out) mutable
{
auto output_stream = context.getOutputStream(format, out, sample_block);
formatKeys(dict_struct, output_stream, key_columns, requested_rows);
auto output_stream = context.getOutputStream(format, out, block.cloneEmpty());
formatBlock(output_stream, block);
out.close();
});
if (implicit_key)
return std::make_shared<BlockInputStreamWithAdditionalColumns>(block, std::move(stream));
else
return std::shared_ptr<BlockInputStreamWithBackgroundThread>(stream.release());
}
bool ExecutableDictionarySource::isModified() const

View File

@ -49,9 +49,9 @@ public:
private:
Poco::Logger * log;
time_t update_time = 0;
const DictionaryStructure dict_struct;
bool implicit_key;
const std::string command;
const std::string update_field;
const std::string format;

View File

@ -131,11 +131,13 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector<UInt64> & id
{
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr)
auto block = blockForIds(dict_struct, ids);
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr)
{
WriteBufferFromOStream out_buffer(ostr);
auto output_stream = context.getOutputStream(format, out_buffer, sample_block);
formatIDs(output_stream, ids);
formatBlock(output_stream, block);
};
Poco::URI uri(url);
@ -150,11 +152,13 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns,
{
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr)
auto block = blockForKeys(dict_struct, key_columns, requested_rows);
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr)
{
WriteBufferFromOStream out_buffer(ostr);
auto output_stream = context.getOutputStream(format, out_buffer, sample_block);
formatKeys(dict_struct, output_stream, key_columns, requested_rows);
formatBlock(output_stream, block);
};
Poco::URI uri(url);

View File

@ -3,8 +3,10 @@
#include <cmath>
#include <type_traits>
#include <Common/Exception.h>
#include <Common/NaNUtils.h>
#include <DataTypes/NumberTraits.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#endif
@ -87,7 +89,19 @@ struct DivideIntegralImpl
return static_cast<Result>(checkedDivision(static_cast<SignedCastA>(a), static_cast<SignedCastB>(b)));
}
else
{
if constexpr (std::is_floating_point_v<A>)
if (isNaN(a) || a > std::numeric_limits<CastA>::max() || a < std::numeric_limits<CastA>::lowest())
throw Exception("Cannot perform integer division on infinite or too large floating point numbers",
ErrorCodes::ILLEGAL_DIVISION);
if constexpr (std::is_floating_point_v<B>)
if (isNaN(b) || b > std::numeric_limits<CastB>::max() || b < std::numeric_limits<CastB>::lowest())
throw Exception("Cannot perform integer division on infinite or too large floating point numbers",
ErrorCodes::ILLEGAL_DIVISION);
return static_cast<Result>(checkedDivision(CastA(a), CastB(b)));
}
}
#if USE_EMBEDDED_COMPILER
@ -114,6 +128,16 @@ struct ModuloImpl
}
else
{
if constexpr (std::is_floating_point_v<A>)
if (isNaN(a) || a > std::numeric_limits<IntegerAType>::max() || a < std::numeric_limits<IntegerAType>::lowest())
throw Exception("Cannot perform integer division on infinite or too large floating point numbers",
ErrorCodes::ILLEGAL_DIVISION);
if constexpr (std::is_floating_point_v<B>)
if (isNaN(b) || b > std::numeric_limits<IntegerBType>::max() || b < std::numeric_limits<IntegerBType>::lowest())
throw Exception("Cannot perform integer division on infinite or too large floating point numbers",
ErrorCodes::ILLEGAL_DIVISION);
throwIfDivisionLeadsToFPE(IntegerAType(a), IntegerBType(b));
if constexpr (is_big_int_v<IntegerAType> || is_big_int_v<IntegerBType>)

View File

@ -68,12 +68,12 @@ struct AddSecondsImpl : public AddOnDateTime64DefaultImpl<AddSecondsImpl>
static constexpr auto name = "addSeconds";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
{
return t + delta;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum(d)) + delta;
}
@ -92,7 +92,7 @@ struct AddMinutesImpl : public AddOnDateTime64DefaultImpl<AddMinutesImpl>
return t + delta * 60;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum(d)) + delta * 60;
}
@ -111,7 +111,7 @@ struct AddHoursImpl : public AddOnDateTime64DefaultImpl<AddHoursImpl>
return t + delta * 3600;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum(d)) + delta * 3600;
}
@ -125,18 +125,12 @@ struct AddDaysImpl : public AddOnDateTime64DefaultImpl<AddDaysImpl>
static constexpr auto name = "addDays";
// static inline UInt32 execute(UInt64 t, Int64 delta, const DateLUTImpl & time_zone)
// {
// // TODO (nemkov): LUT does not support out-of range date values for now.
// return time_zone.addDays(t, delta);
// }
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addDays(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
{
return d + delta;
}
@ -155,7 +149,7 @@ struct AddWeeksImpl : public AddOnDateTime64DefaultImpl<AddWeeksImpl>
return time_zone.addWeeks(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
{
return d + delta * 7;
}

View File

@ -263,6 +263,12 @@ public:
static constexpr auto name = "IPv6StringToNum";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv6StringToNum>(); }
static inline bool tryParseIPv4(const char * pos)
{
UInt32 result = 0;
return DB::parseIPv4(pos, reinterpret_cast<unsigned char *>(&result));
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
@ -270,8 +276,8 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeFixedString>(IPV6_BINARY_LENGTH);
}
@ -292,13 +298,27 @@ public:
const ColumnString::Chars & vec_src = col_in->getChars();
const ColumnString::Offsets & offsets_src = col_in->getOffsets();
size_t src_offset = 0;
char src_ipv4_buf[sizeof("::ffff:") + IPV4_MAX_TEXT_LENGTH + 1] = "::ffff:";
for (size_t out_offset = 0, i = 0;
out_offset < vec_res.size();
out_offset += IPV6_BINARY_LENGTH, ++i)
for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += IPV6_BINARY_LENGTH, ++i)
{
/// In case of failure, the function fills vec_res with zero bytes.
parseIPv6(reinterpret_cast<const char *>(&vec_src[src_offset]), reinterpret_cast<unsigned char *>(&vec_res[out_offset]));
/// For both cases below: In case of failure, the function parseIPv6 fills vec_res with zero bytes.
/// If the source IP address is parsable as an IPv4 address, then transform it into a valid IPv6 address.
/// Keeping it simple by just prefixing `::ffff:` to the IPv4 address to represent it as a valid IPv6 address.
if (tryParseIPv4(reinterpret_cast<const char *>(&vec_src[src_offset])))
{
std::memcpy(
src_ipv4_buf + std::strlen("::ffff:"),
reinterpret_cast<const char *>(&vec_src[src_offset]),
std::min<UInt64>(offsets_src[i] - src_offset, IPV4_MAX_TEXT_LENGTH + 1));
parseIPv6(src_ipv4_buf, reinterpret_cast<unsigned char *>(&vec_res[out_offset]));
}
else
{
parseIPv6(
reinterpret_cast<const char *>(&vec_src[src_offset]), reinterpret_cast<unsigned char *>(&vec_res[out_offset]));
}
src_offset = offsets_src[i];
}

View File

@ -465,7 +465,7 @@ std::vector<size_t> buildKMPPrefixFunction(const SliceType & pattern, const Equa
for (size_t i = 1; i < pattern.size; ++i)
{
result[i] = 0;
for (auto length = i; length > 0;)
for (size_t length = i; length > 0;)
{
length = result[length - 1];
if (isEqualFunc(pattern, i, length))
@ -695,7 +695,7 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source,
if (size >= 0)
{
auto length = static_cast<size_t>(size);
size_t length = static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
@ -711,7 +711,7 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source,
}
else
{
auto length = static_cast<size_t>(-size);
size_t length = -static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
@ -744,7 +744,7 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source
if (size >= 0)
{
auto length = static_cast<size_t>(size);
size_t length = static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
@ -760,7 +760,7 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source
}
else
{
auto length = static_cast<size_t>(-size);
size_t length = -static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);

View File

@ -135,15 +135,16 @@ struct LowerUpperUTF8Impl
{
static const Poco::UTF8Encoding utf8;
int src_sequence_length = UTF8::seqLength(*src);
size_t src_sequence_length = UTF8::seqLength(*src);
int src_code_point = UTF8::queryConvert(src, src_end - src);
if (src_code_point > 0)
auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src);
if (src_code_point)
{
int dst_code_point = to_case(src_code_point);
int dst_code_point = to_case(*src_code_point);
if (dst_code_point > 0)
{
int dst_sequence_length = UTF8::convert(dst_code_point, dst, src_end - src);
size_t dst_sequence_length = UTF8::convertCodePointToUTF8(dst_code_point, dst, src_end - src);
assert(dst_sequence_length <= 4);
/// We don't support cases when lowercase and uppercase characters occupy different number of bytes in UTF-8.
/// As an example, this happens for ß and ẞ.
@ -156,7 +157,9 @@ struct LowerUpperUTF8Impl
}
}
*dst++ = *src++;
*dst = *src;
++dst;
++src;
}
}

View File

@ -168,7 +168,6 @@ namespace MultiRegexps
hs_database_t * db = nullptr;
hs_compile_error_t * compile_error;
std::unique_ptr<unsigned int[]> ids;
/// We mark the patterns to provide the callback results.

View File

@ -3,6 +3,7 @@
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Common/NaNUtils.h>
#include <DataTypes/DataTypeString.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/WriteHelpers.h>
@ -134,24 +135,32 @@ public:
/// Virtual call is Ok (negligible comparing to the rest of calculations).
Float64 value = arguments[0].column->getFloat64(i);
bool is_negative = value < 0;
if (is_negative)
if (!isFinite(value))
{
writeChar('-', buf_to);
value = -value;
/// Cannot decide what unit it is (years, month), just simply write inf or nan.
writeFloatText(value, buf_to);
}
/// To output separators between parts: ", " and " and ".
bool has_output = false;
switch (max_unit) /// A kind of Duff Device.
else
{
case Years: processUnit(365 * 24 * 3600, " year", 5, value, buf_to, has_output); [[fallthrough]];
case Months: processUnit(30.5 * 24 * 3600, " month", 6, value, buf_to, has_output); [[fallthrough]];
case Days: processUnit(24 * 3600, " day", 4, value, buf_to, has_output); [[fallthrough]];
case Hours: processUnit(3600, " hour", 5, value, buf_to, has_output); [[fallthrough]];
case Minutes: processUnit(60, " minute", 7, value, buf_to, has_output); [[fallthrough]];
case Seconds: processUnit(1, " second", 7, value, buf_to, has_output);
bool is_negative = value < 0;
if (is_negative)
{
writeChar('-', buf_to);
value = -value;
}
/// To output separators between parts: ", " and " and ".
bool has_output = false;
switch (max_unit) /// A kind of Duff Device.
{
case Years: processUnit(365 * 24 * 3600, " year", 5, value, buf_to, has_output); [[fallthrough]];
case Months: processUnit(30.5 * 24 * 3600, " month", 6, value, buf_to, has_output); [[fallthrough]];
case Days: processUnit(24 * 3600, " day", 4, value, buf_to, has_output); [[fallthrough]];
case Hours: processUnit(3600, " hour", 5, value, buf_to, has_output); [[fallthrough]];
case Minutes: processUnit(60, " minute", 7, value, buf_to, has_output); [[fallthrough]];
case Seconds: processUnit(1, " second", 7, value, buf_to, has_output);
}
}
writeChar(0, buf_to);

View File

@ -119,8 +119,13 @@ public:
UInt32 code_point2 = generate_code_point(rand >> 32);
/// We have padding in column buffers that we can overwrite.
pos += UTF8::convert(code_point1, pos, sizeof(int));
last_writen_bytes = UTF8::convert(code_point2, pos, sizeof(int));
size_t length1 = UTF8::convertCodePointToUTF8(code_point1, pos, sizeof(int));
assert(length1 <= 4);
pos += length1;
size_t length2 = UTF8::convertCodePointToUTF8(code_point2, pos, sizeof(int));
assert(length2 <= 4);
last_writen_bytes = length2;
pos += last_writen_bytes;
}
offset = pos - data_to.data() + 1;

View File

@ -683,7 +683,7 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
/** CSV format can contain insignificant spaces and tabs.
* Usually the task of skipping them is for the calling code.
* But in this case, it will be difficult to do this, so remove the trailing whitespace by yourself.
* But in this case, it will be difficult to do this, so remove the trailing whitespace by ourself.
*/
size_t size = s.size();
while (size > 0

View File

@ -341,6 +341,7 @@ struct ContextShared
mutable std::optional<BackgroundSchedulePool> buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables.
mutable std::optional<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
mutable std::optional<BackgroundSchedulePool> distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends)
mutable std::optional<BackgroundSchedulePool> message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka)
MultiVersion<Macros> macros; /// Substitutions extracted from config.
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
@ -439,6 +440,7 @@ struct ContextShared
buffer_flush_schedule_pool.reset();
schedule_pool.reset();
distributed_schedule_pool.reset();
message_broker_schedule_pool.reset();
ddl_worker.reset();
/// Stop trace collector if any
@ -1525,6 +1527,17 @@ BackgroundSchedulePool & Context::getDistributedSchedulePool() const
return *shared->distributed_schedule_pool;
}
BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const
{
auto lock = getLock();
if (!shared->message_broker_schedule_pool)
shared->message_broker_schedule_pool.emplace(
settings.background_message_broker_schedule_pool_size,
CurrentMetrics::BackgroundDistributedSchedulePoolTask,
"BgMsgBrkSchPool");
return *shared->message_broker_schedule_pool;
}
bool Context::hasDistributedDDL() const
{
return getConfigRef().has("distributed_ddl");

View File

@ -611,6 +611,7 @@ public:
BackgroundSchedulePool & getBufferFlushSchedulePool() const;
BackgroundSchedulePool & getSchedulePool() const;
BackgroundSchedulePool & getMessageBrokerSchedulePool() const;
BackgroundSchedulePool & getDistributedSchedulePool() const;
/// Has distributed_ddl configuration or not.

View File

@ -26,11 +26,18 @@ void PredicateRewriteVisitorData::visit(ASTSelectWithUnionQuery & union_select_q
{
auto & internal_select_list = union_select_query.list_of_selects->children;
if (!internal_select_list.empty())
visitFirstInternalSelect(*internal_select_list[0]->as<ASTSelectQuery>(), internal_select_list[0]);
for (size_t index = 1; index < internal_select_list.size(); ++index)
visitOtherInternalSelect(*internal_select_list[index]->as<ASTSelectQuery>(), internal_select_list[index]);
for (size_t index = 0; index < internal_select_list.size(); ++index)
{
if (auto * child_union = internal_select_list[index]->as<ASTSelectWithUnionQuery>())
visit(*child_union, internal_select_list[index]);
else
{
if (index == 0)
visitFirstInternalSelect(*internal_select_list[0]->as<ASTSelectQuery>(), internal_select_list[0]);
else
visitOtherInternalSelect(*internal_select_list[index]->as<ASTSelectQuery>(), internal_select_list[index]);
}
}
}
void PredicateRewriteVisitorData::visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &)

View File

@ -140,7 +140,16 @@ void Set::setHeader(const Block & header)
ConstNullMapPtr null_map{};
ColumnPtr null_map_holder;
if (!transform_null_in)
{
/// We convert nullable columns to non nullable we also need to update nullable types
for (size_t i = 0; i < set_elements_types.size(); ++i)
{
data_types[i] = removeNullable(data_types[i]);
set_elements_types[i] = removeNullable(set_elements_types[i]);
}
extractNestedColumnsAndNullMap(key_columns, null_map);
}
if (fill_set_elements)
{
@ -182,7 +191,7 @@ bool Set::insertFromBlock(const Block & block)
ConstNullMapPtr null_map{};
ColumnPtr null_map_holder;
if (!transform_null_in)
null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
/// Filter to extract distinct values from the block.
ColumnUInt8::MutablePtr filter;

View File

@ -8,6 +8,7 @@
#include <condition_variable>
#include <boost/noncopyable.hpp>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <common/types.h>
#include <Core/Defines.h>
#include <Storages/IStorage.h>
@ -229,9 +230,18 @@ void SystemLog<LogElement>::startup()
}
static thread_local bool recursive_add_call = false;
template <typename LogElement>
void SystemLog<LogElement>::add(const LogElement & element)
{
/// It is possible that the method will be called recursively.
/// Better to drop these events to avoid complications.
if (recursive_add_call)
return;
recursive_add_call = true;
SCOPE_EXIT({ recursive_add_call = false; });
/// Memory can be allocated while resizing on queue.push_back.
/// The size of allocation can be in order of a few megabytes.
/// But this should not be accounted for query memory usage.

View File

@ -269,6 +269,18 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (live_view_timeout)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH TIMEOUT " << (settings.hilite ? hilite_none : "")
<< *live_view_timeout;
if (live_view_periodic_refresh)
{
if (live_view_timeout)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AND" << (settings.hilite ? hilite_none : "");
else
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH" << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " PERIODIC REFRESH " << (settings.hilite ? hilite_none : "")
<< *live_view_periodic_refresh;
}
formatOnCluster(settings);
}
else

View File

@ -77,6 +77,8 @@ public:
ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.)
std::optional<UInt64> live_view_timeout; /// For CREATE LIVE VIEW ... WITH TIMEOUT ...
std::optional<UInt64> live_view_periodic_refresh; /// For CREATE LIVE VIEW ... WITH [PERIODIC] REFRESH ...
bool attach_short_syntax{false};
std::optional<String> attach_from_path = std::nullopt;

View File

@ -307,10 +307,11 @@ bool ASTSelectQuery::final() const
bool ASTSelectQuery::withFill() const
{
if (!orderBy())
const ASTPtr order_by = orderBy();
if (!order_by)
return false;
for (const auto & order_expression_element : orderBy()->children)
for (const auto & order_expression_element : order_by->children)
if (order_expression_element->as<ASTOrderByElement &>().with_fill)
return true;

View File

@ -20,7 +20,7 @@ ASTPtr ASTTTLElement::clone() const
for (auto & expr : clone->group_by_key)
expr = expr->clone();
for (auto & [name, expr] : clone->group_by_aggregations)
for (auto & expr : clone->group_by_assignments)
expr = expr->clone();
return clone;
@ -46,15 +46,15 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st
settings.ostr << ", ";
(*it)->formatImpl(settings, state, frame);
}
if (!group_by_aggregations.empty())
if (!group_by_assignments.empty())
{
settings.ostr << " SET ";
for (auto it = group_by_aggregations.begin(); it != group_by_aggregations.end(); ++it)
for (auto it = group_by_assignments.begin(); it != group_by_assignments.end(); ++it)
{
if (it != group_by_aggregations.begin())
if (it != group_by_assignments.begin())
settings.ostr << ", ";
settings.ostr << it->first << " = ";
it->second->formatImpl(settings, state, frame);
(*it)->formatImpl(settings, state, frame);
}
}
}

View File

@ -18,7 +18,7 @@ public:
String destination_name;
ASTs group_by_key;
std::vector<std::pair<String, ASTPtr>> group_by_aggregations;
ASTs group_by_assignments;
ASTPtr recompression_codec;

View File

@ -24,6 +24,7 @@
#include <Parsers/ASTTTLElement.h>
#include <Parsers/ASTWindowDefinition.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTAssignment.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseIntervalKind.h>
@ -2008,9 +2009,12 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserIdentifier parser_identifier;
ParserStringLiteral parser_string_literal;
ParserExpression parser_exp;
ParserExpressionList parser_expression_list(false);
ParserExpressionList parser_keys_list(false);
ParserCodec parser_codec;
ParserList parser_assignment_list(
std::make_unique<ParserAssignment>(), std::make_unique<ParserToken>(TokenType::Comma));
ASTPtr ttl_expr;
if (!parser_exp.parse(pos, ttl_expr, expected))
return false;
@ -2044,9 +2048,9 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
ASTPtr where_expr;
ASTPtr ast_group_by_key;
ASTPtr group_by_key;
ASTPtr recompression_codec;
std::vector<std::pair<String, ASTPtr>> group_by_aggregations;
ASTPtr group_by_assignments;
if (mode == TTLMode::MOVE)
{
@ -2058,30 +2062,13 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
else if (mode == TTLMode::GROUP_BY)
{
if (!parser_expression_list.parse(pos, ast_group_by_key, expected))
if (!parser_keys_list.parse(pos, group_by_key, expected))
return false;
if (s_set.ignore(pos))
{
while (true)
{
if (!group_by_aggregations.empty() && !s_comma.ignore(pos))
break;
ASTPtr name;
ASTPtr value;
if (!parser_identifier.parse(pos, name, expected))
return false;
if (!s_eq.ignore(pos))
return false;
if (!parser_exp.parse(pos, value, expected))
return false;
String name_str;
if (!tryGetIdentifierNameInto(name, name_str))
return false;
group_by_aggregations.emplace_back(name_str, std::move(value));
}
if (!parser_assignment_list.parse(pos, group_by_assignments, expected))
return false;
}
}
else if (mode == TTLMode::DELETE && s_where.ignore(pos))
@ -2105,8 +2092,9 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (mode == TTLMode::GROUP_BY)
{
ttl_element->group_by_key = std::move(ast_group_by_key->children);
ttl_element->group_by_aggregations = std::move(group_by_aggregations);
ttl_element->group_by_key = std::move(group_by_key->children);
if (group_by_assignments)
ttl_element->group_by_assignments = std::move(group_by_assignments->children);
}
if (mode == TTLMode::RECOMPRESS)
@ -2141,4 +2129,31 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node,
return false;
}
bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto assignment = std::make_shared<ASTAssignment>();
node = assignment;
ParserIdentifier p_identifier;
ParserToken s_equals(TokenType::Equals);
ParserExpression p_expression;
ASTPtr column;
if (!p_identifier.parse(pos, column, expected))
return false;
if (!s_equals.ignore(pos, expected))
return false;
ASTPtr expression;
if (!p_expression.parse(pos, expression, expected))
return false;
tryGetIdentifierNameInto(column, assignment->column_name);
if (expression)
assignment->children.push_back(expression);
return true;
}
}

View File

@ -483,4 +483,12 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Part of the UPDATE command or TTL with GROUP BY of the form: col_name = expr
class ParserAssignment : public IParserBase
{
protected:
const char * getName() const override{ return "column assignment"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -11,7 +11,6 @@
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTAssignment.h>
#include <Parsers/parseDatabaseAndTableName.h>
@ -651,34 +650,6 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto assignment = std::make_shared<ASTAssignment>();
node = assignment;
ParserIdentifier p_identifier;
ParserToken s_equals(TokenType::Equals);
ParserExpression p_expression;
ASTPtr column;
if (!p_identifier.parse(pos, column, expected))
return false;
if (!s_equals.ignore(pos, expected))
return false;
ASTPtr expression;
if (!p_expression.parse(pos, expression, expected))
return false;
tryGetIdentifierNameInto(column, assignment->column_name);
if (expression)
assignment->children.push_back(expression);
return true;
}
bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto query = std::make_shared<ASTAlterQuery>();

View File

@ -63,12 +63,4 @@ public:
};
/// Part of the UPDATE command of the form: col_name = expr
class ParserAssignment : public IParserBase
{
protected:
const char * getName() const override{ return "column assignment"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -569,10 +569,14 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
ASTPtr as_table;
ASTPtr select;
ASTPtr live_view_timeout;
ASTPtr live_view_periodic_refresh;
String cluster_str;
bool attach = false;
bool if_not_exists = false;
bool with_and = false;
bool with_timeout = false;
bool with_periodic_refresh = false;
if (!s_create.ignore(pos, expected))
{
@ -594,10 +598,35 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
if (!table_name_p.parse(pos, table, expected))
return false;
if (ParserKeyword{"WITH TIMEOUT"}.ignore(pos, expected))
if (ParserKeyword{"WITH"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_timeout, expected))
live_view_timeout = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC));
if (ParserKeyword{"TIMEOUT"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_timeout, expected))
{
live_view_timeout = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC));
}
/// Optional - AND
if (ParserKeyword{"AND"}.ignore(pos, expected))
with_and = true;
with_timeout = true;
}
if (ParserKeyword{"REFRESH"}.ignore(pos, expected) || ParserKeyword{"PERIODIC REFRESH"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_periodic_refresh, expected))
live_view_periodic_refresh = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC));
with_periodic_refresh = true;
}
else if (with_and)
return false;
if (!with_timeout && !with_periodic_refresh)
return false;
}
if (ParserKeyword{"ON"}.ignore(pos, expected))
@ -656,6 +685,9 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
if (live_view_timeout)
query->live_view_timeout.emplace(live_view_timeout->as<ASTLiteral &>().value.safeGet<UInt64>());
if (live_view_periodic_refresh)
query->live_view_periodic_refresh.emplace(live_view_periodic_refresh->as<ASTLiteral &>().value.safeGet<UInt64>());
return true;
}

View File

@ -4,13 +4,14 @@
namespace DB
{
UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin, Token last)
UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin)
{
/// We have just two kind of parentheses: () and [].
UnmatchedParentheses stack;
for (TokenIterator it = begin;
it.isValid() && it->begin <= last.begin; ++it)
/// We have to iterate through all tokens until the end to avoid false positive "Unmatched parentheses" error
/// when parser failed in the middle of the query.
for (TokenIterator it = begin; it.isValid(); ++it)
{
if (it->type == TokenType::OpeningRoundBracket || it->type == TokenType::OpeningSquareBracket)
{

View File

@ -80,6 +80,6 @@ public:
/// Returns positions of unmatched parentheses.
using UnmatchedParentheses = std::vector<Token>;
UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin, Token last);
UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin);
}

View File

@ -79,7 +79,7 @@ void writeQueryWithHighlightedErrorPositions(
{
const char * current_position_to_hilite = positions_to_hilite[position_to_hilite_idx].begin;
assert(current_position_to_hilite < end);
assert(current_position_to_hilite <= end);
assert(current_position_to_hilite >= begin);
out.write(pos, current_position_to_hilite - pos);
@ -290,7 +290,7 @@ ASTPtr tryParseQuery(
}
/// Unmatched parentheses
UnmatchedParentheses unmatched_parens = checkUnmatchedParentheses(TokenIterator(tokens), last_token);
UnmatchedParentheses unmatched_parens = checkUnmatchedParentheses(TokenIterator(tokens));
if (!unmatched_parens.empty())
{
out_error_message = getUnmatchedParenthesesErrorMessage(query_begin,

View File

@ -105,6 +105,8 @@ Pipe::Holder & Pipe::Holder::operator=(Holder && rhs)
for (auto & plan : rhs.query_plans)
query_plans.emplace_back(std::move(plan));
query_id_holder = std::move(rhs.query_id_holder);
return *this;
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Processors/IProcessor.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/QueryPlan/QueryIdHolder.h>
#include <Processors/QueryPlan/QueryPlan.h>
namespace DB
@ -108,6 +109,7 @@ public:
/// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible.
void addInterpreterContext(std::shared_ptr<Context> context) { holder.interpreter_context.emplace_back(std::move(context)); }
void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); }
void addQueryIdHolder(std::shared_ptr<QueryIdHolder> query_id_holder) { holder.query_id_holder = std::move(query_id_holder); }
/// For queries with nested interpreters (i.e. StorageDistributed)
void addQueryPlan(std::unique_ptr<QueryPlan> plan) { holder.query_plans.emplace_back(std::move(plan)); }
@ -128,6 +130,7 @@ private:
std::vector<StoragePtr> storage_holders;
std::vector<TableLockHolder> table_locks;
std::vector<std::unique_ptr<QueryPlan>> query_plans;
std::shared_ptr<QueryIdHolder> query_id_holder;
};
Holder holder;

View File

@ -0,0 +1,15 @@
#include <Processors/QueryPlan/QueryIdHolder.h>
#include <Storages/MergeTree/MergeTreeData.h>
namespace DB
{
QueryIdHolder::QueryIdHolder(const String & query_id_, const MergeTreeData & data_) : query_id(query_id_), data(data_)
{
}
QueryIdHolder::~QueryIdHolder()
{
data.removeQueryId(query_id);
}
}

View File

@ -0,0 +1,21 @@
#pragma once
#include <string>
namespace DB
{
class MergeTreeData;
/// Holds the current query id and do something meaningful in destructor.
/// Currently it's used for cleaning query id in the MergeTreeData query set.
struct QueryIdHolder
{
QueryIdHolder(const std::string & query_id_, const MergeTreeData & data_);
~QueryIdHolder();
std::string query_id;
const MergeTreeData & data;
};
}

View File

@ -122,6 +122,7 @@ SRCS(
QueryPlan/Optimizations/optimizeTree.cpp
QueryPlan/Optimizations/splitFilter.cpp
QueryPlan/PartialSortingStep.cpp
QueryPlan/QueryIdHolder.cpp
QueryPlan/QueryPlan.cpp
QueryPlan/ReadFromPreparedSource.cpp
QueryPlan/ReadNothingStep.cpp

View File

@ -109,15 +109,16 @@ public:
* because it allows to check the IP ranges of the trusted proxy.
* Proxy-forwarded (original client) IP address is used for quota accounting if quota is keyed by forwarded IP.
*/
TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_)
TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_,
std::string server_display_name_)
: Poco::Net::TCPServerConnection(socket_)
, server(server_)
, parse_proxy_protocol(parse_proxy_protocol_)
, log(&Poco::Logger::get("TCPHandler"))
, connection_context(server.context())
, query_context(server.context())
, server_display_name(std::move(server_display_name_))
{
server_display_name = server.config().getString("display_name", getFQDNOrHostName());
}
void run() override;

View File

@ -17,6 +17,7 @@ private:
IServer & server;
bool parse_proxy_protocol = false;
Poco::Logger * log;
std::string server_display_name;
class DummyTCPHandler : public Poco::Net::TCPServerConnection
{
@ -34,6 +35,7 @@ public:
: server(server_), parse_proxy_protocol(parse_proxy_protocol_)
, log(&Poco::Logger::get(std::string("TCP") + (secure_ ? "S" : "") + "HandlerFactory"))
{
server_display_name = server.config().getString("display_name", getFQDNOrHostName());
}
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override
@ -42,7 +44,7 @@ public:
{
LOG_TRACE(log, "TCP Request. Address: {}", socket.peerAddress().toString());
return new TCPHandler(server, socket, parse_proxy_protocol);
return new TCPHandler(server, socket, parse_proxy_protocol, server_display_name);
}
catch (const Poco::Net::NetException &)
{

View File

@ -196,7 +196,7 @@ StorageKafka::StorageKafka(
auto task_count = thread_per_consumer ? num_consumers : 1;
for (size_t i = 0; i < task_count; ++i)
{
auto task = global_context.getSchedulePool().createTask(log->name(), [this, i]{ threadFunc(i); });
auto task = global_context.getMessageBrokerSchedulePool().createTask(log->name(), [this, i]{ threadFunc(i); });
task->deactivate();
tasks.emplace_back(std::make_shared<TaskContext>(std::move(task)));
}

View File

@ -34,6 +34,7 @@ public:
{
new_blocks_metadata->hash = key_str;
new_blocks_metadata->version = storage.getBlocksVersion() + 1;
new_blocks_metadata->time = std::chrono::system_clock::now();
for (auto & block : *new_blocks)
{
@ -48,6 +49,15 @@ public:
storage.condition.notify_all();
}
else
{
// only update blocks time
new_blocks_metadata->hash = storage.getBlocksHashKey();
new_blocks_metadata->version = storage.getBlocksVersion();
new_blocks_metadata->time = std::chrono::system_clock::now();
(*storage.blocks_metadata_ptr) = new_blocks_metadata;
}
new_blocks.reset();
new_blocks_metadata.reset();

View File

@ -20,6 +20,7 @@ limitations under the License. */
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataStreams/SquashingBlockInputStream.h>
#include <DataStreams/copyData.h>
#include <common/logger_useful.h>
#include <Common/typeid_cast.h>
#include <Common/SipHash.h>
@ -254,6 +255,8 @@ StorageLiveView::StorageLiveView(
live_view_context = std::make_unique<Context>(global_context);
live_view_context->makeQueryContext();
log = &Poco::Logger::get("StorageLiveView (" + table_id_.database_name + "." + table_id_.table_name + ")");
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
setInMemoryMetadata(storage_metadata);
@ -275,12 +278,21 @@ StorageLiveView::StorageLiveView(
if (query.live_view_timeout)
{
is_temporary = true;
temporary_live_view_timeout = std::chrono::seconds{*query.live_view_timeout};
temporary_live_view_timeout = Seconds {*query.live_view_timeout};
}
if (query.live_view_periodic_refresh)
{
is_periodically_refreshed = true;
periodic_live_view_refresh = Seconds {*query.live_view_periodic_refresh};
}
blocks_ptr = std::make_shared<BlocksPtr>();
blocks_metadata_ptr = std::make_shared<BlocksMetadataPtr>();
active_ptr = std::make_shared<bool>(true);
periodic_refresh_task = global_context.getSchedulePool().createTask("LieViewPeriodicRefreshTask", [this]{ periodicRefreshTaskFunc(); });
periodic_refresh_task->deactivate();
}
Block StorageLiveView::getHeader() const
@ -369,10 +381,21 @@ bool StorageLiveView::getNewBlocks()
}
new_blocks_metadata->hash = key.toHexString();
new_blocks_metadata->version = getBlocksVersion() + 1;
new_blocks_metadata->time = std::chrono::system_clock::now();
(*blocks_ptr) = new_blocks;
(*blocks_metadata_ptr) = new_blocks_metadata;
updated = true;
}
else
{
new_blocks_metadata->hash = getBlocksHashKey();
new_blocks_metadata->version = getBlocksVersion();
new_blocks_metadata->time = std::chrono::system_clock::now();
(*blocks_metadata_ptr) = new_blocks_metadata;
}
}
return updated;
}
@ -392,11 +415,18 @@ void StorageLiveView::startup()
{
if (is_temporary)
TemporaryLiveViewCleaner::instance().addView(std::static_pointer_cast<StorageLiveView>(shared_from_this()));
if (is_periodically_refreshed)
periodic_refresh_task->activate();
}
void StorageLiveView::shutdown()
{
shutdown_called = true;
if (is_periodically_refreshed)
periodic_refresh_task->deactivate();
DatabaseCatalog::instance().removeDependency(select_table_id, getStorageID());
}
@ -415,15 +445,55 @@ void StorageLiveView::drop()
condition.notify_all();
}
void StorageLiveView::refresh()
void StorageLiveView::scheduleNextPeriodicRefresh()
{
Seconds current_time = std::chrono::duration_cast<Seconds> (std::chrono::system_clock::now().time_since_epoch());
Seconds blocks_time = std::chrono::duration_cast<Seconds> (getBlocksTime().time_since_epoch());
if ((current_time - periodic_live_view_refresh) >= blocks_time)
{
refresh(false);
blocks_time = std::chrono::duration_cast<Seconds> (getBlocksTime().time_since_epoch());
}
current_time = std::chrono::duration_cast<Seconds> (std::chrono::system_clock::now().time_since_epoch());
auto next_refresh_time = blocks_time + periodic_live_view_refresh;
if (current_time >= next_refresh_time)
periodic_refresh_task->scheduleAfter(0);
else
{
auto schedule_time = std::chrono::duration_cast<MilliSeconds> (next_refresh_time - current_time);
periodic_refresh_task->scheduleAfter(static_cast<size_t>(schedule_time.count()));
}
}
void StorageLiveView::periodicRefreshTaskFunc()
{
LOG_TRACE(log, "periodic refresh task");
std::lock_guard lock(mutex);
if (hasActiveUsers())
scheduleNextPeriodicRefresh();
}
void StorageLiveView::refresh(bool grab_lock)
{
// Lock is already acquired exclusively from InterperterAlterQuery.cpp InterpreterAlterQuery::execute() method.
// So, reacquiring lock is not needed and will result in an exception.
if (grab_lock)
{
std::lock_guard lock(mutex);
if (getNewBlocks())
condition.notify_all();
}
else
{
if (getNewBlocks())
condition.notify_all();
}
}
Pipe StorageLiveView::read(
@ -435,15 +505,21 @@ Pipe StorageLiveView::read(
const size_t /*max_block_size*/,
const unsigned /*num_streams*/)
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
refresh(false);
else if (is_periodically_refreshed)
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
{
if (getNewBlocks())
condition.notify_all();
}
return Pipe(std::make_shared<BlocksSource>(blocks_ptr, getHeader()));
Seconds current_time = std::chrono::duration_cast<Seconds> (std::chrono::system_clock::now().time_since_epoch());
Seconds blocks_time = std::chrono::duration_cast<Seconds> (getBlocksTime().time_since_epoch());
if ((current_time - periodic_live_view_refresh) >= blocks_time)
refresh(false);
}
return Pipe(std::make_shared<BlocksSource>(blocks_ptr, getHeader()));
}
BlockInputStreams StorageLiveView::watch(
@ -458,6 +534,7 @@ BlockInputStreams StorageLiveView::watch(
bool has_limit = false;
UInt64 limit = 0;
BlockInputStreamPtr reader;
if (query.limit_length)
{
@ -466,45 +543,28 @@ BlockInputStreams StorageLiveView::watch(
}
if (query.is_watch_events)
{
auto reader = std::make_shared<LiveViewEventsBlockInputStream>(
reader = std::make_shared<LiveViewEventsBlockInputStream>(
std::static_pointer_cast<StorageLiveView>(shared_from_this()),
blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit,
context.getSettingsRef().live_view_heartbeat_interval.totalSeconds());
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
{
if (getNewBlocks())
condition.notify_all();
}
}
processed_stage = QueryProcessingStage::Complete;
return { reader };
}
else
{
auto reader = std::make_shared<LiveViewBlockInputStream>(
reader = std::make_shared<LiveViewBlockInputStream>(
std::static_pointer_cast<StorageLiveView>(shared_from_this()),
blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit,
context.getSettingsRef().live_view_heartbeat_interval.totalSeconds());
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
{
if (getNewBlocks())
condition.notify_all();
}
}
{
std::lock_guard lock(mutex);
processed_stage = QueryProcessingStage::Complete;
if (!(*blocks_ptr))
refresh(false);
return { reader };
if (is_periodically_refreshed)
scheduleNextPeriodicRefresh();
}
processed_stage = QueryProcessingStage::Complete;
return { reader };
}
NamesAndTypesList StorageLiveView::getVirtuals() const

View File

@ -13,6 +13,7 @@ limitations under the License. */
#include <ext/shared_ptr_helper.h>
#include <Storages/IStorage.h>
#include <Core/BackgroundSchedulePool.h>
#include <mutex>
#include <condition_variable>
@ -21,10 +22,16 @@ limitations under the License. */
namespace DB
{
using Time = std::chrono::time_point<std::chrono::system_clock>;
using Seconds = std::chrono::seconds;
using MilliSeconds = std::chrono::milliseconds;
struct BlocksMetadata
{
String hash;
UInt64 version;
Time time;
};
struct MergeableBlocks
@ -75,8 +82,10 @@ public:
NamesAndTypesList getVirtuals() const override;
bool isTemporary() const { return is_temporary; }
std::chrono::seconds getTimeout() const { return temporary_live_view_timeout; }
bool isPeriodicallyRefreshed() const { return is_periodically_refreshed; }
Seconds getTimeout() const { return temporary_live_view_timeout; }
Seconds getPeriodicRefresh() const { return periodic_live_view_refresh; }
/// Check if we have any readers
/// must be called with mutex locked
@ -109,6 +118,15 @@ public:
return 0;
}
/// Get blocks time
/// must be called with mutex locked
Time getBlocksTime()
{
if (*blocks_metadata_ptr)
return (*blocks_metadata_ptr)->time;
return {};
}
/// Reset blocks
/// must be called with mutex locked
void reset()
@ -124,7 +142,7 @@ public:
void startup() override;
void shutdown() override;
void refresh();
void refresh(const bool grab_lock = true);
Pipe read(
const Names & column_names,
@ -176,8 +194,13 @@ private:
Context & global_context;
std::unique_ptr<Context> live_view_context;
Poco::Logger * log;
bool is_temporary = false;
std::chrono::seconds temporary_live_view_timeout;
bool is_periodically_refreshed = false;
Seconds temporary_live_view_timeout;
Seconds periodic_live_view_refresh;
/// Mutex to protect access to sample block and inner_blocks_query
mutable std::mutex sample_block_lock;
@ -199,6 +222,13 @@ private:
std::atomic<bool> shutdown_called = false;
/// Periodic refresh task used when [PERIODIC] REFRESH is specified in create statement
BackgroundSchedulePool::TaskHolder periodic_refresh_task;
void periodicRefreshTaskFunc();
/// Must be called with mutex locked
void scheduleNextPeriodicRefresh();
StorageLiveView(
const StorageID & table_id_,
Context & local_context,

View File

@ -1278,6 +1278,18 @@ bool IMergeTreeDataPart::checkAllTTLCalculated(const StorageMetadataPtr & metada
return false;
}
for (const auto & group_by_desc : metadata_snapshot->getGroupByTTLs())
{
if (!ttl_infos.group_by_ttl.count(group_by_desc.result_column))
return false;
}
for (const auto & rows_where_desc : metadata_snapshot->getRowsWhereTTLs())
{
if (!ttl_infos.rows_where_ttl.count(rows_where_desc.result_column))
return false;
}
return true;
}

View File

@ -114,6 +114,7 @@ namespace ErrorCodes
extern const int NOT_ENOUGH_SPACE;
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
extern const int SUPPORT_IS_DISABLED;
extern const int TOO_MANY_SIMULTANEOUS_QUERIES;
}
@ -3988,4 +3989,24 @@ void MergeTreeData::setDataVolume(size_t bytes, size_t rows, size_t parts)
total_active_size_rows.store(rows, std::memory_order_release);
total_active_size_parts.store(parts, std::memory_order_release);
}
void MergeTreeData::insertQueryIdOrThrow(const String & query_id, size_t max_queries) const
{
std::lock_guard lock(query_id_set_mutex);
if (query_id_set.find(query_id) != query_id_set.end())
return;
if (query_id_set.size() >= max_queries)
throw Exception(
ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries for table {}. Maximum is: {}", log_name, max_queries);
query_id_set.insert(query_id);
}
void MergeTreeData::removeQueryId(const String & query_id) const
{
std::lock_guard lock(query_id_set_mutex);
if (query_id_set.find(query_id) == query_id_set.end())
LOG_WARNING(log, "We have query_id removed but it's not recorded. This is a bug");
else
query_id_set.erase(query_id);
}
}

View File

@ -702,6 +702,12 @@ public:
/// section from config.xml.
CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const;
/// Record current query id where querying the table. Throw if there are already `max_queries` queries accessing the same table.
void insertQueryIdOrThrow(const String & query_id, size_t max_queries) const;
/// Remove current query id after query finished.
void removeQueryId(const String & query_id) const;
/// Limiting parallel sends per one table, used in DataPartsExchange
std::atomic_uint current_table_sends {0};
@ -958,6 +964,10 @@ private:
std::atomic<size_t> total_active_size_bytes = 0;
std::atomic<size_t> total_active_size_rows = 0;
std::atomic<size_t> total_active_size_parts = 0;
// Record all query ids which access the table. It's guarded by `query_id_set_mutex` and is always mutable.
mutable std::set<String> query_id_set;
mutable std::mutex query_id_set_mutex;
};
}

View File

@ -17,13 +17,23 @@ void MergeTreeDataPartTTLInfos::update(const MergeTreeDataPartTTLInfos & other_i
updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
}
for (const auto & [name, ttl_info] : other_infos.rows_where_ttl)
{
rows_where_ttl[name].update(ttl_info);
updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
}
for (const auto & [name, ttl_info] : other_infos.group_by_ttl)
{
group_by_ttl[name].update(ttl_info);
updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
}
for (const auto & [name, ttl_info] : other_infos.recompression_ttl)
recompression_ttl[name].update(ttl_info);
for (const auto & [expression, ttl_info] : other_infos.moves_ttl)
{
moves_ttl[expression].update(ttl_info);
}
table_ttl.update(other_infos.table_ttl);
updatePartMinMaxTTL(table_ttl.min, table_ttl.max);
@ -59,29 +69,41 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in)
updatePartMinMaxTTL(table_ttl.min, table_ttl.max);
}
auto fill_ttl_info_map = [this](const JSON & json_part, TTLInfoMap & ttl_info_map, bool update_min_max)
{
for (auto elem : json_part) // NOLINT
{
MergeTreeDataPartTTLInfo ttl_info;
ttl_info.min = elem["min"].getUInt();
ttl_info.max = elem["max"].getUInt();
String expression = elem["expression"].getString();
ttl_info_map.emplace(expression, ttl_info);
if (update_min_max)
updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
}
};
if (json.has("moves"))
{
const JSON & moves = json["moves"];
for (auto move : moves) // NOLINT
{
MergeTreeDataPartTTLInfo ttl_info;
ttl_info.min = move["min"].getUInt();
ttl_info.max = move["max"].getUInt();
String expression = move["expression"].getString();
moves_ttl.emplace(expression, ttl_info);
}
fill_ttl_info_map(moves, moves_ttl, false);
}
if (json.has("recompression"))
{
const JSON & recompressions = json["recompression"];
for (auto recompression : recompressions) // NOLINT
{
MergeTreeDataPartTTLInfo ttl_info;
ttl_info.min = recompression["min"].getUInt();
ttl_info.max = recompression["max"].getUInt();
String expression = recompression["expression"].getString();
recompression_ttl.emplace(expression, ttl_info);
}
fill_ttl_info_map(recompressions, recompression_ttl, false);
}
if (json.has("group_by"))
{
const JSON & group_by = json["group_by"];
fill_ttl_info_map(group_by, group_by_ttl, true);
}
if (json.has("rows_where"))
{
const JSON & rows_where = json["rows_where"];
fill_ttl_info_map(rows_where, rows_where_ttl, true);
}
}
@ -118,47 +140,52 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const
writeIntText(table_ttl.max, out);
writeString("}", out);
}
auto write_infos = [&out](const TTLInfoMap & infos, const String & type, bool is_first)
{
if (!is_first)
writeString(",", out);
writeDoubleQuotedString(type, out);
writeString(":[", out);
for (auto it = infos.begin(); it != infos.end(); ++it)
{
if (it != infos.begin())
writeString(",", out);
writeString(R"({"expression":)", out);
writeString(doubleQuoteString(it->first), out);
writeString(R"(,"min":)", out);
writeIntText(it->second.min, out);
writeString(R"(,"max":)", out);
writeIntText(it->second.max, out);
writeString("}", out);
}
writeString("]", out);
};
bool is_first = columns_ttl.empty() && !table_ttl.min;
if (!moves_ttl.empty())
{
if (!columns_ttl.empty() || table_ttl.min)
writeString(",", out);
writeString(R"("moves":[)", out);
for (auto it = moves_ttl.begin(); it != moves_ttl.end(); ++it)
{
if (it != moves_ttl.begin())
writeString(",", out);
writeString(R"({"expression":)", out);
writeString(doubleQuoteString(it->first), out);
writeString(R"(,"min":)", out);
writeIntText(it->second.min, out);
writeString(R"(,"max":)", out);
writeIntText(it->second.max, out);
writeString("}", out);
}
writeString("]", out);
write_infos(moves_ttl, "moves", is_first);
is_first = false;
}
if (!recompression_ttl.empty())
{
if (!moves_ttl.empty() || !columns_ttl.empty() || table_ttl.min)
writeString(",", out);
writeString(R"("recompression":[)", out);
for (auto it = recompression_ttl.begin(); it != recompression_ttl.end(); ++it)
{
if (it != recompression_ttl.begin())
writeString(",", out);
writeString(R"({"expression":)", out);
writeString(doubleQuoteString(it->first), out);
writeString(R"(,"min":)", out);
writeIntText(it->second.min, out);
writeString(R"(,"max":)", out);
writeIntText(it->second.max, out);
writeString("}", out);
}
writeString("]", out);
write_infos(recompression_ttl, "recompression", is_first);
is_first = false;
}
if (!group_by_ttl.empty())
{
write_infos(group_by_ttl, "group_by", is_first);
is_first = false;
}
if (!rows_where_ttl.empty())
write_infos(rows_where_ttl, "rows_where", is_first);
writeString("}", out);
}

View File

@ -45,14 +45,17 @@ struct MergeTreeDataPartTTLInfos
time_t part_min_ttl = 0;
time_t part_max_ttl = 0;
TTLInfoMap rows_where_ttl;
TTLInfoMap moves_ttl;
TTLInfoMap recompression_ttl;
TTLInfoMap group_by_ttl;
/// Return the smallest max recompression TTL value
time_t getMinimalMaxRecompressionTTL() const;
void read(ReadBuffer & in);
void write(WriteBuffer & out) const;
void update(const MergeTreeDataPartTTLInfos & other_infos);
@ -68,6 +71,7 @@ struct MergeTreeDataPartTTLInfos
bool empty() const
{
/// part_min_ttl in minimum of rows, rows_where and group_by TTLs
return !part_min_ttl && moves_ttl.empty() && recompression_ttl.empty();
}
};

View File

@ -33,6 +33,7 @@
#include <Processors/QueryPlan/MergingSortedStep.h>
#include <Processors/QueryPlan/UnionStep.h>
#include <Processors/QueryPlan/MergingFinal.h>
#include <Processors/QueryPlan/ReadNothingStep.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeEnum.h>
@ -708,8 +709,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
if (parts_with_ranges.empty())
return std::make_unique<QueryPlan>();
const auto data_settings = data.getSettings();
auto max_partitions_to_read
= settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data.getSettings()->max_partitions_to_read;
= settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data_settings->max_partitions_to_read;
if (max_partitions_to_read > 0)
{
std::set<String> partitions;
@ -723,6 +725,18 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
max_partitions_to_read);
}
String query_id;
if (data_settings->max_concurrent_queries > 0)
{
if (data_settings->min_marks_to_honor_max_concurrent_queries > 0
&& sum_marks >= data_settings->min_marks_to_honor_max_concurrent_queries)
{
query_id = context.getCurrentQueryId();
if (!query_id.empty())
data.insertQueryIdOrThrow(query_id, data_settings->max_concurrent_queries);
}
}
ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size());
ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges);
ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks);
@ -759,7 +773,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
virt_column_names,
settings,
reader_settings,
result_projection);
result_projection,
query_id);
}
else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && query_info.input_order_info)
{
@ -782,7 +797,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
virt_column_names,
settings,
reader_settings,
result_projection);
result_projection,
query_id);
}
else
{
@ -796,7 +812,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
query_info,
virt_column_names,
settings,
reader_settings);
reader_settings,
query_id);
}
if (!plan)
@ -896,7 +913,7 @@ size_t minMarksForConcurrentRead(
}
static QueryPlanPtr createPlanFromPipe(Pipe pipe, const std::string & description = "")
static QueryPlanPtr createPlanFromPipe(Pipe pipe, const String & query_id, const MergeTreeData & data, const std::string & description = "")
{
auto plan = std::make_unique<QueryPlan>();
@ -904,6 +921,10 @@ static QueryPlanPtr createPlanFromPipe(Pipe pipe, const std::string & descriptio
if (!description.empty())
storage_name += ' ' + description;
// Attach QueryIdHolder if needed
if (!query_id.empty())
pipe.addQueryIdHolder(std::make_shared<QueryIdHolder>(query_id, data));
auto step = std::make_unique<ReadFromStorageStep>(std::move(pipe), storage_name);
plan->addStep(std::move(step));
return plan;
@ -919,7 +940,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
const SelectQueryInfo & query_info,
const Names & virt_columns,
const Settings & settings,
const MergeTreeReaderSettings & reader_settings) const
const MergeTreeReaderSettings & reader_settings,
const String & query_id) const
{
/// Count marks for each part.
std::vector<size_t> sum_marks_in_parts(parts.size());
@ -1004,7 +1026,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
res.emplace_back(std::move(source));
}
return createPlanFromPipe(Pipe::unitePipes(std::move(res)));
return createPlanFromPipe(Pipe::unitePipes(std::move(res)), query_id, data);
}
else
{
@ -1028,7 +1050,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
if (pipe.numOutputPorts() > 1)
pipe.addTransform(std::make_shared<ConcatProcessor>(pipe.getHeader(), pipe.numOutputPorts()));
return createPlanFromPipe(std::move(pipe));
return createPlanFromPipe(std::move(pipe), query_id, data);
}
}
@ -1052,7 +1074,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
const Names & virt_columns,
const Settings & settings,
const MergeTreeReaderSettings & reader_settings,
ActionsDAGPtr & out_projection) const
ActionsDAGPtr & out_projection,
const String & query_id) const
{
size_t sum_marks = 0;
const InputOrderInfoPtr & input_order_info = query_info.input_order_info;
@ -1243,7 +1266,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
}
}
auto plan = createPlanFromPipe(Pipe::unitePipes(std::move(pipes)), " with order");
auto plan = createPlanFromPipe(Pipe::unitePipes(std::move(pipes)), query_id, data, " with order");
if (input_order_info->direction != 1)
{
@ -1311,7 +1334,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
const Names & virt_columns,
const Settings & settings,
const MergeTreeReaderSettings & reader_settings,
ActionsDAGPtr & out_projection) const
ActionsDAGPtr & out_projection,
const String & query_id) const
{
const auto data_settings = data.getSettings();
size_t sum_marks = 0;
@ -1370,6 +1394,12 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
std::vector<QueryPlanPtr> partition_plans;
/// If do_not_merge_across_partitions_select_final is true and num_streams > 1
/// we will store lonely parts with level > 0 to use parallel select on them.
std::vector<RangesInDataPart> lonely_parts;
size_t total_rows_in_lonely_parts = 0;
size_t sum_marks_in_lonely_parts = 0;
for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index)
{
QueryPlanPtr plan;
@ -1377,25 +1407,41 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
{
Pipes pipes;
for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it)
/// If do_not_merge_across_partitions_select_final is true and there is only one part in partition
/// with level > 0 then we won't postprocess this part and if num_streams > 1 we
/// can use parallel select on such parts. We save such parts in one vector and then use
/// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select.
if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final &&
std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 &&
parts_to_merge_ranges[range_index]->data_part->info.level > 0)
{
auto source_processor = std::make_shared<MergeTreeSelectProcessor>(
data,
metadata_snapshot,
part_it->data_part,
max_block_size,
settings.preferred_block_size_bytes,
settings.preferred_max_column_in_block_size_bytes,
column_names,
part_it->ranges,
use_uncompressed_cache,
query_info.prewhere_info,
true,
reader_settings,
virt_columns,
part_it->part_index_in_query);
total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount();
sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount();
lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index]));
continue;
}
else
{
for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it)
{
auto source_processor = std::make_shared<MergeTreeSelectProcessor>(
data,
metadata_snapshot,
part_it->data_part,
max_block_size,
settings.preferred_block_size_bytes,
settings.preferred_max_column_in_block_size_bytes,
column_names,
part_it->ranges,
use_uncompressed_cache,
query_info.prewhere_info,
true,
reader_settings,
virt_columns,
part_it->part_index_in_query);
pipes.emplace_back(std::move(source_processor));
pipes.emplace_back(std::move(source_processor));
}
}
if (pipes.empty())
@ -1407,9 +1453,16 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
if (!out_projection)
out_projection = createProjection(pipe.getHeader());
plan = createPlanFromPipe(std::move(pipe), "with final");
plan = createPlanFromPipe(std::move(pipe), query_id, data, "with final");
}
auto expression_step = std::make_unique<ExpressionStep>(
plan->getCurrentDataStream(),
metadata_snapshot->getSortingKey().expression->getActionsDAG().clone());
expression_step->setStepDescription("Calculate sorting key expression");
plan->addStep(std::move(expression_step));
/// If do_not_merge_across_partitions_select_final is true and there is only one part in partition
/// with level > 0 then we won't postprocess this part
if (settings.do_not_merge_across_partitions_select_final &&
@ -1420,13 +1473,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
continue;
}
auto expression_step = std::make_unique<ExpressionStep>(
plan->getCurrentDataStream(),
metadata_snapshot->getSortingKey().expression->getActionsDAG().clone());
expression_step->setStepDescription("Calculate sorting key expression");
plan->addStep(std::move(expression_step));
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
SortDescription sort_description;
size_t sort_columns_size = sort_columns.size();
@ -1452,6 +1498,69 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
partition_plans.emplace_back(std::move(plan));
}
if (!lonely_parts.empty())
{
Pipes pipes;
size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size();
const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead(
settings.merge_tree_min_rows_for_concurrent_read,
settings.merge_tree_min_bytes_for_concurrent_read,
data_settings->index_granularity,
index_granularity_bytes,
sum_marks_in_lonely_parts);
/// Reduce the number of num_streams_for_lonely_parts if the data is small.
if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts)
num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size());
MergeTreeReadPoolPtr pool = std::make_shared<MergeTreeReadPool>(
num_streams_for_lonely_parts,
sum_marks_in_lonely_parts,
min_marks_for_concurrent_read,
std::move(lonely_parts),
data,
metadata_snapshot,
query_info.prewhere_info,
true,
column_names,
MergeTreeReadPool::BackoffSettings(settings),
settings.preferred_block_size_bytes,
false);
LOG_TRACE(log, "Reading approx. {} rows with {} streams", total_rows_in_lonely_parts, num_streams_for_lonely_parts);
for (size_t i = 0; i < num_streams_for_lonely_parts; ++i)
{
auto source = std::make_shared<MergeTreeThreadSelectBlockInputProcessor>(
i, pool, min_marks_for_concurrent_read, max_block_size,
settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes,
data, metadata_snapshot, use_uncompressed_cache,
query_info.prewhere_info, reader_settings, virt_columns);
pipes.emplace_back(std::move(source));
}
auto pipe = Pipe::unitePipes(std::move(pipes));
/// Drop temporary columns, added by 'sorting_key_expr'
if (!out_projection)
out_projection = createProjection(pipe.getHeader());
QueryPlanPtr plan = createPlanFromPipe(std::move(pipe), query_id, data, "with final");
auto expression_step = std::make_unique<ExpressionStep>(
plan->getCurrentDataStream(),
metadata_snapshot->getSortingKey().expression->getActionsDAG().clone());
expression_step->setStepDescription("Calculate sorting key expression");
plan->addStep(std::move(expression_step));
partition_plans.emplace_back(std::move(plan));
}
if (partition_plans.empty())
return {};

View File

@ -58,7 +58,8 @@ private:
const SelectQueryInfo & query_info,
const Names & virt_columns,
const Settings & settings,
const MergeTreeReaderSettings & reader_settings) const;
const MergeTreeReaderSettings & reader_settings,
const String & query_id) const;
/// out_projection - save projection only with columns, requested to read
QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder(
@ -73,7 +74,8 @@ private:
const Names & virt_columns,
const Settings & settings,
const MergeTreeReaderSettings & reader_settings,
ActionsDAGPtr & out_projection) const;
ActionsDAGPtr & out_projection,
const String & query_id) const;
QueryPlanPtr spreadMarkRangesAmongStreamsFinal(
RangesInDataParts && parts,
@ -86,7 +88,8 @@ private:
const Names & virt_columns,
const Settings & settings,
const MergeTreeReaderSettings & reader_settings,
ActionsDAGPtr & out_projection) const;
ActionsDAGPtr & out_projection,
const String & query_id) const;
/// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index.
size_t getApproximateTotalRowsToRead(

View File

@ -12,6 +12,7 @@
#include <IO/WriteHelpers.h>
#include <Poco/File.h>
#include <Common/typeid_cast.h>
#include <DataStreams/ITTLAlgorithm.h>
#include <Parsers/queryToString.h>
@ -91,31 +92,23 @@ void updateTTL(
const TTLDescription & ttl_entry,
IMergeTreeDataPart::TTLInfos & ttl_infos,
DB::MergeTreeDataPartTTLInfo & ttl_info,
Block & block,
const Block & block,
bool update_part_min_max_ttls)
{
bool remove_column = false;
if (!block.has(ttl_entry.result_column))
{
ttl_entry.expression->execute(block);
remove_column = true;
}
auto ttl_column = ITTLAlgorithm::executeExpressionAndGetColumn(ttl_entry.expression, block, ttl_entry.result_column);
const auto & current = block.getByName(ttl_entry.result_column);
const IColumn * column = current.column.get();
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(ttl_column.get()))
{
const auto & date_lut = DateLUT::instance();
for (const auto & val : column_date->getData())
ttl_info.update(date_lut.fromDayNum(DayNum(val)));
}
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(ttl_column.get()))
{
for (const auto & val : column_date_time->getData())
ttl_info.update(val);
}
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(ttl_column.get()))
{
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
{
@ -134,9 +127,6 @@ void updateTTL(
if (update_part_min_max_ttls)
ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
if (remove_column)
block.erase(ttl_entry.result_column);
}
}
@ -383,6 +373,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
if (metadata_snapshot->hasRowsTTL())
updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true);
for (const auto & ttl_entry : metadata_snapshot->getGroupByTTLs())
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.group_by_ttl[ttl_entry.result_column], block, true);
for (const auto & ttl_entry : metadata_snapshot->getRowsWhereTTLs())
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.rows_where_ttl[ttl_entry.result_column], block, true);
for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs())
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true);

View File

@ -111,6 +111,8 @@ struct Settings;
M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \
M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \
M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \
M(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \
M(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \
\
/** Obsolete settings. Kept for backward compatibility only. */ \
M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \

View File

@ -74,20 +74,9 @@ size_t ReplicatedMergeTreePartCheckThread::size() const
}
void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & part_name)
ReplicatedMergeTreePartCheckThread::MissingPartSearchResult ReplicatedMergeTreePartCheckThread::searchForMissingPartOnOtherReplicas(const String & part_name)
{
auto zookeeper = storage.getZooKeeper();
String part_path = storage.replica_path + "/parts/" + part_name;
/// If the part is in ZooKeeper, remove it from there and add the task to download it to the queue.
if (zookeeper->exists(part_path))
{
LOG_WARNING(log, "Part {} exists in ZooKeeper but not locally. Removing from ZooKeeper and queueing a fetch.", part_name);
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
storage.removePartAndEnqueueFetch(part_name);
return;
}
/// If the part is not in ZooKeeper, we'll check if it's at least somewhere.
auto part_info = MergeTreePartInfo::fromPartName(part_name, storage.format_version);
@ -115,7 +104,7 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & par
* and don't delete the queue entry when in doubt.
*/
LOG_WARNING(log, "Checking if anyone has a part covering {}.", part_name);
LOG_WARNING(log, "Checking if anyone has a part {} or covering part.", part_name);
bool found_part_with_the_same_min_block = false;
bool found_part_with_the_same_max_block = false;
@ -123,15 +112,27 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & par
Strings replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas");
for (const String & replica : replicas)
{
Strings parts = zookeeper->getChildren(storage.zookeeper_path + "/replicas/" + replica + "/parts");
String replica_path = storage.zookeeper_path + "/replicas/" + replica;
Strings parts = zookeeper->getChildren(replica_path + "/parts");
for (const String & part_on_replica : parts)
{
auto part_on_replica_info = MergeTreePartInfo::fromPartName(part_on_replica, storage.format_version);
if (part_info == part_on_replica_info)
{
/// Found missing part at ourself. If we are here then something wrong with this part, so skipping.
if (replica_path == storage.replica_path)
continue;
LOG_WARNING(log, "Found the missing part {} at {} on {}", part_name, part_on_replica, replica);
return MissingPartSearchResult::FoundAndNeedFetch;
}
if (part_on_replica_info.contains(part_info))
{
LOG_WARNING(log, "Found part {} on {} that covers the missing part {}", part_on_replica, replica, part_name);
return;
return MissingPartSearchResult::FoundAndDontNeedFetch;
}
if (part_info.contains(part_on_replica_info))
@ -144,7 +145,7 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & par
if (found_part_with_the_same_min_block && found_part_with_the_same_max_block)
{
LOG_WARNING(log, "Found parts with the same min block and with the same max block as the missing part {}. Hoping that it will eventually appear as a result of a merge.", part_name);
return;
return MissingPartSearchResult::FoundAndDontNeedFetch;
}
}
}
@ -160,28 +161,61 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & par
not_found_msg = "smaller parts with either the same min block or the same max block.";
LOG_ERROR(log, "No replica has part covering {} and a merge is impossible: we didn't find {}", part_name, not_found_msg);
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
/// Is it in the replication queue? If there is - delete, because the task can not be processed.
if (!storage.queue.remove(zookeeper, part_name))
{
/// The part was not in our queue. Why did it happen?
LOG_ERROR(log, "Missing part {} is not in our queue.", part_name);
return;
}
/** This situation is possible if on all the replicas where the part was, it deteriorated.
* For example, a replica that has just written it has power turned off and the data has not been written from cache to disk.
*/
LOG_ERROR(log, "Part {} is lost forever.", part_name);
ProfileEvents::increment(ProfileEvents::ReplicatedDataLoss);
return MissingPartSearchResult::LostForever;
}
CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
void ReplicatedMergeTreePartCheckThread::searchForMissingPartAndFetchIfPossible(const String & part_name, bool exists_in_zookeeper)
{
LOG_WARNING(log, "Checking part {}", part_name);
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks);
auto zookeeper = storage.getZooKeeper();
auto missing_part_search_result = searchForMissingPartOnOtherReplicas(part_name);
/// If the part is in ZooKeeper, remove it from there and add the task to download it to the queue.
if (exists_in_zookeeper)
{
/// If part found on some other replica
if (missing_part_search_result == MissingPartSearchResult::FoundAndNeedFetch)
{
LOG_WARNING(log, "Part {} exists in ZooKeeper but not locally and found on other replica. Removing from ZooKeeper and queueing a fetch.", part_name);
storage.removePartAndEnqueueFetch(part_name);
}
else /// If we have covering part on other replica or part is lost forever we don't need to fetch anything
{
LOG_WARNING(log, "Part {} exists in ZooKeeper but not locally and not found on other replica. Removing it from ZooKeeper.", part_name);
storage.removePartFromZooKeeper(part_name);
}
}
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
if (missing_part_search_result == MissingPartSearchResult::LostForever)
{
/// Is it in the replication queue? If there is - delete, because the task can not be processed.
if (!storage.queue.remove(zookeeper, part_name))
{
/// The part was not in our queue.
LOG_WARNING(log, "Missing part {} is not in our queue, this can happen rarely.", part_name);
}
/** This situation is possible if on all the replicas where the part was, it deteriorated.
* For example, a replica that has just written it has power turned off and the data has not been written from cache to disk.
*/
LOG_ERROR(log, "Part {} is lost forever.", part_name);
ProfileEvents::increment(ProfileEvents::ReplicatedDataLoss);
}
}
std::pair<bool, MergeTreeDataPartPtr> ReplicatedMergeTreePartCheckThread::findLocalPart(const String & part_name)
{
auto zookeeper = storage.getZooKeeper();
String part_path = storage.replica_path + "/parts/" + part_name;
/// It's important to check zookeeper first and after that check local storage,
/// because our checks of local storage and zookeeper are not consistent.
/// If part exists in zookeeper and doesn't exists in local storage definitely require
/// to fetch this part. But if we check local storage first and than check zookeeper
/// some background process can successfully commit part between this checks (both to the local stoarge and zookeeper),
/// but checker thread will remove part from zookeeper and queue fetch.
bool exists_in_zookeeper = zookeeper->exists(part_path);
/// If the part is still in the PreCommitted -> Committed transition, it is not lost
/// and there is no need to go searching for it on other replicas. To definitely find the needed part
@ -190,17 +224,27 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na
if (!part)
part = storage.getActiveContainingPart(part_name);
return std::make_pair(exists_in_zookeeper, part);
}
CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
{
LOG_WARNING(log, "Checking part {}", part_name);
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks);
auto [exists_in_zookeeper, part] = findLocalPart(part_name);
/// We do not have this or a covering part.
if (!part)
{
searchForMissingPart(part_name);
searchForMissingPartAndFetchIfPossible(part_name, exists_in_zookeeper);
return {part_name, false, "Part is missing, will search for it"};
}
/// We have this part, and it's active. We will check whether we need this part and whether it has the right data.
else if (part->name == part_name)
if (part->name == part_name)
{
auto zookeeper = storage.getZooKeeper();
auto table_lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations);
auto local_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums(
@ -254,11 +298,11 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na
tryLogCurrentException(log, __PRETTY_FUNCTION__);
String message = "Part " + part_name + " looks broken. Removing it and queueing a fetch.";
String message = "Part " + part_name + " looks broken. Removing it and will try to fetch.";
LOG_ERROR(log, message);
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
storage.removePartAndEnqueueFetch(part_name);
/// Part is broken, let's try to find it and fetch.
searchForMissingPartAndFetchIfPossible(part_name, exists_in_zookeeper);
/// Delete part locally.
storage.forgetPartAndMoveToDetached(part, "broken");

View File

@ -12,6 +12,7 @@
#include <common/logger_useful.h>
#include <Core/BackgroundSchedulePool.h>
#include <Storages/CheckResults.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
namespace DB
{
@ -73,7 +74,26 @@ public:
private:
void run();
void searchForMissingPart(const String & part_name);
/// Search for missing part and queue fetch if possible. Otherwise
/// remove part from zookeeper and queue.
void searchForMissingPartAndFetchIfPossible(const String & part_name, bool exists_in_zookeeper);
std::pair<bool, MergeTreeDataPartPtr> findLocalPart(const String & part_name);
enum MissingPartSearchResult
{
/// We found this part on other replica, let's fetch it.
FoundAndNeedFetch,
/// We found covering part or source part with same min and max block number
/// don't need to fetch because we should do it during normal queue processing.
FoundAndDontNeedFetch,
/// Covering part not found anywhere and exact part_name doesn't found on other
/// replicas.
LostForever,
};
/// Search for missing part on other replicas or covering part on all replicas (including our replica).
MissingPartSearchResult searchForMissingPartOnOtherReplicas(const String & part_name);
StorageReplicatedMergeTree & storage;
String log_name;

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