Merge branch 'master' into orc_tuple_field_prune

This commit is contained in:
taiyang-li 2023-11-02 17:52:05 +08:00
commit 819c7e75ff
234 changed files with 3700 additions and 1177 deletions

View File

@ -45,4 +45,4 @@ We are a globally diverse and distributed team, united behind a common goal of c
Check out our **current openings** here: https://clickhouse.com/company/careers
Cant find what you are looking for, but want to let us know you are interested in joining ClickHouse? Email careers@clickhouse.com!
Can't find what you are looking for, but want to let us know you are interested in joining ClickHouse? Email careers@clickhouse.com!

2
contrib/libhdfs3 vendored

@ -1 +1 @@
Subproject commit 377220ef351ae24994a5fcd2b5fa3930d00c4db0
Subproject commit bdcb91354b1c05b21e73043a112a6f1e3b013497

View File

@ -1,4 +1,4 @@
if(NOT OS_FREEBSD AND NOT APPLE AND NOT ARCH_PPC64LE AND NOT ARCH_S390X)
if(NOT OS_FREEBSD AND NOT APPLE AND NOT ARCH_PPC64LE)
option(ENABLE_HDFS "Enable HDFS" ${ENABLE_LIBRARIES})
elseif(ENABLE_HDFS)
message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use HDFS3 with current configuration")

View File

@ -189,6 +189,7 @@ rg -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
-e "ZooKeeperClient" \
-e "KEEPER_EXCEPTION" \
-e "DirectoryMonitor" \
-e "DistributedInsertQueue" \
-e "TABLE_IS_READ_ONLY" \
-e "Code: 1000, e.code() = 111, Connection refused" \
-e "UNFINISHED" \

View File

@ -219,13 +219,21 @@ You can also run your custom-built ClickHouse binary with the config file from t
## IDE (Integrated Development Environment) {#ide-integrated-development-environment}
If you do not know which IDE to use, we recommend that you use CLion. CLion is commercial software, but it offers 30 days free trial period. It is also free of charge for students. CLion can be used both on Linux and on macOS.
**CLion (recommended)**
KDevelop and QTCreator are other great alternatives of an IDE for developing ClickHouse. KDevelop comes in as a very handy IDE although unstable. If KDevelop crashes after a while upon opening project, you should click “Stop All” button as soon as it has opened the list of projects files. After doing so KDevelop should be fine to work with.
If you do not know which IDE to use, we recommend that you use [CLion](https://www.jetbrains.com/clion/). CLion is commercial software but it offers a 30 day free trial. It is also free of charge for students. CLion can be used on both Linux and macOS.
As simple code editors, you can use Sublime Text or Visual Studio Code, or Kate (all of which are available on Linux).
A few things to know when using CLion to develop ClickHouse:
Just in case, it is worth mentioning that CLion creates `build` path on its own, it also on its own selects `debug` for build type, for configuration it uses a version of CMake that is defined in CLion and not the one installed by you, and finally, CLion will use `make` to run build tasks instead of `ninja`. This is normal behaviour, just keep that in mind to avoid confusion.
- CLion creates a `build` path on its own and automatically selects `debug` for the build type
- It uses a version of CMake that is defined in CLion and not the one installed by you
- CLion will use `make` to run build tasks instead of `ninja` (this is normal behavior)
**Other alternatives**
[KDevelop](https://kdevelop.org/) and [QTCreator](https://www.qt.io/product/development-tools) are other great alternative IDEs for developing ClickHouse. While KDevelop is a great IDE, it is sometimes unstable. If KDevelop crashes when opening a project, you should click the “Stop All” button as soon as it has opened the list of projects files. After doing so, KDevelop should be fine to work with.
Other IDEs you can use are [Sublime Text](https://www.sublimetext.com/), [Visual Studio Code](https://code.visualstudio.com/), or [Kate](https://kate-editor.org/) (all of which are available on Linux). If you are using VS Code, we recommend using the [clangd extension](https://marketplace.visualstudio.com/items?itemName=llvm-vs-code-extensions.vscode-clangd) to replace IntelliSense as it is much more performant.
## Writing Code {#writing-code}

View File

@ -28,7 +28,6 @@ SETTINGS
kafka_topic_list = 'topic1,topic2,...',
kafka_group_name = 'group_name',
kafka_format = 'data_format'[,]
[kafka_row_delimiter = 'delimiter_symbol',]
[kafka_schema = '',]
[kafka_num_consumers = N,]
[kafka_max_block_size = 0,]
@ -53,7 +52,6 @@ Required parameters:
Optional parameters:
- `kafka_row_delimiter` — Delimiter character, which ends the message. **This setting is deprecated and is no longer used, not left for compatibility reasons.**
- `kafka_schema` — Parameter that must be used if the format requires a schema definition. For example, [Capn Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
- `kafka_num_consumers` — The number of consumers per table. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition, and must not be greater than the number of physical cores on the server where ClickHouse is deployed. Default: `1`.
- `kafka_max_block_size` — The maximum batch size (in messages) for poll. Default: [max_insert_block_size](../../../operations/settings/settings.md#setting-max_insert_block_size).
@ -64,7 +62,7 @@ Optional parameters:
- `kafka_poll_max_batch_size` — Maximum amount of messages to be polled in a single Kafka poll. Default: [max_block_size](../../../operations/settings/settings.md#setting-max_block_size).
- `kafka_flush_interval_ms` — Timeout for flushing data from Kafka. Default: [stream_flush_interval_ms](../../../operations/settings/settings.md#stream-flush-interval-ms).
- `kafka_thread_per_consumer` — Provide independent thread for each consumer. When enabled, every consumer flush the data independently, in parallel (otherwise — rows from several consumers squashed to form one block). Default: `0`.
- `kafka_handle_error_mode` — How to handle errors for Kafka engine. Possible values: default, stream.
- `kafka_handle_error_mode` — How to handle errors for Kafka engine. Possible values: default (the exception will be thrown if we fail to parse a message), stream (the exception message and raw message will be saved in virtual columns `_error` and `_raw_message`).
- `kafka_commit_on_select` — Commit messages when select query is made. Default: `false`.
- `kafka_max_rows_per_message` — The maximum number of rows written in one kafka message for row-based formats. Default : `1`.
@ -249,6 +247,13 @@ Example:
- `_headers.name` — Array of message's headers keys.
- `_headers.value` — Array of message's headers values.
Additional virtual columns when `kafka_handle_error_mode='stream'`:
- `_raw_message` - Raw message that couldn't be parsed successfully.
- `_error` - Exception message happened during failed parsing.
Note: `_raw_message` and `_error` virtual columns are filled only in case of exception during parsing, they are always empty when message was parsed successfully.
## Data formats support {#data-formats-support}
Kafka engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse.

View File

@ -25,7 +25,6 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
nats_url = 'host:port',
nats_subjects = 'subject1,subject2,...',
nats_format = 'data_format'[,]
[nats_row_delimiter = 'delimiter_symbol',]
[nats_schema = '',]
[nats_num_consumers = N,]
[nats_queue_group = 'group_name',]
@ -40,7 +39,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[nats_password = 'password',]
[nats_token = 'clickhouse',]
[nats_startup_connect_tries = '5']
[nats_max_rows_per_message = 1]
[nats_max_rows_per_message = 1,]
[nats_handle_error_mode = 'default']
```
Required parameters:
@ -51,7 +51,6 @@ Required parameters:
Optional parameters:
- `nats_row_delimiter` Delimiter character, which ends the message. **This setting is deprecated and is no longer used, not left for compatibility reasons.**
- `nats_schema` Parameter that must be used if the format requires a schema definition. For example, [Capn Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
- `nats_num_consumers` The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient.
- `nats_queue_group` Name for queue group of NATS subscribers. Default is the table name.
@ -66,6 +65,7 @@ Optional parameters:
- `nats_token` - NATS auth token.
- `nats_startup_connect_tries` - Number of connect tries at startup. Default: `5`.
- `nats_max_rows_per_message` — The maximum number of rows written in one NATS message for row-based formats. (default : `1`).
- `nats_handle_error_mode` — How to handle errors for RabbitMQ engine. Possible values: default (the exception will be thrown if we fail to parse a message), stream (the exception message and raw message will be saved in virtual columns `_error` and `_raw_message`).
SSL connection:
@ -165,6 +165,14 @@ If you want to change the target table by using `ALTER`, we recommend disabling
- `_subject` - NATS message subject.
Additional virtual columns when `kafka_handle_error_mode='stream'`:
- `_raw_message` - Raw message that couldn't be parsed successfully.
- `_error` - Exception message happened during failed parsing.
Note: `_raw_message` and `_error` virtual columns are filled only in case of exception during parsing, they are always empty when message was parsed successfully.
## Data formats support {#data-formats-support}
NATS engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse.

View File

@ -28,7 +28,6 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[rabbitmq_exchange_type = 'exchange_type',]
[rabbitmq_routing_key_list = 'key1,key2,...',]
[rabbitmq_secure = 0,]
[rabbitmq_row_delimiter = 'delimiter_symbol',]
[rabbitmq_schema = '',]
[rabbitmq_num_consumers = N,]
[rabbitmq_num_queues = N,]
@ -45,7 +44,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[rabbitmq_username = '',]
[rabbitmq_password = '',]
[rabbitmq_commit_on_select = false,]
[rabbitmq_max_rows_per_message = 1]
[rabbitmq_max_rows_per_message = 1,]
[rabbitmq_handle_error_mode = 'default']
```
Required parameters:
@ -58,7 +58,6 @@ Optional parameters:
- `rabbitmq_exchange_type` The type of RabbitMQ exchange: `direct`, `fanout`, `topic`, `headers`, `consistent_hash`. Default: `fanout`.
- `rabbitmq_routing_key_list` A comma-separated list of routing keys.
- `rabbitmq_row_delimiter` Delimiter character, which ends the message. **This setting is deprecated and is no longer used, not left for compatibility reasons.**
- `rabbitmq_schema` Parameter that must be used if the format requires a schema definition. For example, [Capn Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object.
- `rabbitmq_num_consumers` The number of consumers per table. Specify more consumers if the throughput of one consumer is insufficient. Default: `1`
- `rabbitmq_num_queues` Total number of queues. Increasing this number can significantly improve performance. Default: `1`.
@ -78,6 +77,7 @@ Optional parameters:
- `rabbitmq_max_rows_per_message` — The maximum number of rows written in one RabbitMQ message for row-based formats. Default : `1`.
- `rabbitmq_empty_queue_backoff_start` — A start backoff point to reschedule read if the rabbitmq queue is empty.
- `rabbitmq_empty_queue_backoff_end` — An end backoff point to reschedule read if the rabbitmq queue is empty.
- `rabbitmq_handle_error_mode` — How to handle errors for RabbitMQ engine. Possible values: default (the exception will be thrown if we fail to parse a message), stream (the exception message and raw message will be saved in virtual columns `_error` and `_raw_message`).
@ -191,6 +191,13 @@ Example:
- `_message_id` - messageID of the received message; non-empty if was set, when message was published.
- `_timestamp` - timestamp of the received message; non-empty if was set, when message was published.
Additional virtual columns when `kafka_handle_error_mode='stream'`:
- `_raw_message` - Raw message that couldn't be parsed successfully.
- `_error` - Exception message happened during failed parsing.
Note: `_raw_message` and `_error` virtual columns are filled only in case of exception during parsing, they are always empty when message was parsed successfully.
## Data formats support {#data-formats-support}
RabbitMQ engine supports all [formats](../../../interfaces/formats.md) supported in ClickHouse.

View File

@ -48,61 +48,61 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] AS [db2.]name2
#### policy_name
`policy_name` - (optionally) policy name, it will be used to store temporary files for async send
`policy_name` - (optionally) policy name, it will be used to store temporary files for background send
**See Also**
- [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) setting
- [distributed_foreground_insert](../../../operations/settings/settings.md#distributed_foreground_insert) setting
- [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) for the examples
### Distributed Settings
#### fsync_after_insert
`fsync_after_insert` - do the `fsync` for the file data after asynchronous insert to Distributed. Guarantees that the OS flushed the whole inserted data to a file **on the initiator node** disk.
`fsync_after_insert` - do the `fsync` for the file data after background insert to Distributed. Guarantees that the OS flushed the whole inserted data to a file **on the initiator node** disk.
#### fsync_directories
`fsync_directories` - do the `fsync` for directories. Guarantees that the OS refreshed directory metadata after operations related to asynchronous inserts on Distributed table (after insert, after sending the data to shard, etc).
`fsync_directories` - do the `fsync` for directories. Guarantees that the OS refreshed directory metadata after operations related to background inserts on Distributed table (after insert, after sending the data to shard, etc).
#### bytes_to_throw_insert
`bytes_to_throw_insert` - if more than this number of compressed bytes will be pending for async INSERT, an exception will be thrown. 0 - do not throw. Default 0.
`bytes_to_throw_insert` - if more than this number of compressed bytes will be pending for background INSERT, an exception will be thrown. 0 - do not throw. Default 0.
#### bytes_to_delay_insert
`bytes_to_delay_insert` - if more than this number of compressed bytes will be pending for async INSERT, the query will be delayed. 0 - do not delay. Default 0.
`bytes_to_delay_insert` - if more than this number of compressed bytes will be pending for background INSERT, the query will be delayed. 0 - do not delay. Default 0.
#### max_delay_to_insert
`max_delay_to_insert` - max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for async send. Default 60.
`max_delay_to_insert` - max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for background send. Default 60.
#### monitor_batch_inserts
#### background_insert_batch
`monitor_batch_inserts` - same as [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts)
`background_insert_batch` - same as [distributed_background_insert_batch](../../../operations/settings/settings.md#distributed_background_insert_batch)
#### monitor_split_batch_on_failure
#### background_insert_split_batch_on_failure
`monitor_split_batch_on_failure` - same as [distributed_directory_monitor_split_batch_on_failure](../../../operations/settings/settings.md#distributed_directory_monitor_split_batch_on_failure)
`background_insert_split_batch_on_failure` - same as [distributed_background_insert_split_batch_on_failure](../../../operations/settings/settings.md#distributed_background_insert_split_batch_on_failure)
#### monitor_sleep_time_ms
#### background_insert_sleep_time_ms
`monitor_sleep_time_ms` - same as [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms)
`background_insert_sleep_time_ms` - same as [distributed_background_insert_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_sleep_time_ms)
#### monitor_max_sleep_time_ms
#### background_insert_max_sleep_time_ms
`monitor_max_sleep_time_ms` - same as [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms)
`background_insert_max_sleep_time_ms` - same as [distributed_background_insert_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_max_sleep_time_ms)
:::note
**Durability settings** (`fsync_...`):
- Affect only asynchronous INSERTs (i.e. `insert_distributed_sync=false`) when data first stored on the initiator node disk and later asynchronously send to shards.
- Affect only background INSERTs (i.e. `distributed_foreground_insert=false`) when data first stored on the initiator node disk and later, in background, send to shards.
- May significantly decrease the inserts' performance
- Affect writing the data stored inside Distributed table folder into the **node which accepted your insert**. If you need to have guarantees of writing data to underlying MergeTree tables - see durability settings (`...fsync...`) in `system.merge_tree_settings`
For **Insert limit settings** (`..._insert`) see also:
- [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) setting
- [distributed_foreground_insert](../../../operations/settings/settings.md#distributed_foreground_insert) setting
- [prefer_localhost_replica](../../../operations/settings/settings.md#settings-prefer-localhost-replica) setting
- `bytes_to_throw_insert` handled before `bytes_to_delay_insert`, so you should not set it to the value less then `bytes_to_delay_insert`
:::
@ -232,7 +232,7 @@ You should be concerned about the sharding scheme in the following cases:
- Queries are used that require joining data (`IN` or `JOIN`) by a specific key. If data is sharded by this key, you can use local `IN` or `JOIN` instead of `GLOBAL IN` or `GLOBAL JOIN`, which is much more efficient.
- A large number of servers is used (hundreds or more) with a large number of small queries, for example, queries for data of individual clients (e.g. websites, advertisers, or partners). In order for the small queries to not affect the entire cluster, it makes sense to locate data for a single client on a single shard. Alternatively, you can set up bi-level sharding: divide the entire cluster into “layers”, where a layer may consist of multiple shards. Data for a single client is located on a single layer, but shards can be added to a layer as necessary, and data is randomly distributed within them. `Distributed` tables are created for each layer, and a single shared distributed table is created for global queries.
Data is written asynchronously. When inserted in the table, the data block is just written to the local file system. The data is sent to the remote servers in the background as soon as possible. The periodicity for sending data is managed by the [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) and [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) settings. The `Distributed` engine sends each file with inserted data separately, but you can enable batch sending of files with the [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) setting. This setting improves cluster performance by better utilizing local server and network resources. You should check whether data is sent successfully by checking the list of files (data waiting to be sent) in the table directory: `/var/lib/clickhouse/data/database/table/`. The number of threads performing background tasks can be set by [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting.
Data is written in background. When inserted in the table, the data block is just written to the local file system. The data is sent to the remote servers in the background as soon as possible. The periodicity for sending data is managed by the [distributed_background_insert_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_sleep_time_ms) and [distributed_background_insert_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_max_sleep_time_ms) settings. The `Distributed` engine sends each file with inserted data separately, but you can enable batch sending of files with the [distributed_background_insert_batch](../../../operations/settings/settings.md#distributed_background_insert_batch) setting. This setting improves cluster performance by better utilizing local server and network resources. You should check whether data is sent successfully by checking the list of files (data waiting to be sent) in the table directory: `/var/lib/clickhouse/data/database/table/`. The number of threads performing background tasks can be set by [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting.
If the server ceased to exist or had a rough restart (for example, due to a hardware failure) after an `INSERT` to a `Distributed` table, the inserted data might be lost. If a damaged data part is detected in the table directory, it is transferred to the `broken` subdirectory and no longer used.

View File

@ -0,0 +1,105 @@
---
slug: /en/engines/table-engines/special/filelog
sidebar_position: 160
sidebar_label: FileLog
---
# FileLog Engine {#filelog-engine}
This engine allows to process application log files as a stream of records.
`FileLog` lets you:
- Subscribe to log files.
- Process new records as they are appended to subscribed log files.
## Creating a Table {#creating-a-table}
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
...
) ENGINE = FileLog('path_to_logs', 'format_name') SETTINGS
[poll_timeout_ms = 0,]
[poll_max_batch_size = 0,]
[max_block_size = 0,]
[max_threads = 0,]
[poll_directory_watch_events_backoff_init = 500,]
[poll_directory_watch_events_backoff_max = 32000,]
[poll_directory_watch_events_backoff_factor = 2,]
[handle_error_mode = 'default']
```
Engine arguments:
- `path_to_logs` Path to log files to subscribe. It can be path to a directory with log files or to a single log file. Note that ClickHouse allows only paths inside `user_files` directory.
- `format_name` - Record format. Note that FileLog process each line in a file as a separate record and not all data formats are suitable for it.
Optional parameters:
- `poll_timeout_ms` - Timeout for single poll from log file. Default: [stream_poll_timeout_ms](../../../operations/settings/settings.md#stream_poll_timeout_ms).
- `poll_max_batch_size` — Maximum amount of records to be polled in a single poll. Default: [max_block_size](../../../operations/settings/settings.md#setting-max_block_size).
- `max_block_size` — The maximum batch size (in records) for poll. Default: [max_insert_block_size](../../../operations/settings/settings.md#setting-max_insert_block_size).
- `max_threads` - Number of max threads to parse files, default is 0, which means the number will be max(1, physical_cpu_cores / 4).
- `poll_directory_watch_events_backoff_init` - The initial sleep value for watch directory thread. Default: `500`.
- `poll_directory_watch_events_backoff_max` - The max sleep value for watch directory thread. Default: `32000`.
- `poll_directory_watch_events_backoff_factor` - The speed of backoff, exponential by default. Default: `2`.
- `handle_error_mode` — How to handle errors for FileLog engine. Possible values: default (the exception will be thrown if we fail to parse a message), stream (the exception message and raw message will be saved in virtual columns `_error` and `_raw_message`).
## Description {#description}
The delivered records are tracked automatically, so each record in a log file is only counted once.
`SELECT` is not particularly useful for reading records (except for debugging), because each record can be read only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this:
1. Use the engine to create a FileLog table and consider it a data stream.
2. Create a table with the desired structure.
3. Create a materialized view that converts data from the engine and puts it into a previously created table.
When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. This allows you to continually receive records from log files and convert them to the required format using `SELECT`.
One FileLog table can have as many materialized views as you like, they do not read data from the table directly, but receive new records (in blocks), this way you can write to several tables with different detail level (with grouping - aggregation and without).
Example:
``` sql
CREATE TABLE logs (
timestamp UInt64,
level String,
message String
) ENGINE = FileLog('user_files/my_app/app.log', 'JSONEachRow');
CREATE TABLE daily (
day Date,
level String,
total UInt64
) ENGINE = SummingMergeTree(day, (day, level), 8192);
CREATE MATERIALIZED VIEW consumer TO daily
AS SELECT toDate(toDateTime(timestamp)) AS day, level, count() as total
FROM queue GROUP BY day, level;
SELECT level, sum(total) FROM daily GROUP BY level;
```
To stop receiving streams data or to change the conversion logic, detach the materialized view:
``` sql
DETACH TABLE consumer;
ATTACH TABLE consumer;
```
If you want to change the target table by using `ALTER`, we recommend disabling the material view to avoid discrepancies between the target table and the data from the view.
## Virtual Columns {#virtual-columns}
- `_filename` - Name of the log file.
- `_offset` - Offset in the log file.
Additional virtual columns when `kafka_handle_error_mode='stream'`:
- `_raw_record` - Raw record that couldn't be parsed successfully.
- `_error` - Exception message happened during failed parsing.
Note: `_raw_record` and `_error` virtual columns are filled only in case of exception during parsing, they are always empty when message was parsed successfully.

View File

@ -74,6 +74,7 @@ The supported formats are:
| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ |
| [ORC](#data-format-orc) | ✔ | ✔ |
| [One](#data-format-one) | ✔ | ✗ |
| [Npy](#data-format-npy) | ✔ | ✗ |
| [RowBinary](#rowbinary) | ✔ | ✔ |
| [RowBinaryWithNames](#rowbinarywithnamesandtypes) | ✔ | ✔ |
| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ |
@ -2454,6 +2455,50 @@ Result:
└──────────────┘
```
## Npy {#data-format-npy}
This function is designed to load a NumPy array from a .npy file into ClickHouse. The NumPy file format is a binary format used for efficiently storing arrays of numerical data. During import, ClickHouse treats top level dimension as an array of rows with single column. Supported Npy data types and their corresponding type in ClickHouse:
| Npy type | ClickHouse type |
|:--------:|:---------------:|
| b1 | UInt8 |
| i1 | Int8 |
| i2 | Int16 |
| i4 | Int32 |
| i8 | Int64 |
| u1 | UInt8 |
| u2 | UInt16 |
| u4 | UInt32 |
| u8 | UInt64 |
| f4 | Float32 |
| f8 | Float64 |
| S | String |
| U | String |
**Example of saving an array in .npy format using Python**
```Python
import numpy as np
arr = np.array([[[1],[2],[3]],[[4],[5],[6]]])
np.save('example_array.npy', arr)
```
**Example of reading a NumPy file in ClickHouse**
Query:
```sql
SELECT *
FROM file('example_array.npy', Npy)
```
Result:
```
┌─array─────────┐
│ [[1],[2],[3]] │
│ [[4],[5],[6]] │
└───────────────┘
```
## LineAsString {#lineasstring}
In this format, every line of input data is interpreted as a single string value. This format can only be parsed for table with a single field of type [String](/docs/en/sql-reference/data-types/string.md). The remaining columns must be set to [DEFAULT](/docs/en/sql-reference/statements/create/table.md/#default) or [MATERIALIZED](/docs/en/sql-reference/statements/create/table.md/#materialized), or omitted.

View File

@ -106,6 +106,15 @@ Possible values:
Default value: 0.
## max_bytes_before_external_sort {#settings-max_bytes_before_external_sort}
Enables or disables execution of `ORDER BY` clauses in external memory. See [ORDER BY Implementation Details](../../sql-reference/statements/select/order-by.md#implementation-details)
- Maximum volume of RAM (in bytes) that can be used by the single [ORDER BY](../../sql-reference/statements/select/order-by.md) operation. Recommended value is half of available system memory
- 0 — `ORDER BY` in external memory disabled.
Default value: 0.
## max_rows_to_sort {#max-rows-to-sort}
A maximum number of rows before sorting. This allows you to limit memory consumption when sorting.

View File

@ -2473,7 +2473,7 @@ See also:
- [distributed_replica_error_cap](#distributed_replica_error_cap)
- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life)
## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms}
## distributed_background_insert_sleep_time_ms {#distributed_background_insert_sleep_time_ms}
Base interval for the [Distributed](../../engines/table-engines/special/distributed.md) table engine to send data. The actual interval grows exponentially in the event of errors.
@ -2483,9 +2483,9 @@ Possible values:
Default value: 100 milliseconds.
## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms}
## distributed_background_insert_max_sleep_time_ms {#distributed_background_insert_max_sleep_time_ms}
Maximum interval for the [Distributed](../../engines/table-engines/special/distributed.md) table engine to send data. Limits exponential growth of the interval set in the [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) setting.
Maximum interval for the [Distributed](../../engines/table-engines/special/distributed.md) table engine to send data. Limits exponential growth of the interval set in the [distributed_background_insert_sleep_time_ms](#distributed_background_insert_sleep_time_ms) setting.
Possible values:
@ -2493,7 +2493,7 @@ Possible values:
Default value: 30000 milliseconds (30 seconds).
## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts}
## distributed_background_insert_batch {#distributed_background_insert_batch}
Enables/disables inserted data sending in batches.
@ -2506,13 +2506,13 @@ Possible values:
Default value: 0.
## distributed_directory_monitor_split_batch_on_failure {#distributed_directory_monitor_split_batch_on_failure}
## distributed_background_insert_split_batch_on_failure {#distributed_background_insert_split_batch_on_failure}
Enables/disables splitting batches on failures.
Sometimes sending particular batch to the remote shard may fail, because of some complex pipeline after (i.e. `MATERIALIZED VIEW` with `GROUP BY`) due to `Memory limit exceeded` or similar errors. In this case, retrying will not help (and this will stuck distributed sends for the table) but sending files from that batch one by one may succeed INSERT.
So installing this setting to `1` will disable batching for such batches (i.e. temporary disables `distributed_directory_monitor_batch_inserts` for failed batches).
So installing this setting to `1` will disable batching for such batches (i.e. temporary disables `distributed_background_insert_batch` for failed batches).
Possible values:
@ -2695,15 +2695,15 @@ Possible values:
Default value: 0.
## insert_distributed_sync {#insert_distributed_sync}
## distributed_foreground_insert {#distributed_foreground_insert}
Enables or disables synchronous data insertion into a [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table.
By default, when inserting data into a `Distributed` table, the ClickHouse server sends data to cluster nodes in asynchronous mode. When `insert_distributed_sync=1`, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).
By default, when inserting data into a `Distributed` table, the ClickHouse server sends data to cluster nodes in background mode. When `distributed_foreground_insert=1`, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).
Possible values:
- 0 — Data is inserted in asynchronous mode.
- 0 — Data is inserted in background mode.
- 1 — Data is inserted in synchronous mode.
Default value: `0`.
@ -2762,7 +2762,7 @@ Result:
## use_compact_format_in_distributed_parts_names {#use_compact_format_in_distributed_parts_names}
Uses compact format for storing blocks for async (`insert_distributed_sync`) INSERT into tables with `Distributed` engine.
Uses compact format for storing blocks for background (`distributed_foreground_insert`) INSERT into tables with `Distributed` engine.
Possible values:
@ -2772,7 +2772,7 @@ Possible values:
Default value: `1`.
:::note
- with `use_compact_format_in_distributed_parts_names=0` changes from cluster definition will not be applied for async INSERT.
- with `use_compact_format_in_distributed_parts_names=0` changes from cluster definition will not be applied for background INSERT.
- with `use_compact_format_in_distributed_parts_names=1` changing the order of the nodes in the cluster definition, will change the `shard_index`/`replica_index` so be aware.
:::
@ -3944,6 +3944,16 @@ Possible values:
Default value: `0`.
## force_optimize_projection_name {#force-optimize-projection_name}
If it is set to a non-empty string, check that this projection is used in the query at least once.
Possible values:
- string: name of projection that used in a query
Default value: `''`.
## alter_sync {#alter-sync}
Allows to set up waiting for actions to be executed on replicas by [ALTER](../../sql-reference/statements/alter/index.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries.

View File

@ -78,6 +78,11 @@ If procfs is supported and enabled on the system, ClickHouse server collects the
- `OSReadBytes`
- `OSWriteBytes`
:::note
`OSIOWaitMicroseconds` is disabled by default in Linux kernels starting from 5.14.x.
You can enable it using `sudo sysctl kernel.task_delayacct=1` or by creating a `.conf` file in `/etc/sysctl.d/` with `kernel.task_delayacct = 1`
:::
## Related content
- Blog: [System Tables and a window into the internals of ClickHouse](https://clickhouse.com/blog/clickhouse-debugging-issues-with-system-tables)

View File

@ -115,7 +115,7 @@ Parameters:
<settings>
<connect_timeout>3</connect_timeout>
<!-- Sync insert is set forcibly, leave it here just in case. -->
<insert_distributed_sync>1</insert_distributed_sync>
<distributed_foreground_insert>1</distributed_foreground_insert>
</settings>
<!-- Copying tasks description.

View File

@ -2016,7 +2016,7 @@ Result:
## addDate
Adds the time interval or date interval to the provided date or date with time.
Adds the time interval to the provided date, date with time or String-encoded date / date with time.
If the addition results in a value outside the bounds of the data type, the result is undefined.
@ -2028,7 +2028,7 @@ addDate(date, interval)
**Arguments**
- `date` — The date or date with time to which `interval` is added. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
- `date` — The date or date with time to which `interval` is added. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), or [String](../../sql-reference/data-types/string.md)
- `interval` — Interval to add. [Interval](../../sql-reference/data-types/special-data-types/interval.md).
**Returned value**
@ -2059,7 +2059,7 @@ Alias: `ADDDATE`
## subDate
Subtracts the time interval or date interval from the provided date or date with time.
Subtracts the time interval from the provided date, date with time or String-encoded date / date with time.
If the subtraction results in a value outside the bounds of the data type, the result is undefined.
@ -2071,7 +2071,7 @@ subDate(date, interval)
**Arguments**
- `date` — The date or date with time from which `interval` is subtracted. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
- `date` — The date or date with time from which `interval` is subtracted. [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md), or [String](../../sql-reference/data-types/string.md)
- `interval` — Interval to subtract. [Interval](../../sql-reference/data-types/special-data-types/interval.md).
**Returned value**

View File

@ -4,60 +4,88 @@ sidebar_position: 36
sidebar_label: DELETE
description: Lightweight deletes simplify the process of deleting data from the database.
keywords: [delete]
title: DELETE Statement
title: The Lightweight DELETE Statement
---
The lightweight `DELETE` statement removes rows from the table `[db.]table` that match the expression `expr`. It is only available for the *MergeTree table engine family.
``` sql
DELETE FROM [db.]table [ON CLUSTER cluster] WHERE expr
DELETE FROM [db.]table [ON CLUSTER cluster] WHERE expr;
```
`DELETE FROM` removes rows from the table `[db.]table` that match the expression `expr`. The deleted rows are marked as deleted immediately and will be automatically filtered out of all subsequent queries. Cleanup of data happens asynchronously in the background. This feature is only available for the MergeTree table engine family.
It is called "lightweight `DELETE`" to contrast it to the [ALTER table DELETE](/en/sql-reference/statements/alter/delete) command, which is a heavyweight process.
For example, the following query deletes all rows from the `hits` table where the `Title` column contains the text `hello`:
## Examples
```sql
-- Deletes all rows from the `hits` table where the `Title` column contains the text `hello`
DELETE FROM hits WHERE Title LIKE '%hello%';
```
Lightweight deletes are asynchronous by default. Set `mutations_sync` equal to 1 to wait for one replica to process the statement, and set `mutations_sync` to 2 to wait for all replicas.
## Lightweight `DELETE` does not delete data from storage immediately
With lightweight `DELETE`, deleted rows are internally marked as deleted immediately and will be automatically filtered out of all subsequent queries. However, cleanup of data happens during the next merge. As a result, it is possible that for an unspecified period, data is not actually deleted from storage and is only marked as deleted.
If you need to guarantee that your data is deleted from storage in a predictable time, consider using the [ALTER table DELETE](/en/sql-reference/statements/alter/delete) command. Note that deleting data using `ALTER table DELETE` may consume significant resources as it recreates all affected parts.
## Deleting large amounts of data
Large deletes can negatively affect ClickHouse performance. If you are attempting to delete all rows from a table, consider using the [`TRUNCATE TABLE`](/en/sql-reference/statements/truncate) command.
If you anticipate frequent deletes, consider using a [custom partitioning key](/en/engines/table-engines/mergetree-family/custom-partitioning-key). You can then use the [`ALTER TABLE...DROP PARTITION`](/en/sql-reference/statements/alter/partition#drop-partitionpart) command to quickly drop all rows associated with that partition.
## Limitations of lightweight `DELETE`
### Lightweight `DELETE`s do not work with projections
Currently, `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation and may require the projection to be rebuilt, negatively affecting `DELETE` performance.
## Performance considerations when using lightweight `DELETE`
**Deleting large volumes of data with the lightweight `DELETE` statement can negatively affect SELECT query performance.**
The following can also negatively impact lightweight `DELETE` performance:
- A heavy `WHERE` condition in a `DELETE` query.
- If the mutations queue is filled with many other mutations, this can possibly lead to performance issues as all mutations on a table are executed sequentially.
- The affected table having a very large number of data parts.
- Having a lot of data in compact parts. In a Compact part, all columns are stored in one file.
## Delete permissions
`DELETE` requires the `ALTER DELETE` privilege. To enable `DELETE` statements on a specific table for a given user, run the following command:
:::note
`DELETE FROM` requires the `ALTER DELETE` privilege:
```sql
grant ALTER DELETE ON db.table to username;
GRANT ALTER DELETE ON db.table to username;
```
:::
## Lightweight Delete Internals
## How lightweight DELETEs work internally in ClickHouse
The idea behind Lightweight Delete is that when a `DELETE FROM table ...` query is executed ClickHouse only saves a mask where each row is marked as either “existing” or as “deleted”. Those “deleted” rows become invisible for subsequent queries, but physically the rows are removed only later by subsequent merges. Writing this mask is usually much more lightweight than what is done by `ALTER table DELETE ...` query.
1. A "mask" is applied to affected rows
### How it is implemented
The mask is implemented as a hidden `_row_exists` system column that stores True for all visible rows and False for deleted ones. This column is only present in a part if some rows in this part were deleted. In other words, the column is not persisted when it has all values equal to True.
When a `DELETE FROM table ...` query is executed, ClickHouse saves a mask where each row is marked as either “existing” or as “deleted”. Those “deleted” rows are omitted for subsequent queries. However, rows are actually only removed later by subsequent merges. Writing this mask is much more lightweight than what is done by an `ALTER table DELETE` query.
## SELECT query
When the column is present `SELECT ... FROM table WHERE condition` query internally is extended by an additional predicate on `_row_exists` and becomes similar to
The mask is implemented as a hidden `_row_exists` system column that stores `True` for all visible rows and `False` for deleted ones. This column is only present in a part if some rows in the part were deleted. This column does not exist when a part has all values equal to `True`.
2. `SELECT` queries are transformed to include the mask
When a masked column is used in a query, the `SELECT ... FROM table WHERE condition` query internally is extended by the predicate on `_row_exists` and is transformed to:
```sql
SELECT ... FROM table PREWHERE _row_exists WHERE condition
SELECT ... FROM table PREWHERE _row_exists WHERE condition
```
At execution time the column `_row_exists` is read to figure out which rows are not visible and if there are many deleted rows it can figure out which granules can be fully skipped when reading the rest of the columns.
At execution time, the column `_row_exists` is read to determine which rows should not be returned. If there are many deleted rows, ClickHouse can determine which granules can be fully skipped when reading the rest of the columns.
## DELETE query
`DELETE FROM table WHERE condition` is translated into `ALTER table UPDATE _row_exists = 0 WHERE condition` mutation. Internally this mutation is executed in 2 steps:
1. `SELECT count() FROM table WHERE condition` for each individual part to figure out if the part is affected.
2. Mutate affected parts, and make hardlinks for unaffected parts. Mutating a part in fact only writes `_row_exists` column and just hardlinks all other columns files in the case of Wide parts. But for Compact parts, all columns are rewritten because they all are stored together in one file.
3. `DELETE` queries are transformed to `ALTER table UPDATE` queries
So if we compare Lightweight Delete to `ALTER DELETE` in the first step they both do the same thing to figure out which parts are affected, but in the second step `ALTER DELETE` does much more work because it reads and rewrites all columns files for the affected parts.
The `DELETE FROM table WHERE condition` is translated into an `ALTER table UPDATE _row_exists = 0 WHERE condition` mutation.
With the described implementation now we can see what can negatively affect 'DELETE FROM' execution time:
- Heavy WHERE condition in DELETE query
- Mutations queue filled with other mutations, because all mutations on a table are executed sequentially
- Table having a very large number of data parts
- Having a lot of data in Compact parts—in a Compact part, all columns are stored in one file.
Internally, this mutation is executed in two steps:
:::note
Currently, Lightweight delete does not work for tables with projection as rows in projection may be affected and require the projection to be rebuilt. Rebuilding projection makes the deletion not lightweight, so this is not supported.
:::
1. A `SELECT count() FROM table WHERE condition` command is executed for each individual part to determine if the part is affected.
2. Based on the commands above, affected parts are then mutated, and hardlinks are created for unaffected parts. In the case of wide parts, the `_row_exists` column for each row is updated and all other columns' files are hardlinked. For compact parts, all columns are re-written because they are all stored together in one file.
From the steps above, we can see that lightweight deletes using the masking technique improves performance over traditional `ALTER table DELETE` commands because `ALTER table DELETE` reads and re-writes all the columns' files for affected parts.
## Related content

View File

@ -668,6 +668,15 @@ If either `LIKE` or `ILIKE` clause is specified, the query returns a list of sys
Returns a list of merges. All merges are listed in the [system.merges](../../operations/system-tables/merges.md) table.
- `table` -- Table name.
- `database` -- The name of the database the table is in.
- `estimate_complete` -- The estimated time to complete (in seconds).
- `elapsed` -- The time elapsed (in seconds) since the merge started.
- `progress` -- The percentage of completed work (0-100 percent).
- `is_mutation` -- 1 if this process is a part mutation.
- `size_compressed` -- The total size of the compressed data of the merged parts.
- `memory_usage` -- Memory consumption of the merge process.
**Syntax**
@ -686,10 +695,9 @@ SHOW MERGES;
Result:
```text
┌─table──────┬─database─┬─estimate_complete─┬─────elapsed─┬─progress─┬─is_mutation─┬─size─────┬─mem───────┐
│ your_table │ default │ 0.14 │ 0.365592338 │ 0.73 │ 0 │ 5.40 MiB │ 10.25 MiB │
└────────────┴──────────┴───────────────────┴─────────────┴──────────┴─────────────┴────────────┴─────────┘
┌─table──────┬─database─┬─estimate_complete─┬─elapsed─┬─progress─┬─is_mutation─┬─size_compressed─┬─memory_usage─┐
│ your_table │ default │ 0.14 │ 0.36 │ 73.01 │ 0 │ 5.40 MiB │ 10.25 MiB │
└────────────┴──────────┴───────────────────┴─────────┴──────────┴─────────────┴─────────────────┴──────────────┘
```
Query:
@ -701,9 +709,8 @@ SHOW MERGES LIKE 'your_t%' LIMIT 1;
Result:
```text
┌─table──────┬─database─┬─estimate_complete─┬─────elapsed─┬─progress─┬─is_mutation─┬─size─────┬─mem───────┐
│ your_table │ default │ 0.05 │ 1.727629065 │ 0.97 │ 0 │ 5.40 MiB │ 10.25 MiB │
└────────────┴──────────┴───────────────────┴─────────────┴──────────┴─────────────┴────────────┴─────────┘
┌─table──────┬─database─┬─estimate_complete─┬─elapsed─┬─progress─┬─is_mutation─┬─size_compressed─┬─memory_usage─┐
│ your_table │ default │ 0.14 │ 0.36 │ 73.01 │ 0 │ 5.40 MiB │ 10.25 MiB │
└────────────┴──────────┴───────────────────┴─────────┴──────────┴─────────────┴─────────────────┴──────────────┘
```

View File

@ -166,7 +166,7 @@ Aborts ClickHouse process (like `kill -9 {$ pid_clickhouse-server}`)
## Managing Distributed Tables
ClickHouse can manage [distributed](../../engines/table-engines/special/distributed.md) tables. When a user inserts data into these tables, ClickHouse first creates a queue of the data that should be sent to cluster nodes, then asynchronously sends it. You can manage queue processing with the [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), and [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) queries. You can also synchronously insert distributed data with the [insert_distributed_sync](../../operations/settings/settings.md#insert_distributed_sync) setting.
ClickHouse can manage [distributed](../../engines/table-engines/special/distributed.md) tables. When a user inserts data into these tables, ClickHouse first creates a queue of the data that should be sent to cluster nodes, then asynchronously sends it. You can manage queue processing with the [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), and [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) queries. You can also synchronously insert distributed data with the [distributed_foreground_insert](../../operations/settings/settings.md#distributed_foreground_insert) setting.
### STOP DISTRIBUTED SENDS

View File

@ -22,7 +22,7 @@ sidebar_label: Distributed
Смотрите также:
- настройка `insert_distributed_sync`
- настройка `distributed_foreground_insert`
- [MergeTree](../mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) для примера
Пример:
@ -131,7 +131,7 @@ logs - имя кластера в конфигурационном файле с
- используются запросы, требующие соединение данных (IN, JOIN) по определённому ключу - тогда если данные шардированы по этому ключу, то можно использовать локальные IN, JOIN вместо GLOBAL IN, GLOBAL JOIN, что кардинально более эффективно.
- используется большое количество серверов (сотни и больше) и большое количество маленьких запросов (запросы отдельных клиентов - сайтов, рекламодателей, партнёров) - тогда, для того, чтобы маленькие запросы не затрагивали весь кластер, имеет смысл располагать данные одного клиента на одном шарде, или сделать двухуровневое шардирование: разбить весь кластер на «слои», где слой может состоять из нескольких шардов; данные для одного клиента располагаются на одном слое, но в один слой можно по мере необходимости добавлять шарды, в рамках которых данные распределены произвольным образом; создаются распределённые таблицы на каждый слой и одна общая распределённая таблица для глобальных запросов.
Запись данных осуществляется полностью асинхронно. При вставке в таблицу, блок данных сначала записывается в файловую систему. Затем, в фоновом режиме отправляются на удалённые серверы при первой возможности. Период отправки регулируется настройками [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) и [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms). Движок таблиц `Distributed` отправляет каждый файл со вставленными данными отдельно, но можно включить пакетную отправку данных настройкой [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts). Эта настройка улучшает производительность кластера за счет более оптимального использования ресурсов сервера-отправителя и сети. Необходимо проверять, что данные отправлены успешно, для этого проверьте список файлов (данных, ожидающих отправки) в каталоге таблицы `/var/lib/clickhouse/data/database/table/`. Количество потоков для выполнения фоновых задач можно задать с помощью настройки [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size).
Запись данных осуществляется полностью асинхронно. При вставке в таблицу, блок данных сначала записывается в файловую систему. Затем, в фоновом режиме отправляются на удалённые серверы при первой возможности. Период отправки регулируется настройками [distributed_background_insert_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_sleep_time_ms) и [distributed_background_insert_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_max_sleep_time_ms). Движок таблиц `Distributed` отправляет каждый файл со вставленными данными отдельно, но можно включить пакетную отправку данных настройкой [distributed_background_insert_batch](../../../operations/settings/settings.md#distributed_background_insert_batch). Эта настройка улучшает производительность кластера за счет более оптимального использования ресурсов сервера-отправителя и сети. Необходимо проверять, что данные отправлены успешно, для этого проверьте список файлов (данных, ожидающих отправки) в каталоге таблицы `/var/lib/clickhouse/data/database/table/`. Количество потоков для выполнения фоновых задач можно задать с помощью настройки [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size).
Если после INSERT-а в Distributed таблицу, сервер перестал существовать или был грубо перезапущен (например, в следствие аппаратного сбоя), то записанные данные могут быть потеряны. Если в директории таблицы обнаружен повреждённый кусок данных, то он переносится в поддиректорию broken и больше не используется.

View File

@ -4,17 +4,17 @@ sidebar_position: 12
sidebar_label: Tutorial
---
# ClickHouse Tutorial {#clickhouse-tutorial}
# Руководство {#clickhouse-tutorial}
## What to Expect from This Tutorial? {#what-to-expect-from-this-tutorial}
## Что вы получите, пройдя это руководство? {#what-to-expect-from-this-tutorial}
By going through this tutorial, youll learn how to set up a simple ClickHouse cluster. Itll be small, but fault-tolerant and scalable. Then we will use one of the example datasets to fill it with data and execute some demo queries.
Пройдя это руководство вы научитесь устанавливать простой кластер Clickhouse. Он будет небольшим, но отказоустойчивым и масштабируемым. Далее мы воспользуемся одним из готовых наборов данных для наполнения кластера данными и выполнения над ними нескольких демонстрационных запросов.
## Single Node Setup {#single-node-setup}
## Установка на одном узле {#single-node-setup}
To postpone the complexities of a distributed environment, well start with deploying ClickHouse on a single server or virtual machine. ClickHouse is usually installed from [deb](../getting-started/install.md#install-from-deb-packages) or [rpm](../getting-started/install.md#from-rpm-packages) packages, but there are [alternatives](../getting-started/install.md#from-docker-image) for the operating systems that do no support them.
Чтобы не погружаться сразу в сложности распределённого окружения мы начнём с развёртывания ClickHouse на одном сервере или одной виртуальной машине. ClickHouse обычно устанавливаается из [deb](../getting-started/install.md#install-from-deb-packages)- или [rpm](../getting-started/install.md#from-rpm-packages)-пакетов, но есть и [альтернативы](../getting-started/install.md#from-docker-image) для операционных систем без соответствующих пакетных менеджеров.
For example, you have chosen `deb` packages and executed:
Например, выбираем нужные `deb`-пакеты и выполняем:
``` bash
sudo apt-get install -y apt-transport-https ca-certificates dirmngr
@ -30,49 +30,49 @@ sudo service clickhouse-server start
clickhouse-client # or "clickhouse-client --password" if you've set up a password.
```
What do we have in the packages that got installed:
Что мы получим по результатам установки этих пакетов:
- `clickhouse-client` package contains [clickhouse-client](../interfaces/cli.md) application, interactive ClickHouse console client.
- `clickhouse-common` package contains a ClickHouse executable file.
- `clickhouse-server` package contains configuration files to run ClickHouse as a server.
- с пакетом `clickhouse-client` будет установлена программа [clickhouse-client](../interfaces/cli.md) — интерактивный консольный клиент ClickHouse.
- пакет `clickhouse-common` включает исполняемый файл ClickHouse.
- пакет `clickhouse-server` содержит конфигурационные файлы для запуска ClickHouse в качестве сервера.
Server config files are located in `/etc/clickhouse-server/`. Before going further, please notice the `<path>` element in `config.xml`. Path determines the location for data storage, so it should be located on volume with large disk capacity; the default value is `/var/lib/clickhouse/`. If you want to adjust the configuration, its not handy to directly edit `config.xml` file, considering it might get rewritten on future package updates. The recommended way to override the config elements is to create [files in config.d directory](../operations/configuration-files.md) which serve as “patches” to config.xml.
Файлы конфигурации сервера располагаются в каталоге `/etc/clickhouse-server/`. Прежде чем идти дальше, обратите внимание на элемент `<path>` в файле `config.xml`. Путь, задаваемый этим элементом, определяет местоположение данных, таким образом, он должен быть расположен на томе большой ёмкости; значение по умолчанию — `/var/lib/clickhouse/`. Если вы хотите изменить конфигурацию, то лучше не редактировать вручную файл `config.xml`, поскольку он может быть переписан будущими пакетными обновлениями; рекомендуется создать файлы с необходимыми конфигурационными элементами [в каталоге config.d](../operations/configuration-files.md), которые рассматриваются как “патчи” к config.xml.
As you might have noticed, `clickhouse-server` is not launched automatically after package installation. It wont be automatically restarted after updates, either. The way you start the server depends on your init system, usually, it is:
Вы могли заметить, что `clickhouse-server` не запускается автоматически после установки пакетов. Также сервер не будет автоматически перезапускаться после обновлений. Способ запуска сервера зависит от используемой подсистемы инициализации, обычно это делается так:
``` bash
sudo service clickhouse-server start
```
or
или
``` bash
sudo /etc/init.d/clickhouse-server start
```
The default location for server logs is `/var/log/clickhouse-server/`. The server is ready to handle client connections once it logs the `Ready for connections` message.
Журналы сервера по умолчанию ведутся в `/var/log/clickhouse-server/`. Как только в журнале появится сообщение `Ready for connections` — сервер готов принимать клиентские соединения.
Once the `clickhouse-server` is up and running, we can use `clickhouse-client` to connect to the server and run some test queries like `SELECT "Hello, world!";`.
Теперь, когда `clickhouse-server` запущен, можно подключиться к нему с использованием `clickhouse-client` и выполнить тестовый запрос, например, `SELECT 'Hello, world!';`.
<details markdown="1">
<summary>Quick tips for clickhouse-client</summary>
<summary>Советы по использованию clickhouse-client</summary>
Interactive mode:
Интерактивный режим:
``` bash
clickhouse-client
clickhouse-client --host=... --port=... --user=... --password=...
```
Enable multiline queries:
Включить многострочный режим запросов:
``` bash
clickhouse-client -m
clickhouse-client --multiline
```
Run queries in batch-mode:
Включить пакетный режим запуска запросов:
``` bash
clickhouse-client --query='SELECT 1'
@ -80,7 +80,7 @@ echo 'SELECT 1' | clickhouse-client
clickhouse-client <<< 'SELECT 1'
```
Insert data from a file in specified format:
Вставить данные из файла заданного формата:
``` bash
clickhouse-client --query='INSERT INTO table VALUES' < data.txt
@ -89,39 +89,39 @@ clickhouse-client --query='INSERT INTO table FORMAT TabSeparated' < data.tsv
</details>
## Import Sample Dataset {#import-sample-dataset}
## Загрузка набора данных из примеров {#import-sample-dataset}
Now its time to fill our ClickHouse server with some sample data. In this tutorial, well use some anonymized metric data. There are [multiple ways to import the dataset](../getting-started/example-datasets/metrica.md), and for the sake of the tutorial, well go with the most realistic one.
Настало время загрузить в ClickHouse данные из примеров. В этом руководстве мы используем анонимизированные данные посещений сайтов (веб-метрики). Существует [множество способов импортировать набор данных](../getting-started/example-datasets/metrica.md), но для целей данного руководства мы используем наиболее практичный из них.
### Download and Extract Table Data {#download-and-extract-table-data}
### Загрузка и извлечение табличных данных {#download-and-extract-table-data}
``` bash
curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv
curl https://datasets.clickhouse.com/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv
```
The extracted files are about 10GB in size.
Распакованные файлы занимают около 10 ГБ.
### Create Tables {#create-tables}
### Создание таблиц {#create-tables}
As in most databases management systems, ClickHouse logically groups tables into “databases”. Theres a `default` database, but well create a new one named `tutorial`:
ClickHouse, как и большинство СУБД, логически объединяет таблицы в «базы данных». Существует база данных по умолчанию — `default`, но мы созданим новую, дав ей наименование `tutorial`:
``` bash
clickhouse-client --query "CREATE DATABASE IF NOT EXISTS tutorial"
```
Syntax for creating tables is way more complicated compared to databases (see [reference](../sql-reference/statements/create/table.md). In general `CREATE TABLE` statement has to specify three key things:
Ситаксис для создания таблиц более сложен в сравнении с другими СУБД (см. [руководство по SQL](../sql-reference/statements/create/table.md). Оператор `CREATE TABLE` должен указывать на три ключевых момента:
1. Name of table to create.
2. Table schema, i.e. list of columns and their [data types](../sql-reference/data-types/index.md).
3. [Table engine](../engines/table-engines/index.md) and its settings, which determines all the details on how queries to this table will be physically executed.
1. Имя создаваемой таблицы.
2. Схему таблицы, то есть задавать список столбцов и их [типы данных](../sql-reference/data-types/index.md).
3. [Движок таблицы](../engines/table-engines/index.md) и его параметры, которые определяют все детали того, как запросы к данной таблице будут физически исполняться.
There are only two tables to create:
Мы создадим все лишь две таблицы:
- `hits` is a table with each action done by all users on all websites covered by the service.
- `visits` is a table that contains pre-built sessions instead of individual actions.
- таблицу `hits` с действиями, осуществлёнными всеми пользователями на всех сайтах, обслуживаемых сервисом;
- таблицу `visits`, содержащую посещения — преднастроенные сессии вместо каждого действия.
Lets see and execute the real create table queries for these tables:
Выполним операторы `CREATE TABLE` для создания этих таблиц:
``` sql
CREATE TABLE tutorial.hits_v1
@ -462,22 +462,22 @@ ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)
SAMPLE BY intHash32(UserID)
```
You can execute those queries using the interactive mode of `clickhouse-client` (just launch it in a terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you want.
Эти операторы можно выполнить с использованием интерактивного режима в `clickhouse-client` (запустите его из командной строки не указывая заранее запросы) или, при желании, воспользоваться [альтернативным интерфейсом](../interfaces/index.md) .
As we can see, `hits_v1` uses the [basic MergeTree engine](../engines/table-engines/mergetree-family/mergetree.md), while the `visits_v1` uses the [Collapsing](../engines/table-engines/mergetree-family/collapsingmergetree.md) variant.
Как вы можете видеть, `hits_v1` использует [базовый вариант движка MergeTree](../engines/table-engines/mergetree-family/mergetree.md), тогда как `visits_v1` использует вариант [Collapsing](../engines/table-engines/mergetree-family/collapsingmergetree.md).
### Import Data {#import-data}
### Импорт данных {#import-data}
Data import to ClickHouse is done via [INSERT INTO](../sql-reference/statements/insert-into.md) query like in many other SQL databases. However, data is usually provided in one of the [supported serialization formats](../interfaces/formats.md) instead of `VALUES` clause (which is also supported).
Импорт данных в ClickHouse выполняется оператором [INSERT INTO](../sql-reference/statements/insert-into.md) как в большинстве SQL-систем. Однако данные для вставки в таблицы ClickHouse обычно предоставляются в одном из [поддерживаемых форматов](../interfaces/formats.md) вместо их непосредственного указания в предложении `VALUES` (хотя и этот способ поддерживается).
The files we downloaded earlier are in tab-separated format, so heres how to import them via console client:
В нашем случае файлы были загружены ранее в формате со значениями, разделёнными знаком табуляции; импортируем их, указав соответствующие запросы в аргументах командной строки:
``` bash
clickhouse-client --query "INSERT INTO tutorial.hits_v1 FORMAT TSV" --max_insert_block_size=100000 < hits_v1.tsv
clickhouse-client --query "INSERT INTO tutorial.visits_v1 FORMAT TSV" --max_insert_block_size=100000 < visits_v1.tsv
```
ClickHouse has a lot of [settings to tune](../operations/settings/index.md) and one way to specify them in console client is via arguments, as we can see with `--max_insert_block_size`. The easiest way to figure out what settings are available, what do they mean and what the defaults are is to query the `system.settings` table:
ClickHouse оснащён множеством [изменяемых настроек](../operations/settings/index.md) и один из способов их указать — передать при запуске консольного клиенте их в качестве аргументов, как вы видели в этом примере с `--max_insert_block_size`. Простейший способ узнать, какие настройки доступны, что они означают и какие у них значения по умолчанию — запросить содержимое таблицы `system.settings`:
``` sql
SELECT name, value, changed, description
@ -488,23 +488,23 @@ FORMAT TSV
max_insert_block_size 1048576 0 "The maximum block size for insertion, if we control the creation of blocks for insertion."
```
Optionally you can [OPTIMIZE](../sql-reference/statements/optimize.md) the tables after import. Tables that are configured with an engine from MergeTree-family always do merges of data parts in the background to optimize data storage (or at least check if it makes sense). These queries force the table engine to do storage optimization right now instead of some time later:
Можно также применить оператор [OPTIMIZE](../sql-reference/statements/optimize.md) к таблицам после импорта. Для таблиц, созданных с движками семейства MergeTree, слияние частей загруженных данных выполняется в фоновом режиме (по крайней мере проверяется, имеет ли смысл его осуществить); этот оператор принудительно запускает соответствующие процессы слияния вместо того, чтобы эти действия были выполнены в фоне когда-нибудь позже.
``` bash
clickhouse-client --query "OPTIMIZE TABLE tutorial.hits_v1 FINAL"
clickhouse-client --query "OPTIMIZE TABLE tutorial.visits_v1 FINAL"
```
These queries start an I/O and CPU intensive operation, so if the table consistently receives new data, its better to leave it alone and let merges run in the background.
Эти запросы запускают интеснивные по отношению к вводу-выводу и процессорным ресурсам операции, таким образом, если таблица всё ещё получает новые данные, лучше дать возможность слияниям запуститься в фоне.
Now we can check if the table import was successful:
Проверим, успешно ли загрузились данные:
``` bash
clickhouse-client --query "SELECT COUNT(*) FROM tutorial.hits_v1"
clickhouse-client --query "SELECT COUNT(*) FROM tutorial.visits_v1"
```
## Example Queries {#example-queries}
## Примеры запросов {#example-queries}
``` sql
SELECT
@ -526,18 +526,18 @@ FROM tutorial.visits_v1
WHERE (CounterID = 912887) AND (toYYYYMM(StartDate) = 201403)
```
## Cluster Deployment {#cluster-deployment}
## Кластерное развёртывание {#cluster-deployment}
ClickHouse cluster is a homogenous cluster. Steps to set up:
Кластер ClickHouse — гомогенный, то есть все узлы в нём равны, ведущие и ведомые не выделяются. Шаги по установке кластера:
1. Install ClickHouse server on all machines of the cluster
2. Set up cluster configs in configuration files
3. Create local tables on each instance
4. Create a [Distributed table](../engines/table-engines/special/distributed.md)
1. Установить сервер ClickHouse на всех узлах будущего кластера.
2. Прописать кластерные конфигурации в конфигурационных файлах.
3. Создать локальные таблицы на каждом экземпляре.
4. Создать [распределённую таблицу](../engines/table-engines/special/distributed.md).
[Distributed table](../engines/table-engines/special/distributed.md) is actually a kind of “view” to local tables of ClickHouse cluster. SELECT query from a distributed table executes using resources of all clusters shards. You may specify configs for multiple clusters and create multiple distributed tables providing views to different clusters.
[Распределённая таблица](../engines/table-engines/special/distributed.md) — в некотором смысле «представление» над локальными таблицами кластера ClickHouse. Запрос SELECT к распределённой таблице выполняется на всех узлах кластера. Вы можете указать конфигурации для нескольких кластеров и создать множество распределённых таблиц, «смотрящих» на разные кластеры.
Example config for a cluster with three shards, one replica each:
Пример конфигурации кластера с тремя сегментами и одной репликой для каждой:
``` xml
<remote_servers>
@ -564,38 +564,38 @@ Example config for a cluster with three shards, one replica each:
</remote_servers>
```
For further demonstration, lets create a new local table with the same `CREATE TABLE` query that we used for `hits_v1`, but different table name:
Далее создадим новую локальную таблицу с помощью того же запроса `CREATE TABLE`, что использовался для таблицы `hits_v1`, но с другим именем:
``` sql
CREATE TABLE tutorial.hits_local (...) ENGINE = MergeTree() ...
```
Creating a distributed table providing a view into local tables of the cluster:
Создадим распределённую таблицу, обеспечивающую представление над локальными таблицами кластера:
``` sql
CREATE TABLE tutorial.hits_all AS tutorial.hits_local
ENGINE = Distributed(perftest_3shards_1replicas, tutorial, hits_local, rand());
```
A common practice is to create similar Distributed tables on all machines of the cluster. It allows running distributed queries on any machine of the cluster. Also theres an alternative option to create temporary distributed table for a given SELECT query using [remote](../sql-reference/table-functions/remote.md) table function.
Стандартная практика — создание одинаковых распределённых таблиц на всех узлах кластера. Это позволит запускать распределённые запросы с любого узла. Альтернативой может быть создание временной распределённой таблицы для заданного отдельно взятого запроса с использованием табличной функции [remote](../sql-reference/table-functions/remote.md).
Lets run [INSERT SELECT](../sql-reference/statements/insert-into.md) into the Distributed table to spread the table to multiple servers.
Выполним [INSERT SELECT](../sql-reference/statements/insert-into.md) в распределённую таблицу, чтобы распределить данные по нескольким узлам.
``` sql
INSERT INTO tutorial.hits_all SELECT * FROM tutorial.hits_v1;
```
:::danger Notice
This approach is not suitable for the sharding of large tables. Theres a separate tool [clickhouse-copier](../operations/utilities/clickhouse-copier.md) that can re-shard arbitrary large tables.
:::danger Внимание!
Этот подход не годится для сегментирования больших таблиц. Есть инструмент [clickhouse-copier](../operations/utilities/clickhouse-copier.md), специально предназначенный для перераспределения любых больших таблиц.
:::
As you could expect, computationally heavy queries run N times faster if they utilize 3 servers instead of one.
Как и следовало ожидать, вычислительно сложные запросы работают втрое быстрее, если они выполняются на трёх серверах, а не на одном.
In this case, we have used a cluster with 3 shards, and each contains a single replica.
В данном случае мы использовали кластер из трёх сегментов с одной репликой для каждого.
To provide resilience in a production environment, we recommend that each shard should contain 2-3 replicas spread between multiple availability zones or datacenters (or at least racks). Note that ClickHouse supports an unlimited number of replicas.
В продуктивных окружениях для обеспечения надёжности мы рекомендуем чтобы каждый сегмент был защищён 2—3 репликами, разнесёнными на разные зоны отказоустойчивости или разные центры обработки данных (или хотя бы разные стойки). Особо отметим, что ClickHouse поддерживает неограниченное количество реплик.
Example config for a cluster of one shard containing three replicas:
Пример конфигурации кластера с одним сегментом и тремя репликами:
``` xml
<remote_servers>
@ -619,13 +619,13 @@ Example config for a cluster of one shard containing three replicas:
</remote_servers>
```
To enable native replication [ZooKeeper](http://zookeeper.apache.org/) is required. ClickHouse takes care of data consistency on all replicas and runs restore procedure after failure automatically. Its recommended to deploy the ZooKeeper cluster on separate servers (where no other processes including ClickHouse are running).
Для работы встроенной репликации необходимо использовать [ZooKeeper](http://zookeeper.apache.org/). ClickHouse заботится о согласованности данных на всех репликах и автоматически запускает процедуры восстановления в случае сбоев. Рекомендуется развёртывание кластера ZooKeeper на отдельных серверах (на которых не запущено других процессов, в том числе ClickHouse).
:::note Note
ZooKeeper is not a strict requirement: in some simple cases, you can duplicate the data by writing it into all the replicas from your application code. This approach is **not** recommended, in this case, ClickHouse wont be able to guarantee data consistency on all replicas. Thus it becomes the responsibility of your application.
:::note Примечание
Использование ZooKeeper — нестрогая рекомендация: можно продублировать данные, записывая их непосредственно из приложения на несколько реплик. Но этот поход **не рекомедуется** в общем случае, поскольку ClickHouse не сможет гарантировать согласованность данных на всех репликах; обеспечение согласованности станет заботой вашего приложения.
:::
ZooKeeper locations are specified in the configuration file:
Адреса узлов ZooKeeper указываются в файле конфиуграции:
``` xml
<zookeeper>
@ -644,7 +644,7 @@ ZooKeeper locations are specified in the configuration file:
</zookeeper>
```
Also, we need to set macros for identifying each shard and replica which are used on table creation:
Также необходимо в секции macros указать идентификаторы для сегментов и реплик, они нужны будут при создании таблиц:
``` xml
<macros>
@ -653,7 +653,7 @@ Also, we need to set macros for identifying each shard and replica which are use
</macros>
```
If there are no replicas at the moment on replicated table creation, a new first replica is instantiated. If there are already live replicas, the new replica clones data from existing ones. You have an option to create all replicated tables first, and then insert data to it. Another option is to create some replicas and add the others after or during data insertion.
Если в момент создания реплцированной таблицы ни одной реплики ещё нет, то будет создана первая из них. Если уже есть работающие реплики, то в новые реплики данные будут склонированы из существующих. Есть возможность вначале создать реплицируемые таблицы, а затем вставить в них данные. Но можно создать вначале создать только часть реплик и добавить ещё несколько после вставки или в процессе вставки данных.
``` sql
CREATE TABLE tutorial.hits_replica (...)
@ -664,10 +664,10 @@ ENGINE = ReplicatedMergeTree(
...
```
Here we use [ReplicatedMergeTree](../engines/table-engines/mergetree-family/replication.md) table engine. In parameters we specify ZooKeeper path containing shard and replica identifiers.
Здесь мы используем движок [ReplicatedMergeTree](../engines/table-engines/mergetree-family/replication.md). Указываем в параметрах путь в Zookeeper к идентификаторам сегмента и реплики.
``` sql
INSERT INTO tutorial.hits_replica SELECT * FROM tutorial.hits_local;
```
Replication operates in multi-master mode. Data can be loaded into any replica, and the system then syncs it with other instances automatically. Replication is asynchronous so at a given moment, not all replicas may contain recently inserted data. At least one replica should be up to allow data ingestion. Others will sync up data and repair consistency once they will become active again. Note that this approach allows for the low possibility of a loss of recently inserted data.
Репликация работает в режиме мультимастера. Это означает, что данные могут быть загружены на любую из реплик и система автоматически синхронизирует данные между остальными репликами. Репликация асинхронна, то есть в конкретный момент времнени не все реплики могут содержать недавно добавленные данные. Как минимум одна реплика должна быть в строю для приёма данных. Прочие реплики синхронизируются и восстановят согласованное состояния как только снова станут активными. Заметим, что при таком подходе есть вероятность утраты недавно добавленных данных.

View File

@ -108,6 +108,15 @@ sidebar_label: "Ограничения на сложность запроса"
Значение по умолчанию — 0.
## max_bytes_before_external_sort {#settings-max_bytes_before_external_sort}
Включает или отключает выполнение `ORDER BY` во внешней памяти. См. [Детали реализации ORDER BY](../../sql-reference/statements/select/order-by.md#implementation-details)
- Максимальный объем оперативной памяти (в байтах), который может использоваться одной операцией [ORDER BY](../../sql-reference/statements/select/order-by.md). Рекомендуемое значение — половина доступной системной памяти.
- 0 — `ORDER BY` во внешней памяти отключен.
Значение по умолчанию: 0.
## max_rows_to_sort {#max-rows-to-sort}
Максимальное количество строк до сортировки. Позволяет ограничить потребление оперативки при сортировке.

View File

@ -2136,7 +2136,7 @@ SELECT * FROM test_table
- [distributed_replica_error_cap](#settings-distributed_replica_error_cap)
- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life)
## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms}
## distributed_background_insert_sleep_time_ms {#distributed_background_insert_sleep_time_ms}
Основной интервал отправки данных движком таблиц [Distributed](../../engines/table-engines/special/distributed.md). Фактический интервал растёт экспоненциально при возникновении ошибок.
@ -2146,9 +2146,9 @@ SELECT * FROM test_table
Значение по умолчанию: 100 миллисекунд.
## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms}
## distributed_background_insert_max_sleep_time_ms {#distributed_background_insert_max_sleep_time_ms}
Максимальный интервал отправки данных движком таблиц [Distributed](../../engines/table-engines/special/distributed.md). Ограничивает экпоненциальный рост интервала, установленого настройкой [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms).
Максимальный интервал отправки данных движком таблиц [Distributed](../../engines/table-engines/special/distributed.md). Ограничивает экпоненциальный рост интервала, установленого настройкой [distributed_background_insert_sleep_time_ms](#distributed_background_insert_sleep_time_ms).
Возможные значения:
@ -2156,7 +2156,7 @@ SELECT * FROM test_table
Значение по умолчанию: 30000 миллисекунд (30 секунд).
## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts}
## distributed_background_insert_batch {#distributed_background_insert_batch}
Включает/выключает пакетную отправку вставленных данных.
@ -2323,11 +2323,11 @@ SELECT * FROM test_table
Значение по умолчанию: 0.
## insert_distributed_sync {#insert_distributed_sync}
## distributed_foreground_insert {#distributed_foreground_insert}
Включает или отключает режим синхронного добавления данных в распределенные таблицы (таблицы с движком [Distributed](../../engines/table-engines/special/distributed.md#distributed)).
По умолчанию ClickHouse вставляет данные в распределённую таблицу в асинхронном режиме. Если `insert_distributed_sync=1`, то данные вставляются сихронно, а запрос `INSERT` считается выполненным успешно, когда данные записаны на все шарды (по крайней мере на одну реплику для каждого шарда, если `internal_replication = true`).
По умолчанию ClickHouse вставляет данные в распределённую таблицу в асинхронном режиме. Если `distributed_foreground_insert=1`, то данные вставляются сихронно, а запрос `INSERT` считается выполненным успешно, когда данные записаны на все шарды (по крайней мере на одну реплику для каждого шарда, если `internal_replication = true`).
Возможные значения:

View File

@ -111,7 +111,7 @@ $ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --bas
<settings>
<connect_timeout>3</connect_timeout>
<!-- Sync insert is set forcibly, leave it here just in case. -->
<insert_distributed_sync>1</insert_distributed_sync>
<distributed_foreground_insert>1</distributed_foreground_insert>
</settings>
<!-- Copying tasks description.

View File

@ -128,7 +128,7 @@ SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name]
## Управление распределёнными таблицами {#query-language-system-distributed}
ClickHouse может оперировать [распределёнными](../../sql-reference/statements/system.md) таблицами. Когда пользователь вставляет данные в эти таблицы, ClickHouse сначала формирует очередь из данных, которые должны быть отправлены на узлы кластера, а затем асинхронно отправляет подготовленные данные. Вы можете управлять очередью с помощью запросов [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) и [FLUSH DISTRIBUTED](#query_language-system-flush-distributed). Также есть возможность синхронно вставлять распределенные данные с помощью настройки [insert_distributed_sync](../../operations/settings/settings.md#insert_distributed_sync).
ClickHouse может оперировать [распределёнными](../../sql-reference/statements/system.md) таблицами. Когда пользователь вставляет данные в эти таблицы, ClickHouse сначала формирует очередь из данных, которые должны быть отправлены на узлы кластера, а затем асинхронно отправляет подготовленные данные. Вы можете управлять очередью с помощью запросов [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) и [FLUSH DISTRIBUTED](#query_language-system-flush-distributed). Также есть возможность синхронно вставлять распределенные данные с помощью настройки [distributed_foreground_insert](../../operations/settings/settings.md#distributed_foreground_insert).
### STOP DISTRIBUTED SENDS {#query_language-system-stop-distributed-sends}

View File

@ -43,7 +43,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] AS [db2.]name2
**详见**
- [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) 设置
- [distributed_foreground_insert](../../../operations/settings/settings.md#distributed_foreground_insert) 设置
- [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) 查看示例
**分布式设置**
@ -58,24 +58,24 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] AS [db2.]name2
- `max_delay_to_insert` - 最大延迟多少秒插入数据到分布式表如果有很多挂起字节异步发送。默认值60。
- `monitor_batch_inserts` - 等同于 [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts)
- `background_insert_batch` - 等同于 [distributed_background_insert_batch](../../../operations/settings/settings.md#distributed_background_insert_batch)
- `monitor_split_batch_on_failure` - 等同于[distributed_directory_monitor_split_batch_on_failure](../../../operations/settings/settings.md#distributed_directory_monitor_split_batch_on_failure)
- `background_insert_split_batch_on_failure` - 等同于[distributed_background_insert_split_batch_on_failure](../../../operations/settings/settings.md#distributed_background_insert_split_batch_on_failure)
- `monitor_sleep_time_ms` - 等同于 [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms)
- `background_insert_sleep_time_ms` - 等同于 [distributed_background_insert_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_sleep_time_ms)
- `monitor_max_sleep_time_ms` - 等同于 [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms)
- `background_insert_max_sleep_time_ms` - 等同于 [distributed_background_insert_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_max_sleep_time_ms)
::note
**稳定性设置** (`fsync_...`):
- 只影响异步插入(例如:`insert_distributed_sync=false`), 当数据首先存储在启动节点磁盘上然后再异步发送到shard。
- 只影响异步插入(例如:`distributed_foreground_insert=false`), 当数据首先存储在启动节点磁盘上然后再异步发送到shard。
— 可能会显著降低`insert`的性能
- 影响将存储在分布式表文件夹中的数据写入 **接受您插入的节点** 。如果你需要保证写入数据到底层的MergeTree表中请参阅 `system.merge_tree_settings` 中的持久性设置(`...fsync...`)
**插入限制设置** (`..._insert`) 请见:
- [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) 设置
- [distributed_foreground_insert](../../../operations/settings/settings.md#distributed_foreground_insert) 设置
- [prefer_localhost_replica](../../../operations/settings/settings.md#settings-prefer-localhost-replica) 设置
- `bytes_to_throw_insert``bytes_to_delay_insert` 之前处理,所以你不应该设置它的值小于 `bytes_to_delay_insert`
:::
@ -209,7 +209,7 @@ SELECT 查询会被发送到所有分片,并且无论数据在分片中如何
- 使用需要特定键连接数据( IN 或 JOIN )的查询。如果数据是用该键进行分片,则应使用本地 IN 或 JOIN 而不是 GLOBAL IN 或 GLOBAL JOIN这样效率更高。
- 使用大量服务器(上百或更多),但有大量小查询(个别客户的查询 - 网站,广告商或合作伙伴)。为了使小查询不影响整个集群,让单个客户的数据处于单个分片上是有意义的。或者 你可以配置两级分片:将整个集群划分为«层»,一个层可以包含多个分片。单个客户的数据位于单个层上,根据需要将分片添加到层中,层中的数据随机分布。然后给每层创建分布式表,再创建一个全局的分布式表用于全局的查询。
数据是异步写入的。对于分布式表的 INSERT数据块只写本地文件系统。之后会尽快地在后台发送到远程服务器。发送数据的周期性是由[distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms)和[distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms)设置。分布式引擎会分别发送每个插入数据的文件,但是你可以使用[distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts)设置启用批量发送文件。该设置通过更好地利用本地服务器和网络资源来提高集群性能。你应该检查表目录`/var/lib/clickhouse/data/database/table/`中的文件列表(等待发送的数据)来检查数据是否发送成功。执行后台任务的线程数可以通过[background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size)设置。
数据是异步写入的。对于分布式表的 INSERT数据块只写本地文件系统。之后会尽快地在后台发送到远程服务器。发送数据的周期性是由[distributed_background_insert_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_sleep_time_ms)和[distributed_background_insert_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_background_insert_max_sleep_time_ms)设置。分布式引擎会分别发送每个插入数据的文件,但是你可以使用[distributed_background_insert_batch](../../../operations/settings/settings.md#distributed_background_insert_batch)设置启用批量发送文件。该设置通过更好地利用本地服务器和网络资源来提高集群性能。你应该检查表目录`/var/lib/clickhouse/data/database/table/`中的文件列表(等待发送的数据)来检查数据是否发送成功。执行后台任务的线程数可以通过[background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size)设置。
如果在 INSERT 到分布式表时服务器节点丢失或重启设备故障则插入的数据可能会丢失。如果在表目录中检测到损坏的数据分片则会将其转移到«broken»子目录并不再使用。

View File

@ -1088,7 +1088,7 @@ ClickHouse生成异常
- [表引擎分布式](../../engines/table-engines/special/distributed.md)
- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life)
## distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms}
## distributed_background_insert_sleep_time_ms {#distributed_background_insert_sleep_time_ms}
对于基本间隔 [分布](../../engines/table-engines/special/distributed.md) 表引擎发送数据。 在发生错误时,实际间隔呈指数级增长。
@ -1098,9 +1098,9 @@ ClickHouse生成异常
默认值100毫秒。
## distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms}
## distributed_background_insert_max_sleep_time_ms {#distributed_background_insert_max_sleep_time_ms}
的最大间隔 [分布](../../engines/table-engines/special/distributed.md) 表引擎发送数据。 限制在设置的区间的指数增长 [distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) 设置。
的最大间隔 [分布](../../engines/table-engines/special/distributed.md) 表引擎发送数据。 限制在设置的区间的指数增长 [distributed_background_insert_sleep_time_ms](#distributed_background_insert_sleep_time_ms) 设置。
可能的值:
@ -1108,7 +1108,7 @@ ClickHouse生成异常
默认值30000毫秒30秒
## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts}
## distributed_background_insert_batch {#distributed_background_insert_batch}
启用/禁用批量发送插入的数据。

View File

@ -100,7 +100,7 @@ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --base-
<settings>
<connect_timeout>3</connect_timeout>
<!-- Sync insert is set forcibly, leave it here just in case. -->
<insert_distributed_sync>1</insert_distributed_sync>
<distributed_foreground_insert>1</distributed_foreground_insert>
</settings>
<!-- Copying tasks description.

View File

@ -93,7 +93,7 @@ SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name]
## Managing Distributed Tables {#query-language-system-distributed}
ClickHouse可以管理 [distribute](../../engines/table-engines/special/distributed.md)表。当用户向这类表插入数据时ClickHouse首先为需要发送到集群节点的数据创建一个队列然后异步的发送它们。你可以维护队列的处理过程通过[STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), 以及 [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends)。你也可以设置 `insert_distributed_sync`参数来以同步的方式插入分布式数据。
ClickHouse可以管理 [distribute](../../engines/table-engines/special/distributed.md)表。当用户向这类表插入数据时ClickHouse首先为需要发送到集群节点的数据创建一个队列然后异步的发送它们。你可以维护队列的处理过程通过[STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), 以及 [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends)。你也可以设置 `distributed_foreground_insert`参数来以同步的方式插入分布式数据。
### STOP DISTRIBUTED SENDS {#query_language-system-stop-distributed-sends}

View File

@ -58,7 +58,7 @@ void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & c
/// Override important settings
settings_pull.readonly = 1;
settings_pull.prefer_localhost_replica = false;
settings_push.insert_distributed_sync = true;
settings_push.distributed_foreground_insert = true;
settings_push.prefer_localhost_replica = false;
set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME);
@ -66,7 +66,7 @@ void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & c
set_default_value(settings_pull.max_block_size, 8192UL);
set_default_value(settings_pull.preferred_block_size_bytes, 0);
set_default_value(settings_push.insert_distributed_timeout, 0);
set_default_value(settings_push.distributed_background_insert_timeout, 0);
set_default_value(settings_push.alter_sync, 2);
}

View File

@ -151,10 +151,12 @@ public:
}
else if (BitmapKind::Bitmap == kind)
{
auto size = roaring_bitmap->getSizeInBytes();
std::unique_ptr<RoaringBitmap> bitmap = std::make_unique<RoaringBitmap>(*roaring_bitmap);
bitmap->runOptimize();
auto size = bitmap->getSizeInBytes();
writeVarUInt(size, out);
std::unique_ptr<char[]> buf(new char[size]);
roaring_bitmap->write(buf.get());
bitmap->write(buf.get());
out.write(buf.get(), size);
}
}

View File

@ -214,6 +214,12 @@ ASTPtr FunctionNode::toASTImpl(const ConvertToASTOptions & options) const
if (function_name == "_CAST" && (*getArguments().begin())->getNodeType() == QueryTreeNodeType::CONSTANT)
new_options.add_cast_for_constants = false;
/// Avoid cast for `IN tuple(...)` expression.
/// Tuples colud be quite big, and adding a type may significantly increase query size.
/// It should be safe because set type for `column IN tuple` is deduced from `column` type.
if (isNameOfInFunction(function_name) && (*(++getArguments().begin()))->getNodeType() == QueryTreeNodeType::CONSTANT)
new_options.add_cast_for_constants = false;
const auto & parameters = getParameters();
if (!parameters.getNodes().empty())
{

View File

@ -4,6 +4,7 @@
#include <Poco/Net/NetException.h>
#include <Poco/Util/HelpFormatter.h>
#include <Common/ErrorHandlers.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/logger_useful.h>
@ -209,6 +210,9 @@ int IBridge::main(const std::vector<std::string> & /*args*/)
if (is_help)
return Application::EXIT_OK;
static ServerErrorHandler error_handler;
Poco::ErrorHandler::set(&error_handler);
registerFormats();
LOG_INFO(log, "Starting up {} on host: {}, port: {}", bridgeName(), hostname, port);

View File

@ -154,6 +154,17 @@ public:
data.push_back(assert_cast<const Self &>(src).getData()[n]);
}
void insertManyFrom(const IColumn & src, size_t position, size_t length) override
{
ValueType v = assert_cast<const Self &>(src).getData()[position];
data.resize_fill(data.size() + length, v);
}
void insertMany(const Field & field, size_t length) override
{
data.resize_fill(data.size() + length, static_cast<T>(field.get<T>()));
}
void insertData(const char * pos, size_t) override
{
data.emplace_back(unalignedLoad<T>(pos));

View File

@ -195,6 +195,7 @@
M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \
M(FilesystemCacheDownloadQueueElements, "Filesystem cache elements in download queue") \
M(FilesystemCacheDelayedCleanupElements, "Filesystem cache elements in background cleanup queue") \
M(FilesystemCacheHoldFileSegments, "Filesystem cache file segment which are currently hold as unreleasable") \
M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \
M(S3Requests, "S3 requests") \
M(KeeperAliveConnections, "Number of alive connections") \

View File

@ -236,7 +236,7 @@
M(DictCacheLockWriteNs, "Number of nanoseconds spend in waiting for write lock to update the data for the dictionaries of 'cache' types.") \
M(DictCacheLockReadNs, "Number of nanoseconds spend in waiting for read lock to lookup the data for the dictionaries of 'cache' types.") \
\
M(DistributedSyncInsertionTimeoutExceeded, "A timeout has exceeded while waiting for shards during synchronous insertion into a Distributed table (with 'insert_distributed_sync' = 1)") \
M(DistributedSyncInsertionTimeoutExceeded, "A timeout has exceeded while waiting for shards during synchronous insertion into a Distributed table (with 'distributed_foreground_insert' = 1)") \
M(DataAfterMergeDiffersFromReplica, R"(
Number of times data after merge is not byte-identical to the data on another replicas. There could be several reasons:
1. Using newer version of compression library after server update.
@ -422,6 +422,8 @@ The server successfully detected this situation and will download merged part fr
M(FileSegmentUseMicroseconds, "File segment use() time") \
M(FileSegmentRemoveMicroseconds, "File segment remove() time") \
M(FileSegmentHolderCompleteMicroseconds, "File segments holder complete() time") \
M(FilesystemCacheHoldFileSegments, "Filesystem cache file segments count, which were hold") \
M(FilesystemCacheUnusedHoldFileSegments, "Filesystem cache file segments count, which were hold, but not used (because of seek or LIMIT n, etc)") \
\
M(RemoteFSSeeks, "Total number of seeks for async buffer") \
M(RemoteFSPrefetches, "Number of prefetches made with asynchronous reading from remote filesystem") \

View File

@ -2324,13 +2324,15 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(
? list_watches
: watches;
watches_type[zk_request->getPath()].emplace(session_id);
sessions_and_watchers[session_id].emplace(zk_request->getPath());
auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id);
if (add_watch_result.second)
sessions_and_watchers[session_id].emplace(zk_request->getPath());
}
else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists)
{
watches[zk_request->getPath()].emplace(session_id);
sessions_and_watchers[session_id].emplace(zk_request->getPath());
auto add_watch_result = watches[zk_request->getPath()].emplace(session_id);
if (add_watch_result.second)
sessions_and_watchers[session_id].emplace(zk_request->getPath());
}
}

View File

@ -135,7 +135,7 @@ public:
/// Just vector of SHA1 from user:password
using AuthIDs = std::vector<AuthID>;
using SessionAndAuth = std::unordered_map<int64_t, AuthIDs>;
using Watches = std::map<String /* path, relative of root_path */, SessionIDs>;
using Watches = std::unordered_map<String /* path, relative of root_path */, SessionIDs>;
int64_t session_id_counter{1};

View File

@ -124,11 +124,13 @@ class IColumn;
M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \
M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \
\
M(Milliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \
M(Milliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \
M(Bool, distributed_foreground_insert, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in background. When `distributed_foreground_insert` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) ALIAS(insert_distributed_sync) \
M(UInt64, distributed_background_insert_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) ALIAS(insert_distributed_timeout) \
M(Milliseconds, distributed_background_insert_sleep_time_ms, 100, "Sleep time for background INSERTs into Distributed, in case of any errors delay grows exponentially.", 0) ALIAS(distributed_directory_monitor_sleep_time_ms) \
M(Milliseconds, distributed_background_insert_max_sleep_time_ms, 30000, "Maximum sleep time for background INSERTs into Distributed, it limits exponential growth too.", 0) ALIAS(distributed_directory_monitor_max_sleep_time_ms) \
\
M(Bool, distributed_directory_monitor_batch_inserts, false, "Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.", 0) \
M(Bool, distributed_directory_monitor_split_batch_on_failure, false, "Should StorageDistributed DirectoryMonitors try to split batch into smaller in case of failures.", 0) \
M(Bool, distributed_background_insert_batch, false, "Should background INSERTs into Distributed be batched into bigger blocks.", 0) ALIAS(distributed_directory_monitor_batch_inserts) \
M(Bool, distributed_background_insert_split_batch_on_failure, false, "Should batches of the background INSERT into Distributed be split into smaller batches in case of failures.", 0) ALIAS(distributed_directory_monitor_split_batch_on_failure) \
\
M(Bool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \
M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \
@ -304,8 +306,6 @@ class IColumn;
M(UInt64, parts_to_throw_insert, 0, "If more than this number active parts in a single partition of the destination table, throw 'Too many parts ...' exception.", 0) \
M(UInt64, number_of_mutations_to_delay, 0, "If the mutated table contains at least that many unfinished mutations, artificially slow down mutations of table. 0 - disabled", 0) \
M(UInt64, number_of_mutations_to_throw, 0, "If the mutated table contains at least that many unfinished mutations, throw 'Too many mutations ...' exception. 0 - disabled", 0) \
M(Bool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in asynchronous mode. When `insert_distributed_sync` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) \
M(UInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) \
M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \
M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \
M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \
@ -601,6 +601,7 @@ class IColumn;
M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \
M(Bool, optimize_use_implicit_projections, true, "Automatically choose implicit projections to perform SELECT query", 0) \
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
M(String, force_optimize_projection_name, "", "If it is set to a non-empty string, check that this projection is used in the query at least once.", 0) \
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \
M(Bool, insert_null_as_default, true, "Insert DEFAULT values instead of NULL in INSERT SELECT (UNION ALL)", 0) \
@ -835,7 +836,7 @@ class IColumn;
MAKE_OBSOLETE(M, Bool, allow_experimental_geo_types, true) \
\
MAKE_OBSOLETE(M, Milliseconds, async_insert_stale_timeout_ms, 0) \
MAKE_OBSOLETE(M, HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT) \
MAKE_OBSOLETE(M, StreamingHandleErrorMode, handle_kafka_error_mode, StreamingHandleErrorMode::DEFAULT) \
MAKE_OBSOLETE(M, Bool, database_replicated_ddl_output, true) \
MAKE_OBSOLETE(M, UInt64, replication_alter_columns_timeout, 60) \
MAKE_OBSOLETE(M, UInt64, odbc_max_field_size, 0) \

View File

@ -111,9 +111,9 @@ IMPLEMENT_SETTING_ENUM(DistributedDDLOutputMode, ErrorCodes::BAD_ARGUMENTS,
{"null_status_on_timeout", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT},
{"never_throw", DistributedDDLOutputMode::NEVER_THROW}})
IMPLEMENT_SETTING_ENUM(HandleKafkaErrorMode, ErrorCodes::BAD_ARGUMENTS,
{{"default", HandleKafkaErrorMode::DEFAULT},
{"stream", HandleKafkaErrorMode::STREAM}})
IMPLEMENT_SETTING_ENUM(StreamingHandleErrorMode, ErrorCodes::BAD_ARGUMENTS,
{{"default", StreamingHandleErrorMode::DEFAULT},
{"stream", StreamingHandleErrorMode::STREAM}})
IMPLEMENT_SETTING_ENUM(ShortCircuitFunctionEvaluation, ErrorCodes::BAD_ARGUMENTS,
{{"enable", ShortCircuitFunctionEvaluation::ENABLE},

View File

@ -165,7 +165,7 @@ enum class DistributedDDLOutputMode
DECLARE_SETTING_ENUM(DistributedDDLOutputMode)
enum class HandleKafkaErrorMode
enum class StreamingHandleErrorMode
{
DEFAULT = 0, // Ignore errors with threshold.
STREAM, // Put errors to stream in the virtual column named ``_error.
@ -173,7 +173,7 @@ enum class HandleKafkaErrorMode
/*CUSTOM_SYSTEM_TABLE, Put errors to in a custom system table. This is not implemented now. */
};
DECLARE_SETTING_ENUM(HandleKafkaErrorMode)
DECLARE_SETTING_ENUM(StreamingHandleErrorMode)
enum class ShortCircuitFunctionEvaluation
{

View File

@ -0,0 +1,123 @@
#pragma once
#include <cstddef>
#include <Storages/NamedCollectionsHelpers.h>
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
enum class NumpyDataTypeIndex
{
Int8,
Int16,
Int32,
Int64,
UInt8,
UInt16,
UInt32,
UInt64,
Float32,
Float64,
String,
Unicode,
};
class NumpyDataType
{
public:
enum Endianness
{
LITTLE,
BIG,
NONE,
};
NumpyDataTypeIndex type_index;
explicit NumpyDataType(Endianness endianness_) : endianness(endianness_) {}
virtual ~NumpyDataType() = default;
Endianness getEndianness() const { return endianness; }
virtual NumpyDataTypeIndex getTypeIndex() const = 0;
private:
Endianness endianness;
};
class NumpyDataTypeInt : public NumpyDataType
{
public:
NumpyDataTypeInt(Endianness endianness, size_t size_, bool is_signed_) : NumpyDataType(endianness), size(size_), is_signed(is_signed_)
{
switch (size)
{
case 1: type_index = is_signed ? NumpyDataTypeIndex::Int8 : NumpyDataTypeIndex::UInt8; break;
case 2: type_index = is_signed ? NumpyDataTypeIndex::Int16 : NumpyDataTypeIndex::UInt16; break;
case 4: type_index = is_signed ? NumpyDataTypeIndex::Int32 : NumpyDataTypeIndex::UInt32; break;
case 8: type_index = is_signed ? NumpyDataTypeIndex::Int64 : NumpyDataTypeIndex::UInt64; break;
default:
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Incorrect int type with size {}", size);
}
}
NumpyDataTypeIndex getTypeIndex() const override
{
return type_index;
}
bool isSigned() const { return is_signed; }
private:
size_t size;
bool is_signed;
};
class NumpyDataTypeFloat : public NumpyDataType
{
public:
NumpyDataTypeFloat(Endianness endianness, size_t size_) : NumpyDataType(endianness), size(size_)
{
switch (size)
{
case 4: type_index = NumpyDataTypeIndex::Float32; break;
case 8: type_index = NumpyDataTypeIndex::Float64; break;
default:
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Numpy float type with size {} is not supported", size);
}
}
NumpyDataTypeIndex getTypeIndex() const override
{
return type_index;
}
private:
size_t size;
};
class NumpyDataTypeString : public NumpyDataType
{
public:
NumpyDataTypeString(Endianness endianness, size_t size_) : NumpyDataType(endianness), size(size_)
{
type_index = NumpyDataTypeIndex::String;
}
NumpyDataTypeIndex getTypeIndex() const override { return type_index; }
size_t getSize() const { return size; }
private:
size_t size;
};
class NumpyDataTypeUnicode : public NumpyDataType
{
public:
NumpyDataTypeUnicode(Endianness endianness, size_t size_) : NumpyDataType(endianness), size(size_)
{
type_index = NumpyDataTypeIndex::Unicode;
}
NumpyDataTypeIndex getTypeIndex() const override { return type_index; }
size_t getSize() const { return size * 4; }
private:
size_t size;
};

View File

@ -126,7 +126,7 @@ String prepareNullableAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypeP
String prepareTupleAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & tuple_type = assert_cast<const DataTypeTuple &>(*data_type);
auto nested_names_and_types = getCollectedTupleElements(tuple_type);
auto nested_names_and_types = getCollectedTupleElements(tuple_type, false, "CapnProto");
String struct_name = getSchemaMessageName(column_name);
startStruct(buf, struct_name, indent);
@ -222,7 +222,7 @@ String prepareAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & dat
void StructureToCapnProtoSchema::writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_)
{
auto names_and_types = collectNested(names_and_types_);
auto names_and_types = collectNested(names_and_types_, true, "CapnProto");
writeCapnProtoHeader(buf);
startStruct(buf, getSchemaMessageName(message_name), 0);

View File

@ -4,6 +4,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
namespace StructureToFormatSchemaUtils
{
@ -57,27 +62,34 @@ String getSchemaMessageName(const String & column_name)
namespace
{
std::pair<String, String> splitName(const String & name)
std::pair<String, String> splitName(const String & name, bool allow_split_by_underscore)
{
const auto * begin = name.data();
const auto * end = name.data() + name.size();
const auto * it = find_first_symbols<'_', '.'>(begin, end);
const char * it = nullptr;
if (allow_split_by_underscore)
it = find_first_symbols<'_', '.'>(begin, end);
else
it = find_first_symbols<'.'>(begin, end);
String first = String(begin, it);
String second = it == end ? "" : String(it + 1, end);
return {std::move(first), std::move(second)};
}
}
NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types)
NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types, bool allow_split_by_underscore, const String & format_name)
{
/// Find all columns with dots '.' or underscores '_' and move them into a tuple.
/// Find all columns with dots '.' or underscores '_' (if allowed) and move them into a tuple.
/// For example if we have columns 'a.b UInt32, a.c UInt32, x_y String' we will
/// change it to 'a Tuple(b UInt32, c UInt32), x Tuple(y String)'
NamesAndTypesList result;
std::unordered_map<String, NamesAndTypesList> nested;
for (const auto & [name, type] : names_and_types)
{
auto [field_name, nested_name] = splitName(name);
auto [field_name, nested_name] = splitName(name, allow_split_by_underscore);
if (isdigit(field_name[0]))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format {} doesn't support field names that starts with a digit: '{}'", format_name, field_name);
if (nested_name.empty())
result.emplace_back(name, type);
else
@ -90,7 +102,7 @@ NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types)
return result;
}
NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type)
NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type, bool allow_split_by_underscore, const String & format_name)
{
const auto & nested_types = tuple_type.getElements();
Names nested_names;
@ -109,7 +121,7 @@ NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type)
for (size_t i = 0; i != nested_names.size(); ++i)
result.emplace_back(nested_names[i], nested_types[i]);
return collectNested(result);
return collectNested(result, allow_split_by_underscore, format_name);
}
}

View File

@ -19,9 +19,9 @@ namespace StructureToFormatSchemaUtils
String getSchemaMessageName(const String & column_name);
NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types);
NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types, bool allow_split_by_underscore, const String & format_name);
NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type);
NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type, bool allow_split_by_underscore, const String & format_name);
}
}

View File

@ -105,7 +105,7 @@ String prepareArrayAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr &
String prepareTupleAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent)
{
const auto & tuple_type = assert_cast<const DataTypeTuple &>(*data_type);
auto nested_names_and_types = getCollectedTupleElements(tuple_type);
auto nested_names_and_types = getCollectedTupleElements(tuple_type, false, "Protobuf");
String message_name = getSchemaMessageName(column_name);
startMessage(buf, message_name, indent);
@ -202,7 +202,7 @@ String prepareAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data
void StructureToProtobufSchema::writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_)
{
auto names_and_types = collectNested(names_and_types_);
auto names_and_types = collectNested(names_and_types_, false, "Protobuf");
writeProtobufHeader(buf);
startMessage(buf, getSchemaMessageName(message_name), 0);
size_t field_index = 1;

View File

@ -103,6 +103,7 @@ void registerInputFormatMySQLDump(FormatFactory & factory);
void registerInputFormatParquetMetadata(FormatFactory & factory);
void registerInputFormatDWARF(FormatFactory & factory);
void registerInputFormatOne(FormatFactory & factory);
void registerInputFormatNpy(FormatFactory & factory);
#if USE_HIVE
void registerInputFormatHiveText(FormatFactory & factory);
@ -146,6 +147,7 @@ void registerBSONEachRowSchemaReader(FormatFactory & factory);
void registerParquetMetadataSchemaReader(FormatFactory & factory);
void registerDWARFSchemaReader(FormatFactory & factory);
void registerOneSchemaReader(FormatFactory & factory);
void registerNpySchemaReader(FormatFactory & factory);
void registerFileExtensions(FormatFactory & factory);
@ -249,6 +251,7 @@ void registerFormats()
registerInputFormatParquetMetadata(factory);
registerInputFormatDWARF(factory);
registerInputFormatOne(factory);
registerInputFormatNpy(factory);
registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(factory);
registerNonTrivialPrefixAndSuffixCheckerJSONAsString(factory);
@ -287,6 +290,7 @@ void registerFormats()
registerParquetMetadataSchemaReader(factory);
registerDWARFSchemaReader(factory);
registerOneSchemaReader(factory);
registerNpySchemaReader(factory);
}
}

View File

@ -796,22 +796,23 @@ class FunctionBinaryArithmetic : public IFunction
static FunctionOverloadResolverPtr
getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context)
{
bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0);
bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1);
bool first_arg_is_date_or_datetime_or_string = isDateOrDate32OrDateTimeOrDateTime64(type0) || isString(type0);
bool second_arg_is_date_or_datetime_or_string = isDateOrDate32OrDateTimeOrDateTime64(type1) || isString(type1);
/// Exactly one argument must be Date or DateTime
if (first_is_date_or_datetime == second_is_date_or_datetime)
/// Exactly one argument must be Date or DateTime or String
if (first_arg_is_date_or_datetime_or_string == second_arg_is_date_or_datetime_or_string)
return {};
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
/// Special case when the function is plus or minus, one of arguments is Date or DateTime or String and another is Interval.
/// We construct another function (example: addMonths) and call it.
if constexpr (!is_plus && !is_minus)
return {};
const DataTypePtr & type_time = first_is_date_or_datetime ? type0 : type1;
const DataTypePtr & type_interval = first_is_date_or_datetime ? type1 : type0;
const DataTypePtr & type_time = first_arg_is_date_or_datetime_or_string ? type0 : type1;
const DataTypePtr & type_interval = first_arg_is_date_or_datetime_or_string ? type1 : type0;
bool first_or_second_arg_is_string = isString(type0) || isString(type1);
bool interval_is_number = isNumber(type_interval);
const DataTypeInterval * interval_data_type = nullptr;
@ -822,8 +823,12 @@ class FunctionBinaryArithmetic : public IFunction
if (!interval_data_type)
return {};
}
else if (first_or_second_arg_is_string)
{
return {};
}
if (second_is_date_or_datetime && is_minus)
if (second_arg_is_date_or_datetime_or_string && is_minus)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong order of arguments for function {}: "
"argument of type Interval cannot be first", name);
@ -848,11 +853,11 @@ class FunctionBinaryArithmetic : public IFunction
static FunctionOverloadResolverPtr
getFunctionForDateTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context)
{
bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0);
bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1);
bool first_arg_is_date_or_datetime = isDateOrDate32OrDateTimeOrDateTime64(type0);
bool second_arg_is_date_or_datetime = isDateOrDate32OrDateTimeOrDateTime64(type1);
/// Exactly one argument must be Date or DateTime
if (first_is_date_or_datetime == second_is_date_or_datetime)
if (first_arg_is_date_or_datetime == second_arg_is_date_or_datetime)
return {};
if (!isTuple(type0) && !isTuple(type1))
@ -863,7 +868,7 @@ class FunctionBinaryArithmetic : public IFunction
if constexpr (!is_plus && !is_minus)
return {};
if (isTuple(type0) && second_is_date_or_datetime && is_minus)
if (isTuple(type0) && second_arg_is_date_or_datetime && is_minus)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong order of arguments for function {}: "
"argument of Tuple type cannot be first", name);
@ -1123,7 +1128,7 @@ class FunctionBinaryArithmetic : public IFunction
ColumnsWithTypeAndName new_arguments = arguments;
/// Interval argument must be second.
if (isDateOrDate32(arguments[1].type) || isDateTime(arguments[1].type) || isDateTime64(arguments[1].type))
if (isDateOrDate32OrDateTimeOrDateTime64(arguments[1].type) || isString(arguments[1].type))
std::swap(new_arguments[0], new_arguments[1]);
/// Change interval argument type to its representation
@ -1511,7 +1516,7 @@ public:
}
}
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
/// Special case when the function is plus or minus, one of arguments is Date/DateTime/String and another is Interval.
if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context))
{
ColumnsWithTypeAndName new_arguments(2);
@ -1520,7 +1525,7 @@ public:
new_arguments[i].type = arguments[i];
/// Interval argument must be second.
if (isDateOrDate32(new_arguments[1].type) || isDateTime(new_arguments[1].type) || isDateTime64(new_arguments[1].type))
if (isDateOrDate32OrDateTimeOrDateTime64(new_arguments[1].type) || isString(new_arguments[1].type))
std::swap(new_arguments[0], new_arguments[1]);
/// Change interval argument to its representation
@ -2104,7 +2109,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A
return executeAggregateAddition(arguments, result_type, input_rows_count);
}
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
/// Special case when the function is plus or minus, one of arguments is Date/DateTime/String and another is Interval.
if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0].type, arguments[1].type, context))
{
return executeDateTimeIntervalPlusMinus(arguments, result_type, input_rows_count, function_builder);

View File

@ -1,5 +1,4 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
@ -32,11 +31,19 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors args{
{"date", &isDateOrDate32OrDateTimeOrDateTime64<IDataType>, nullptr, "Date or date with time"},
{"interval", &isInterval<IDataType>, nullptr, "Interval"}
};
validateFunctionArgumentTypes(*this, arguments, args);
if (!isDateOrDate32OrDateTimeOrDateTime64(arguments[0].type) && !isString(arguments[0].type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 1st argument of function {}. Should be a date, a date with time or a string",
arguments[0].type->getName(),
getName());
if (!isInterval(arguments[1].type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 2nd argument of function {}. Should be an interval",
arguments[1].type->getName(),
getName());
auto op = FunctionFactory::instance().get(Op::internal_name, context);
auto op_build = op->build(arguments);
@ -48,7 +55,7 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
if (!isDateOrDate32OrDateTimeOrDateTime64(arguments[0].type) && !isString(arguments[0].type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of 1st argument of function {}. Should be a date or a date with time",

View File

@ -79,9 +79,9 @@ static bool isBetweenZeroAndOne(Float64 v)
return v >= 0.0 && v <= 1.0 && fabs(v - 0.0) >= DBL_EPSILON && fabs(v - 1.0) >= DBL_EPSILON;
}
struct ContinousImpl
struct ContinuousImpl
{
static constexpr auto name = "minSampleSizeContinous";
static constexpr auto name = "minSampleSizeContinuous";
static constexpr size_t num_args = 5;
static constexpr size_t const_args[] = {2, 3, 4};
@ -284,7 +284,9 @@ struct ConversionImpl
REGISTER_FUNCTION(MinSampleSize)
{
factory.registerFunction<FunctionMinSampleSize<ContinousImpl>>();
factory.registerFunction<FunctionMinSampleSize<ContinuousImpl>>();
/// Needed for backward compatibility
factory.registerAlias("minSampleSizeContinous", FunctionMinSampleSize<ContinuousImpl>::name);
factory.registerFunction<FunctionMinSampleSize<ConversionImpl>>();
}

View File

@ -18,7 +18,7 @@ void PeekableWriteBuffer::nextImpl()
size_t prev_size = position() - memory.data();
size_t new_size = memory.size() * 2;
memory.resize(new_size);
BufferBase::set(memory.data(), memory.size(), prev_size);
BufferBase::set(memory.data() + prev_size, memory.size() - prev_size, 0);
return;
}
@ -41,6 +41,7 @@ void PeekableWriteBuffer::dropCheckpoint()
{
assert(checkpoint);
checkpoint = std::nullopt;
/// If we have saved data in own memory, write it to sub-buf.
if (write_to_own_memory)
{
@ -59,9 +60,9 @@ void PeekableWriteBuffer::dropCheckpoint()
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf.offset());
write_to_own_memory = false;
throw;
}
}
}
void PeekableWriteBuffer::rollbackToCheckpoint(bool drop)

View File

@ -578,7 +578,7 @@ static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * nod
Block ActionsDAG::updateHeader(Block header) const
{
std::unordered_map<const Node *, ColumnWithTypeAndName> node_to_column;
IntermediateExecutionResult node_to_column;
std::set<size_t> pos_to_remove;
{
@ -602,8 +602,40 @@ Block ActionsDAG::updateHeader(Block header) const
}
ColumnsWithTypeAndName result_columns;
try
{
result_columns = evaluatePartialResult(node_to_column, outputs, true);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK)
e.addMessage(" in block {}", header.dumpStructure());
throw;
}
if (isInputProjected())
header.clear();
else
header.erase(pos_to_remove);
Block res;
for (auto & col : result_columns)
res.insert(std::move(col));
for (auto && item : header)
res.insert(std::move(item));
return res;
}
ColumnsWithTypeAndName ActionsDAG::evaluatePartialResult(
IntermediateExecutionResult & node_to_column,
const NodeRawConstPtrs & outputs,
bool throw_on_error)
{
ColumnsWithTypeAndName result_columns;
result_columns.reserve(outputs.size());
struct Frame
@ -628,58 +660,50 @@ Block ActionsDAG::updateHeader(Block header) const
while (frame.next_child < node->children.size())
{
const auto * child = node->children[frame.next_child];
++frame.next_child;
if (!node_to_column.contains(child))
{
stack.push({.node = child});
break;
}
++frame.next_child;
}
if (frame.next_child < node->children.size())
if (stack.top().node != node)
continue;
stack.pop();
ColumnsWithTypeAndName arguments(node->children.size());
bool has_all_arguments = true;
for (size_t i = 0; i < arguments.size(); ++i)
{
arguments[i] = node_to_column[node->children[i]];
if (!arguments[i].column)
has_all_arguments = false;
if (!has_all_arguments && throw_on_error)
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK,
"Not found column {} in block {}", node->children[i]->result_name,
header.dumpStructure());
"Not found column {}", node->children[i]->result_name);
}
if (node->type == ActionsDAG::ActionType::INPUT)
if (node->type == ActionsDAG::ActionType::INPUT && throw_on_error)
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK,
"Not found column {} in block {}",
node->result_name, header.dumpStructure());
"Not found column {}",
node->result_name);
node_to_column[node] = executeActionForHeader(node, std::move(arguments));
if (node->type != ActionsDAG::ActionType::INPUT && has_all_arguments)
node_to_column[node] = executeActionForHeader(node, std::move(arguments));
}
}
if (node_to_column[output_node].column)
auto it = node_to_column.find(output_node);
if (it != node_to_column.end())
result_columns.push_back(node_to_column[output_node]);
else
result_columns.emplace_back(nullptr, output_node->result_type, output_node->result_name);
}
}
if (isInputProjected())
header.clear();
else
header.erase(pos_to_remove);
Block res;
for (auto & col : result_columns)
res.insert(std::move(col));
for (auto && item : header)
res.insert(std::move(item));
return res;
return result_columns;
}
NameSet ActionsDAG::foldActionsByProjection(

View File

@ -265,6 +265,12 @@ public:
/// In case if function return constant, but arguments are not constant, materialize it.
Block updateHeader(Block header) const;
using IntermediateExecutionResult = std::unordered_map<const Node *, ColumnWithTypeAndName>;
static ColumnsWithTypeAndName evaluatePartialResult(
IntermediateExecutionResult & node_to_column,
const NodeRawConstPtrs & outputs,
bool throw_on_error);
/// For apply materialize() function for every output.
/// Also add aliases so the result names remain unchanged.
void addMaterializingOutputActions();

View File

@ -159,8 +159,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment:
return { file_segment };
}
const auto & file_segments = *locked_key.getKeyMetadata();
if (file_segments.empty())
if (locked_key.empty())
return {};
FileSegments result;
@ -183,6 +182,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment:
result.push_back(file_segment);
};
const auto & file_segments = locked_key;
auto segment_it = file_segments.lower_bound(range.left);
if (segment_it == file_segments.end())
{
@ -524,7 +524,7 @@ KeyMetadata::iterator FileCache::addFileSegment(
auto file_segment = std::make_shared<FileSegment>(key, offset, size, result_state, settings, background_download_threads > 0, this, locked_key.getKeyMetadata());
auto file_segment_metadata = std::make_shared<FileSegmentMetadata>(std::move(file_segment));
auto [file_segment_metadata_it, inserted] = locked_key.getKeyMetadata()->emplace(offset, file_segment_metadata);
auto [file_segment_metadata_it, inserted] = locked_key.emplace(offset, file_segment_metadata);
if (!inserted)
{
throw Exception(
@ -816,7 +816,7 @@ void FileCache::loadMetadata()
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheLoadMetadataMicroseconds);
if (!metadata.empty())
if (!metadata.isEmpty())
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
@ -1023,7 +1023,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
key_metadata,
cache_it);
inserted = key_metadata->emplace(offset, std::make_shared<FileSegmentMetadata>(std::move(file_segment))).second;
inserted = key_metadata->emplaceUnlocked(offset, std::make_shared<FileSegmentMetadata>(std::move(file_segment))).second;
}
catch (...)
{
@ -1054,8 +1054,10 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
}
}
if (key_metadata->empty())
if (key_metadata->sizeUnlocked() == 0)
{
metadata.removeKey(key, false, false);
}
}
}
@ -1100,7 +1102,7 @@ FileSegments FileCache::getSnapshot(const Key & key)
{
FileSegments file_segments;
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW_LOGICAL);
for (const auto & [_, file_segment_metadata] : *locked_key->getKeyMetadata())
for (const auto & [_, file_segment_metadata] : *locked_key)
file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment));
return file_segments;
}
@ -1129,7 +1131,7 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
std::vector<String> cache_paths;
for (const auto & [offset, file_segment_metadata] : *locked_key->getKeyMetadata())
for (const auto & [offset, file_segment_metadata] : *locked_key)
{
if (file_segment_metadata->file_segment->state() == FileSegment::State::DOWNLOADED)
cache_paths.push_back(metadata.getPathForFileSegment(key, offset, file_segment_metadata->file_segment->getKind()));

View File

@ -1,13 +1,9 @@
#pragma once
#include <atomic>
#include <chrono>
#include <list>
#include <map>
#include <memory>
#include <mutex>
#include <unordered_map>
#include <unordered_set>
#include <boost/functional/hash.hpp>
#include <IO/ReadSettings.h>

View File

@ -23,6 +23,13 @@ namespace ProfileEvents
extern const Event FileSegmentWriteMicroseconds;
extern const Event FileSegmentUseMicroseconds;
extern const Event FileSegmentHolderCompleteMicroseconds;
extern const Event FilesystemCacheHoldFileSegments;
extern const Event FilesystemCacheUnusedHoldFileSegments;
}
namespace CurrentMetrics
{
extern const Metric FilesystemCacheHoldFileSegments;
}
namespace DB
@ -917,10 +924,11 @@ void FileSegment::use()
}
}
FileSegments::iterator FileSegmentsHolder::completeAndPopFrontImpl()
FileSegmentsHolder::FileSegmentsHolder(FileSegments && file_segments_, bool complete_on_dtor_)
: file_segments(std::move(file_segments_)), complete_on_dtor(complete_on_dtor_)
{
front().complete();
return file_segments.erase(file_segments.begin());
CurrentMetrics::add(CurrentMetrics::FilesystemCacheHoldFileSegments, file_segments.size());
ProfileEvents::increment(ProfileEvents::FilesystemCacheHoldFileSegments, file_segments.size());
}
FileSegmentsHolder::~FileSegmentsHolder()
@ -930,10 +938,26 @@ FileSegmentsHolder::~FileSegmentsHolder()
if (!complete_on_dtor)
return;
ProfileEvents::increment(ProfileEvents::FilesystemCacheUnusedHoldFileSegments, file_segments.size());
for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end();)
file_segment_it = completeAndPopFrontImpl();
}
FileSegments::iterator FileSegmentsHolder::completeAndPopFrontImpl()
{
front().complete();
CurrentMetrics::sub(CurrentMetrics::FilesystemCacheHoldFileSegments);
return file_segments.erase(file_segments.begin());
}
FileSegment & FileSegmentsHolder::add(FileSegmentPtr && file_segment)
{
file_segments.push_back(file_segment);
CurrentMetrics::add(CurrentMetrics::FilesystemCacheHoldFileSegments);
ProfileEvents::increment(ProfileEvents::FilesystemCacheHoldFileSegments);
return *file_segments.back();
}
String FileSegmentsHolder::toString()
{
String ranges;

View File

@ -324,8 +324,7 @@ struct FileSegmentsHolder : private boost::noncopyable
{
FileSegmentsHolder() = default;
explicit FileSegmentsHolder(FileSegments && file_segments_, bool complete_on_dtor_ = true)
: file_segments(std::move(file_segments_)), complete_on_dtor(complete_on_dtor_) {}
explicit FileSegmentsHolder(FileSegments && file_segments_, bool complete_on_dtor_ = true);
~FileSegmentsHolder();
@ -341,11 +340,7 @@ struct FileSegmentsHolder : private boost::noncopyable
FileSegment & back() { return *file_segments.back(); }
FileSegment & add(FileSegmentPtr && file_segment)
{
file_segments.push_back(file_segment);
return *file_segments.back();
}
FileSegment & add(FileSegmentPtr && file_segment);
FileSegments::iterator begin() { return file_segments.begin(); }
FileSegments::iterator end() { return file_segments.end(); }

View File

@ -134,7 +134,6 @@ std::string KeyMetadata::getFileSegmentPath(const FileSegment & file_segment) co
/ CacheMetadata::getFileNameForFileSegment(file_segment.offset(), file_segment.getKind());
}
CacheMetadata::CacheMetadata(const std::string & path_)
: path(path_)
, cleanup_queue(std::make_shared<CleanupQueue>())
@ -168,12 +167,18 @@ String CacheMetadata::getPathForKey(const Key & key) const
return fs::path(path) / key_str.substr(0, 3) / key_str;
}
CacheMetadataGuard::Lock CacheMetadata::lockMetadata() const
CacheMetadataGuard::Lock CacheMetadata::MetadataBucket::lock() const
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheLockMetadataMicroseconds);
return guard.lock();
}
CacheMetadata::MetadataBucket & CacheMetadata::getMetadataBucket(const Key & key)
{
const auto bucket = key.key % buckets_num;
return metadata_buckets[bucket];
}
LockedKeyPtr CacheMetadata::lockKeyMetadata(
const FileCacheKey & key,
KeyNotFoundPolicy key_not_found_policy,
@ -220,10 +225,11 @@ KeyMetadataPtr CacheMetadata::getKeyMetadata(
KeyNotFoundPolicy key_not_found_policy,
bool is_initial_load)
{
auto lock = lockMetadata();
auto & bucket = getMetadataBucket(key);
auto lock = bucket.lock();
auto it = find(key);
if (it == end())
auto it = bucket.find(key);
if (it == bucket.end())
{
if (key_not_found_policy == KeyNotFoundPolicy::THROW)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}` in cache", key);
@ -232,7 +238,7 @@ KeyMetadataPtr CacheMetadata::getKeyMetadata(
else if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL)
return nullptr;
it = emplace(
it = bucket.emplace(
key, std::make_shared<KeyMetadata>(
key, getPathForKey(key), cleanup_queue, download_queue, log, key_prefix_directory_mutex, is_initial_load)).first;
}
@ -240,52 +246,66 @@ KeyMetadataPtr CacheMetadata::getKeyMetadata(
return it->second;
}
bool CacheMetadata::isEmpty() const
{
for (const auto & bucket : metadata_buckets)
if (!bucket.empty())
return false;
return true;
}
void CacheMetadata::iterate(IterateFunc && func)
{
auto lock = lockMetadata();
for (auto & [key, key_metadata] : *this)
for (auto & bucket : metadata_buckets)
{
auto locked_key = key_metadata->lockNoStateCheck();
const auto key_state = locked_key->getKeyState();
if (key_state == KeyMetadata::KeyState::ACTIVE)
auto lk = bucket.lock();
for (auto & [key, key_metadata] : bucket)
{
func(*locked_key);
continue;
}
else if (key_state == KeyMetadata::KeyState::REMOVING)
continue;
auto locked_key = key_metadata->lockNoStateCheck();
const auto key_state = locked_key->getKeyState();
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Cannot lock key {}: key does not exist", key_metadata->key);
if (key_state == KeyMetadata::KeyState::ACTIVE)
{
func(*locked_key);
continue;
}
else if (key_state == KeyMetadata::KeyState::REMOVING)
continue;
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Cannot lock key {}: key does not exist", key_metadata->key);
}
}
}
void CacheMetadata::removeAllKeys(bool if_releasable)
{
auto lock = lockMetadata();
for (auto it = begin(); it != end();)
for (auto & bucket : metadata_buckets)
{
auto locked_key = it->second->lockNoStateCheck();
if (locked_key->getKeyState() == KeyMetadata::KeyState::ACTIVE)
auto lock = bucket.lock();
for (auto it = bucket.begin(); it != bucket.end();)
{
bool removed_all = locked_key->removeAllFileSegments(if_releasable);
if (removed_all)
auto locked_key = it->second->lockNoStateCheck();
if (locked_key->getKeyState() == KeyMetadata::KeyState::ACTIVE)
{
it = removeEmptyKey(it, *locked_key, lock);
continue;
bool removed_all = locked_key->removeAllFileSegments(if_releasable);
if (removed_all)
{
it = removeEmptyKey(bucket, it, *locked_key, lock);
continue;
}
}
++it;
}
++it;
}
}
void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasable)
{
auto metadata_lock = lockMetadata();
auto it = find(key);
if (it == end())
auto & bucket = getMetadataBucket(key);
auto lock = bucket.lock();
auto it = bucket.find(key);
if (it == bucket.end())
{
if (if_exists)
return;
@ -305,18 +325,23 @@ void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasabl
bool removed_all = locked_key->removeAllFileSegments(if_releasable);
if (removed_all)
removeEmptyKey(it, *locked_key, metadata_lock);
removeEmptyKey(bucket, it, *locked_key, lock);
}
CacheMetadata::iterator CacheMetadata::removeEmptyKey(iterator it, LockedKey & locked_key, const CacheMetadataGuard::Lock &)
CacheMetadata::MetadataBucket::iterator
CacheMetadata::removeEmptyKey(
MetadataBucket & bucket,
MetadataBucket::iterator it,
LockedKey & locked_key,
const CacheMetadataGuard::Lock &)
{
const auto & key = locked_key.getKey();
if (!it->second->empty())
if (!locked_key.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove non-empty key: {}", key);
locked_key.markAsRemoved();
auto next_it = erase(it);
auto next_it = bucket.erase(it);
LOG_DEBUG(log, "Key {} is removed from metadata", key);
@ -412,16 +437,18 @@ void CacheMetadata::cleanupThreadFunc()
try
{
auto lock = lockMetadata();
auto it = find(key);
if (it == end())
auto & bucket = getMetadataBucket(key);
auto lock = bucket.lock();
auto it = bucket.find(key);
if (it == bucket.end())
continue;
auto locked_key = it->second->lockNoStateCheck();
if (locked_key->getKeyState() == KeyMetadata::KeyState::REMOVING)
{
removeEmptyKey(it, *locked_key, lock);
removeEmptyKey(bucket, it, *locked_key, lock);
}
}
catch (...)

View File

@ -38,12 +38,13 @@ struct FileSegmentMetadata : private boost::noncopyable
using FileSegmentMetadataPtr = std::shared_ptr<FileSegmentMetadata>;
struct KeyMetadata : public std::map<size_t, FileSegmentMetadataPtr>,
struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
private boost::noncopyable,
public std::enable_shared_from_this<KeyMetadata>
{
friend struct LockedKey;
using Key = FileCacheKey;
using iterator = iterator;
KeyMetadata(
const Key & key_,
@ -75,6 +76,13 @@ struct KeyMetadata : public std::map<size_t, FileSegmentMetadataPtr>,
std::string getFileSegmentPath(const FileSegment & file_segment) const;
/// This method is used for loadMetadata() on server startup,
/// where we know there is no concurrency on Key and we do not want therefore taking a KeyGuard::Lock,
/// therefore we use this Unlocked version. This method should not be used anywhere else.
template< class... Args >
auto emplaceUnlocked(Args &&... args) { return emplace(std::forward<Args>(args)...); }
size_t sizeUnlocked() const { return size(); }
private:
KeyState key_state = KeyState::ACTIVE;
KeyGuard guard;
@ -88,7 +96,7 @@ private:
using KeyMetadataPtr = std::shared_ptr<KeyMetadata>;
struct CacheMetadata : public std::unordered_map<FileCacheKey, KeyMetadataPtr>, private boost::noncopyable
struct CacheMetadata
{
public:
using Key = FileCacheKey;
@ -107,6 +115,7 @@ public:
static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind);
void iterate(IterateFunc && func);
bool isEmpty() const;
enum class KeyNotFoundPolicy
{
@ -146,16 +155,30 @@ public:
void cancelDownload();
private:
CacheMetadataGuard::Lock lockMetadata() const;
const std::string path; /// Cache base path
mutable CacheMetadataGuard guard;
CleanupQueuePtr cleanup_queue;
DownloadQueuePtr download_queue;
const CleanupQueuePtr cleanup_queue;
const DownloadQueuePtr download_queue;
std::shared_mutex key_prefix_directory_mutex;
Poco::Logger * log;
struct MetadataBucket : public std::unordered_map<FileCacheKey, KeyMetadataPtr>
{
CacheMetadataGuard::Lock lock() const;
private:
mutable CacheMetadataGuard guard;
};
static constexpr size_t buckets_num = 1024;
std::vector<MetadataBucket> metadata_buckets{buckets_num};
MetadataBucket & getMetadataBucket(const Key & key);
void downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory);
iterator removeEmptyKey(iterator it, LockedKey &, const CacheMetadataGuard::Lock &);
MetadataBucket::iterator removeEmptyKey(
MetadataBucket & bucket,
MetadataBucket::iterator it,
LockedKey &,
const CacheMetadataGuard::Lock &);
};
@ -182,7 +205,15 @@ struct LockedKey : private boost::noncopyable
const Key & getKey() const { return key_metadata->key; }
auto begin() const { return key_metadata->begin(); }
auto rbegin() const { return key_metadata->rbegin(); }
auto end() const { return key_metadata->end(); }
auto rend() const { return key_metadata->rend(); }
bool empty() const { return key_metadata->empty(); }
auto lower_bound(size_t size) const { return key_metadata->lower_bound(size); } /// NOLINT
template< class... Args >
auto emplace(Args &&... args) { return key_metadata->emplace(std::forward<Args>(args)...); }
std::shared_ptr<const FileSegmentMetadata> getByOffset(size_t offset) const;
std::shared_ptr<FileSegmentMetadata> getByOffset(size_t offset);

View File

@ -370,7 +370,7 @@ Chain InterpreterInsertQuery::buildPreSinkChain(
/// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side.
/// Client-side bufferization might cause excessive timeouts (especially in case of big blocks).
if (!(settings.insert_distributed_sync && table->isRemote()) && !async_insert && !no_squash && !(query && query->watch))
if (!(settings.distributed_foreground_insert && table->isRemote()) && !async_insert && !no_squash && !(query && query->watch))
{
bool table_prefers_large_blocks = table->prefersLargeBlocks();

View File

@ -120,9 +120,9 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
if (query.merges)
{
WriteBufferFromOwnString rewritten_query;
rewritten_query << "SELECT table, database, round((elapsed * (1 / progress)) - elapsed, 2) AS estimate_complete, elapsed, "
"round(progress, 2) AS progress, is_mutation, formatReadableSize(total_size_bytes_compressed) AS size, "
"formatReadableSize(memory_usage) AS mem FROM system.merges";
rewritten_query << "SELECT table, database, round((elapsed * (1 / merges.progress)) - merges.elapsed, 2) AS estimate_complete, round(elapsed,2) elapsed, "
"round(progress*100, 2) AS progress, is_mutation, formatReadableSize(total_size_bytes_compressed) AS size_compressed, "
"formatReadableSize(memory_usage) AS memory_usage FROM system.merges";
if (!query.like.empty())
{

View File

@ -95,19 +95,17 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d
if (!identifier)
return;
if (!data.sharding_key_expr->getRequiredColumnsWithTypes().contains(identifier->name()))
auto name = identifier->shortName();
if (!data.sharding_key_expr->getRequiredColumnsWithTypes().contains(name))
return;
/// NOTE: that we should not take care about empty tuple,
/// since after optimize_skip_unused_shards,
/// at least one element should match each shard.
if (auto * tuple_func = right->as<ASTFunction>(); tuple_func && tuple_func->name == "tuple")
{
auto * tuple_elements = tuple_func->children.front()->as<ASTExpressionList>();
std::erase_if(tuple_elements->children, [&](auto & child)
{
auto * literal = child->template as<ASTLiteral>();
return literal && !shardContains(literal->value, identifier->name(), data);
return tuple_elements->children.size() > 1 && literal && !shardContains(literal->value, name, data);
});
}
else if (auto * tuple_literal = right->as<ASTLiteral>();
@ -116,7 +114,7 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d
auto & tuple = tuple_literal->value.get<Tuple &>();
std::erase_if(tuple, [&](auto & child)
{
return !shardContains(child, identifier->name(), data);
return tuple.size() > 1 && !shardContains(child, name, data);
});
}
}

View File

@ -346,6 +346,24 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co
return res;
}
bool Set::hasNull() const
{
checkIsCreated();
if (!transform_null_in)
return false;
if (data_types.size() != 1)
return false;
if (!data_types[0]->isNullable())
return false;
auto col = data_types[0]->createColumn();
col->insert(Field());
auto res = execute({ColumnWithTypeAndName(std::move(col), data_types[0], std::string())}, false);
return res->getBool(0);
}
bool Set::empty() const
{

View File

@ -67,6 +67,8 @@ public:
*/
ColumnPtr execute(const ColumnsWithTypeAndName & columns, bool negative) const;
bool hasNull() const;
bool empty() const;
size_t getTotalRowCount() const;
size_t getTotalByteCount() const;

View File

@ -1,21 +1,29 @@
#include <Interpreters/evaluateConstantExpression.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnSet.h>
#include <Common/typeid_cast.h>
#include <Core/Block.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/TreeRewriter.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSubquery.h>
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/typeid_cast.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <unordered_map>
@ -368,6 +376,325 @@ namespace
return {};
}
/// This is a map which stores constants for a single conjunction.
/// It can contain execution results from different stanges.
/// Example: for expression `(a + b) * c` and predicate `a = 1 and b = 2 and a + b = 3` the map will be
/// a -> 1, b -> 2, a + b -> 3
/// It is allowed to have a map with contradictive conditions, like for `a = 1 and b = 2 and a + b = 5`,
/// but a map for predicate like `a = 1 and a = 2` cannot be built.
using ConjunctionMap = ActionsDAG::IntermediateExecutionResult;
using DisjunctionList = std::list<ConjunctionMap>;
std::optional<ConjunctionMap> andConjunctions(const ConjunctionMap & lhs, const ConjunctionMap & rhs)
{
ConjunctionMap res;
for (const auto & [node, column] : rhs)
{
auto it = lhs.find(node);
/// If constants are different, the conjunction is invalid.
if (it != lhs.end() && column.column->compareAt(0, 0, *it->second.column, 1))
return {};
if (it == lhs.end())
res.emplace(node, column);
}
res.insert(lhs.begin(), lhs.end());
return res;
}
DisjunctionList andDisjunctions(const DisjunctionList & lhs, const DisjunctionList & rhs)
{
DisjunctionList res;
for (const auto & lhs_map : lhs)
for (const auto & rhs_map : rhs)
if (auto conj = andConjunctions(lhs_map, rhs_map))
res.emplace_back(std::move(*conj));
return res;
}
DisjunctionList orDisjunctions(DisjunctionList && lhs, DisjunctionList && rhs)
{
lhs.splice(lhs.end(), std::move(rhs));
return lhs;
}
const ActionsDAG::Node * findMatch(const ActionsDAG::Node * key, const MatchedTrees::Matches & matches)
{
auto it = matches.find(key);
if (it == matches.end())
return {};
const auto & match = it->second;
if (!match.node || match.monotonicity)
return nullptr;
return match.node;
}
ColumnPtr tryCastColumn(ColumnPtr col, const DataTypePtr & from_type, const DataTypePtr & to_type)
{
auto to_type_no_lc = recursiveRemoveLowCardinality(to_type);
// std::cerr << ".. casting " << from_type->getName() << " -> " << to_type_no_lc->getName() << std::endl;
if (!to_type_no_lc->canBeInsideNullable())
return {};
auto res = castColumnAccurateOrNull({col, from_type, std::string()}, makeNullable(to_type_no_lc));
if (res->onlyNull())
return nullptr;
if (!typeid_cast<const ColumnNullable *>(res.get()))
return nullptr;
return res;
}
std::optional<ConjunctionMap::value_type> analyzeConstant(
const ActionsDAG::Node * key,
const ActionsDAG::Node * value,
const MatchedTrees::Matches & matches)
{
if (value->type != ActionsDAG::ActionType::COLUMN)
return {};
if (const auto * col = typeid_cast<const ColumnConst *>(value->column.get()))
{
if (const auto * node = findMatch(key, matches))
{
ColumnPtr column = col->getPtr();
if (!value->result_type->equals(*node->result_type))
{
auto inner = tryCastColumn(col->getDataColumnPtr(), value->result_type, node->result_type);
if (!inner || inner->isNullAt(0))
return {};
auto innder_column = node->result_type->createColumn();
innder_column->insert((*inner)[0]);
column = ColumnConst::create(std::move(innder_column), 1);
}
return ConjunctionMap::value_type{node, {column, node->result_type, node->result_name}};
}
}
return {};
}
std::optional<DisjunctionList> analyzeSet(
const ActionsDAG::Node * key,
const ActionsDAG::Node * value,
const MatchedTrees::Matches & matches,
const ContextPtr & context,
size_t max_elements)
{
if (value->type != ActionsDAG::ActionType::COLUMN)
return {};
auto col = value->column;
if (const auto * col_const = typeid_cast<const ColumnConst *>(col.get()))
col = col_const->getDataColumnPtr();
const auto * col_set = typeid_cast<const ColumnSet *>(col.get());
if (!col_set || !col_set->getData())
return {};
auto * set_from_tuple = typeid_cast<FutureSetFromTuple *>(col_set->getData().get());
if (!set_from_tuple)
return {};
SetPtr set = set_from_tuple->buildOrderedSetInplace(context);
if (!set || !set->hasExplicitSetElements())
return {};
const auto * node = findMatch(key, matches);
if (!node)
return {};
auto elements = set->getSetElements();
auto types = set->getElementsTypes();
ColumnPtr column;
DataTypePtr type;
if (elements.empty())
return {};
if (elements.size() == 1)
{
column = elements[0];
type = types[0];
}
else
{
column = ColumnTuple::create(std::move(elements));
type = std::make_shared<DataTypeTuple>(std::move(types));
}
if (column->size() > max_elements)
return {};
ColumnPtr casted_col;
const NullMap * null_map = nullptr;
if (!type->equals(*node->result_type))
{
casted_col = tryCastColumn(column, value->result_type, node->result_type);
if (!casted_col)
return {};
const auto & col_nullable = assert_cast<const ColumnNullable &>(*casted_col);
null_map = &col_nullable.getNullMapData();
column = col_nullable.getNestedColumnPtr();
}
DisjunctionList res;
if (node->result_type->isNullable() && set->hasNull())
{
auto col_null = node->result_type->createColumnConst(1, Field());
res.push_back({ConjunctionMap{{node, {col_null, node->result_type, node->result_name}}}});
}
size_t num_rows = column->size();
for (size_t row = 0; row < num_rows; ++row)
{
if (null_map && (*null_map)[row])
continue;
auto innder_column = node->result_type->createColumn();
innder_column->insert((*column)[row]);
auto column_const = ColumnConst::create(std::move(innder_column), 1);
res.push_back({ConjunctionMap{{node, {std::move(column_const), node->result_type, node->result_name}}}});
}
return res;
}
std::optional<DisjunctionList> analyze(const ActionsDAG::Node * node, const MatchedTrees::Matches & matches, const ContextPtr & context, size_t max_elements)
{
if (node->type == ActionsDAG::ActionType::FUNCTION)
{
if (node->function_base->getName() == "equals")
{
const auto * lhs_node = node->children.at(0);
const auto * rhs_node = node->children.at(1);
if (auto val = analyzeConstant(lhs_node, rhs_node, matches))
return DisjunctionList{ConjunctionMap{std::move(*val)}};
if (auto val = analyzeConstant(rhs_node, lhs_node, matches))
return DisjunctionList{ConjunctionMap{std::move(*val)}};
}
else if (node->function_base->getName() == "in")
{
const auto * lhs_node = node->children.at(0);
const auto * rhs_node = node->children.at(1);
return analyzeSet(lhs_node, rhs_node, matches, context, max_elements);
}
else if (node->function_base->getName() == "or")
{
DisjunctionList res;
for (const auto * child : node->children)
{
auto val = analyze(child, matches, context, max_elements);
if (!val)
return {};
if (val->size() + res.size() > max_elements)
return {};
res = orDisjunctions(std::move(res), std::move(*val));
}
return res;
}
else if (node->function_base->getName() == "and")
{
std::vector<DisjunctionList> lists;
for (const auto * child : node->children)
{
auto val = analyze(child, matches, context, max_elements);
if (!val)
continue;
lists.push_back(std::move(*val));
}
if (lists.empty())
return {};
std::sort(lists.begin(), lists.end(),
[](const auto & lhs, const auto & rhs) { return lhs.size() < rhs.size(); });
DisjunctionList res;
bool first = true;
for (auto & list : lists)
{
if (first)
{
first = false;
res = std::move(list);
continue;
}
if (res.size() * list.size() > max_elements)
break;
res = andDisjunctions(res, list);
}
return res;
}
}
else if (node->type == ActionsDAG::ActionType::COLUMN)
{
if (isColumnConst(*node->column) && node->result_type->canBeUsedInBooleanContext())
{
if (!node->column->getBool(0))
return DisjunctionList{};
}
}
return {};
}
std::optional<ColumnsWithTypeAndName> evaluateConjunction(
const ActionsDAG::NodeRawConstPtrs & target_expr,
ConjunctionMap && conjunction)
{
auto columns = ActionsDAG::evaluatePartialResult(conjunction, target_expr, false);
for (const auto & column : columns)
if (!column.column)
return {};
return columns;
}
}
std::optional<ConstantVariants> evaluateExpressionOverConstantCondition(
const ActionsDAG::Node * predicate,
const ActionsDAG::NodeRawConstPtrs & expr,
const ContextPtr & context,
size_t max_elements)
{
auto inverted_dag = KeyCondition::cloneASTWithInversionPushDown({predicate}, context);
auto matches = matchTrees(expr, *inverted_dag, false);
auto predicates = analyze(inverted_dag->getOutputs().at(0), matches, context, max_elements);
if (!predicates)
return {};
ConstantVariants res;
for (auto & conjunction : *predicates)
{
auto vals = evaluateConjunction(expr, std::move(conjunction));
if (!vals)
return {};
res.push_back(std::move(*vals));
}
return res;
}
std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr, size_t & limit)
@ -376,7 +703,6 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
if (const auto * fn = node->as<ASTFunction>())
{
std::unordered_map<std::string, bool> always_false_map;
const auto dnf = analyzeFunction(fn, target_expr, limit);
if (dnf.empty() || !limit)
@ -408,6 +734,7 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
for (const auto & conjunct : dnf)
{
Block block;
bool always_false = false;
for (const auto & elem : conjunct)
{
@ -426,23 +753,16 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
Field prev_value = assert_cast<const ColumnConst &>(*prev.column).getField();
Field curr_value = assert_cast<const ColumnConst &>(*elem.column).getField();
if (!always_false_map.contains(elem.name))
{
always_false_map[elem.name] = prev_value != curr_value;
}
else
{
auto & always_false = always_false_map[elem.name];
/// If at least one of conjunct is not always false, we should preserve this.
if (always_false)
{
always_false = prev_value != curr_value;
}
}
always_false = prev_value != curr_value;
if (always_false)
break;
}
}
}
if (always_false)
continue;
// Block should contain all required columns from `target_expr`
if (!has_required_columns(block))
{
@ -466,11 +786,6 @@ std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & nod
return {};
}
}
bool any_always_false = std::any_of(always_false_map.begin(), always_false_map.end(), [](const auto & v) { return v.second; });
if (any_always_false)
return Blocks{};
}
else if (const auto * literal = node->as<ASTLiteral>())
{

View File

@ -3,6 +3,7 @@
#include <Core/Block.h>
#include <Core/Field.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/ActionsDAG.h>
#include <Parsers/IAST.h>
#include <memory>
@ -55,4 +56,12 @@ ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const Cont
*/
std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr, size_t & limit);
using ConstantVariants = std::vector<ColumnsWithTypeAndName>;
/// max_elements is a hint
std::optional<ConstantVariants> evaluateExpressionOverConstantCondition(
const ActionsDAG::Node * predicate,
const ActionsDAG::NodeRawConstPtrs & expr,
const ContextPtr & context,
size_t max_elements);
}

View File

@ -101,6 +101,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int QUERY_WAS_CANCELLED;
extern const int INCORRECT_DATA;
}
@ -1131,6 +1132,29 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
}
}
// Here we check if our our projections contain force_optimize_projection_name
if (!settings.force_optimize_projection_name.value.empty())
{
bool found = false;
std::set<std::string> projections = context->getQueryAccessInfo().projections;
for (const auto &projection : projections)
{
// projection value has structure like: <db_name>.<table_name>.<projection_name>
// We need to get only the projection name
size_t last_dot_pos = projection.find_last_of('.');
std::string projection_name = (last_dot_pos != std::string::npos) ? projection.substr(last_dot_pos + 1) : projection;
if (settings.force_optimize_projection_name.value == projection_name)
{
found = true;
break;
}
}
if (!found)
throw Exception(ErrorCodes::INCORRECT_DATA, "Projection {} is specified in setting force_optimize_projection_name but not used",
settings.force_optimize_projection_name.value);
}
if (process_list_entry)
{

View File

@ -1373,6 +1373,9 @@ void Planner::buildPlanForQueryNode()
{
if (table_expression_data.getPrewhereFilterActions())
result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions());
if (table_expression_data.getRowLevelFilterActions())
result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions());
}
if (query_processing_info.isIntermediateStage())

View File

@ -447,7 +447,7 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage,
const auto & query_context = planner_context->getQueryContext();
auto row_policy_filter = query_context->getRowPolicyFilter(storage_id.getDatabaseName(), storage_id.getTableName(), RowPolicyFilterType::SELECT_FILTER);
if (!row_policy_filter)
if (!row_policy_filter || row_policy_filter->empty())
return {};
return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context);
@ -783,6 +783,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context);
add_filter(row_policy_filter_info, "Row-level security filter");
if (row_policy_filter_info.actions)
table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions);
if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY)
{

View File

@ -245,6 +245,16 @@ public:
return prewhere_filter_actions;
}
void setRowLevelFilterActions(ActionsDAGPtr row_level_filter_actions_value)
{
row_level_filter_actions = std::move(row_level_filter_actions_value);
}
const ActionsDAGPtr & getRowLevelFilterActions() const
{
return row_level_filter_actions;
}
void setPrewhereFilterActions(ActionsDAGPtr prewhere_filter_actions_value)
{
prewhere_filter_actions = std::move(prewhere_filter_actions_value);
@ -290,6 +300,9 @@ private:
/// Valid for table, table function
ActionsDAGPtr prewhere_filter_actions;
/// Valid for table, table function
ActionsDAGPtr row_level_filter_actions;
/// Is storage remote
bool is_remote = false;
};

View File

@ -158,8 +158,7 @@ void ArrowBlockInputFormat::prepareReader()
format_settings.arrow.allow_missing_columns,
format_settings.null_as_default,
format_settings.date_time_overflow_behavior,
format_settings.arrow.case_insensitive_column_matching,
stream);
format_settings.arrow.case_insensitive_column_matching);
if (stream)
record_batch_total = -1;

View File

@ -1058,15 +1058,13 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn(
bool allow_missing_columns_,
bool null_as_default_,
FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior_,
bool case_insensitive_matching_,
bool is_stream_)
bool case_insensitive_matching_)
: header(header_)
, format_name(format_name_)
, allow_missing_columns(allow_missing_columns_)
, null_as_default(null_as_default_)
, date_time_overflow_behavior(date_time_overflow_behavior_)
, case_insensitive_matching(case_insensitive_matching_)
, is_stream(is_stream_)
, date_time_overflow_behavior(date_time_overflow_behavior_)
{
}
@ -1166,10 +1164,6 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr &
if (null_as_default)
insertNullAsDefaultIfNeeded(column, header_column, column_i, block_missing_values);
/// In ArrowStream batches can have different dictionaries.
if (is_stream)
dictionary_infos.clear();
try
{
column.column = castColumn(column, header_column.type);

View File

@ -27,8 +27,7 @@ public:
bool allow_missing_columns_,
bool null_as_default_,
FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior_,
bool case_insensitive_matching_ = false,
bool is_stream_ = false);
bool case_insensitive_matching_ = false);
void arrowTableToCHChunk(Chunk & res, std::shared_ptr<arrow::Table> & table, size_t num_rows, BlockMissingValues * block_missing_values = nullptr);
@ -57,9 +56,8 @@ private:
/// If false, throw exception if some columns in header not exists in arrow table.
bool allow_missing_columns;
bool null_as_default;
FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior;
bool case_insensitive_matching;
bool is_stream;
FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior;
/// Map {column name : dictionary column}.
/// To avoid converting dictionary from Arrow Dictionary

View File

@ -22,6 +22,7 @@ namespace ErrorCodes
extern const int THERE_IS_NO_COLUMN;
}
// This is only used for parquet now.
class ArrowFieldIndexUtil
{
public:
@ -61,30 +62,13 @@ public:
std::unordered_set<int> added_indices;
/// Flat all named fields' index information into a map.
auto fields_indices = calculateFieldIndices(schema);
for (size_t i = 0; i < header.columns(); ++i)
for (size_t i = 0, n = header.columns(); i < n; ++i)
{
const auto & named_col = header.getByPosition(i);
std::string col_name = named_col.name;
if (ignore_case)
boost::to_lower(col_name);
/// Since all named fields are flatten into a map, we should find the column by name
/// in this map.
auto it = fields_indices.find(col_name);
if (it == fields_indices.end())
{
if (!allow_missing_columns)
throw Exception(
ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the following Arrow schema:\n{}\n", named_col.name, schema.ToString());
else
continue;
}
for (int j = 0; j < it->second.second; ++j)
{
auto index = it->second.first + j;
if (added_indices.insert(index).second)
required_indices.emplace_back(index);
}
findRequiredIndices(col_name, named_col.type, fields_indices, added_indices, required_indices);
}
return required_indices;
}
@ -146,9 +130,7 @@ private:
calculateFieldIndices(*sub_field, sub_field->name(), current_start_index, result, full_path_name);
}
}
else if (
field_type->id() == arrow::Type::LIST
&& static_cast<arrow::ListType *>(field_type.get())->value_type()->id() == arrow::Type::STRUCT)
else if (field_type->id() == arrow::Type::LIST)
{
// It is a nested table.
const auto * list_type = static_cast<arrow::ListType *>(field_type.get());
@ -159,12 +141,75 @@ private:
// rewrite it back to the original value.
index_info.first = index_snapshot;
}
else if (field_type->id() == arrow::Type::MAP)
{
const auto * map_type = static_cast<arrow::MapType *>(field_type.get());
auto index_snapshot = current_start_index;
current_start_index += countIndicesForType(map_type->key_type());
calculateFieldIndices(*map_type->item_field(), field_name, current_start_index, result, name_prefix);
index_info.first = index_snapshot;
}
else
{
current_start_index += countIndicesForType(field_type);
}
index_info.second = current_start_index - index_info.first;
}
void findRequiredIndices(
const String & name,
DataTypePtr data_type,
const std::unordered_map<std::string, std::pair<int, int>> & field_indices,
std::unordered_set<int> & added_indices,
std::vector<int> & required_indices)
{
auto nested_type = removeNullable(data_type);
if (const DB::DataTypeTuple * type_tuple = typeid_cast<const DB::DataTypeTuple *>(nested_type.get()))
{
if (type_tuple->haveExplicitNames())
{
auto field_names = type_tuple->getElementNames();
auto field_types = type_tuple->getElements();
for (size_t i = 0, n = field_names.size(); i < n; ++i)
{
auto field_name = field_names[i];
if (ignore_case)
boost::to_lower(field_name);
const auto & field_type = field_types[i];
findRequiredIndices(Nested::concatenateName(name, field_name), field_type, field_indices, added_indices, required_indices);
}
return;
}
}
else if (const auto * type_array = typeid_cast<const DB::DataTypeArray *>(nested_type.get()))
{
findRequiredIndices(name, type_array->getNestedType(), field_indices, added_indices, required_indices);
return;
}
else if (const auto * type_map = typeid_cast<const DB::DataTypeMap *>(nested_type.get()))
{
findRequiredIndices(name, type_map->getKeyType(), field_indices, added_indices, required_indices);
findRequiredIndices(name, type_map->getValueType(), field_indices, added_indices, required_indices);
return;
}
auto it = field_indices.find(name);
if (it == field_indices.end())
{
if (!allow_missing_columns)
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({})", name);
}
else
{
for (int j = 0; j < it->second.second; ++j)
{
auto index = it->second.first + j;
if (added_indices.insert(index).second)
{
required_indices.emplace_back(index);
}
}
}
}
};
}
#endif

View File

@ -58,7 +58,6 @@ namespace DB
extern const int UNKNOWN_TYPE;
extern const int LOGICAL_ERROR;
extern const int DECIMAL_OVERFLOW;
extern const int ILLEGAL_COLUMN;
}
static const std::initializer_list<std::pair<String, std::shared_ptr<arrow::DataType>>> internal_type_to_arrow_type =
@ -357,18 +356,6 @@ namespace DB
}
}
static void checkIfIndexesTypeIsExceeded(const std::shared_ptr<arrow::DataType> & arrow_type, size_t dict_size)
{
const auto & dict_indexes_arrow_type = assert_cast<const arrow::DictionaryType *>(arrow_type.get())->index_type();
/// We use UInt32 or UInt64 type for indexes. It makes sense to check overflow only for UInt32.
const auto * indexes_uint32_type = typeid_cast<const arrow::UInt32Type *>(dict_indexes_arrow_type.get());
if (indexes_uint32_type && dict_size > UINT32_MAX)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Cannot convert ClickHouse LowCardinality column to Arrow Dictionary column:"
" resulting dictionary size exceeds the max value of index type UInt32");
}
template<typename ValueType>
static void fillArrowArrayWithLowCardinalityColumnDataImpl(
const String & column_name,
@ -409,7 +396,6 @@ namespace DB
{
const auto & new_values = new_dict.getNestedColumn();
mapping = dict.uniqueInsertRangeFrom(*new_values, 0, new_values->size());
checkIfIndexesTypeIsExceeded(array_builder->type(), dict.size());
}
}
@ -813,26 +799,18 @@ namespace DB
static std::shared_ptr<arrow::DataType> getArrowTypeForLowCardinalityIndexes(ColumnPtr indexes_column)
{
/// Arrow docs recommend preferring signed integers over unsigned integers for representing dictionary indices.
/// https://arrow.apache.org/docs/format/Columnar.html#dictionary-encoded-layout
switch (indexes_column->getDataType())
{
/// In ClickHouse blocks with same header can contain LowCardinality columns with
/// different dictionaries.
/// Arrow supports only single dictionary for all batches, but it allows to extend
/// dictionary if previous dictionary is a prefix of a new one.
/// But it can happen that LowCardinality columns contains UInt8 indexes columns
/// but resulting extended arrow dictionary will exceed UInt8 and we will need UInt16,
/// but it's not possible to change the type of Arrow dictionary indexes as it's
/// written in Arrow schema.
/// We can just always use type UInt64, but it can be inefficient.
/// In most cases UInt32 should be enough (with more unique values using dictionary is quite meaningless).
/// So we use minimum UInt32 type here (actually maybe even UInt16 will be enough).
/// In case if it's exceeded during dictionary extension, an exception will be thrown.
case TypeIndex::UInt8:
return arrow::int8();
case TypeIndex::UInt16:
return arrow::int16();
case TypeIndex::UInt32:
return arrow::uint32();
return arrow::int32();
case TypeIndex::UInt64:
return arrow::uint64();
return arrow::int64();
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Indexes column for getUniqueIndex must be ColumnUInt, got {}.", indexes_column->getName());
}

View File

@ -1,6 +1,7 @@
#include <Processors/Formats/Impl/JSONRowInputFormat.h>
#include <Formats/JSONUtils.h>
#include <Formats/FormatFactory.h>
#include <Formats/EscapingRuleUtils.h>
#include <IO/ReadHelpers.h>
namespace DB
@ -134,6 +135,11 @@ void registerJSONSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader(
format, [](ReadBuffer & buf, const FormatSettings & format_settings) { return std::make_unique<JSONRowSchemaReader>(buf, format_settings); });
factory.registerAdditionalInfoForSchemaCacheGetter(format, [](const FormatSettings & settings)
{
return getAdditionalFormatInfoByEscapingRule(settings, FormatSettings::EscapingRule::JSON);
});
};
register_schema_reader("JSON");
/// JSONCompact has the same suffix with metadata.

View File

@ -0,0 +1,426 @@
#include <string>
#include <vector>
#include <Processors/Formats/Impl/NpyRowInputFormat.h>
#include <DataTypes/DataTypeString.h>
#include <Common/assert_cast.h>
#include <Common/Exception.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Formats/FormatFactory.h>
#include <Formats/NumpyDataTypes.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/IDataType.h>
#include <IO/ReadBuffer.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <boost/algorithm/string/split.hpp>
#include <IO/ReadBufferFromString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int BAD_ARGUMENTS;
extern const int TOO_LARGE_STRING_SIZE;
extern const int UNKNOWN_TYPE;
extern const int ILLEGAL_COLUMN;
}
namespace
{
DataTypePtr getDataTypeFromNumpyType(const std::shared_ptr<NumpyDataType> & numpy_type)
{
switch (numpy_type->getTypeIndex())
{
case NumpyDataTypeIndex::Int8:
return std::make_shared<DataTypeInt8>();
case NumpyDataTypeIndex::Int16:
return std::make_shared<DataTypeInt16>();
case NumpyDataTypeIndex::Int32:
return std::make_shared<DataTypeInt32>();
case NumpyDataTypeIndex::Int64:
return std::make_shared<DataTypeInt64>();
case NumpyDataTypeIndex::UInt8:
return std::make_shared<DataTypeUInt8>();
case NumpyDataTypeIndex::UInt16:
return std::make_shared<DataTypeUInt16>();
case NumpyDataTypeIndex::UInt32:
return std::make_shared<DataTypeUInt32>();
case NumpyDataTypeIndex::UInt64:
return std::make_shared<DataTypeUInt64>();
case NumpyDataTypeIndex::Float32:
return std::make_shared<DataTypeFloat32>();
case NumpyDataTypeIndex::Float64:
return std::make_shared<DataTypeFloat64>();
case NumpyDataTypeIndex::String:
return std::make_shared<DataTypeString>();
case NumpyDataTypeIndex::Unicode:
return std::make_shared<DataTypeString>();
}
throw Exception(ErrorCodes::UNKNOWN_TYPE, "Numpy type {} is not supported", magic_enum::enum_name(numpy_type->getTypeIndex()));
}
DataTypePtr createNestedArrayType(const DataTypePtr & nested_type, size_t depth)
{
DataTypePtr result_type = nested_type;
assert(depth > 0);
if (depth > 1)
{
for (size_t i = 0; i < depth - 1; ++i)
result_type = std::make_shared<DataTypeArray>(std::move(result_type));
}
return result_type;
}
size_t parseTypeSize(const std::string & size_str)
{
ReadBufferFromString buf(size_str);
size_t size;
if (!tryReadIntText(size, buf))
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid data type size: {}", size_str);
return size;
}
std::shared_ptr<NumpyDataType> parseType(String type)
{
/// Parse endianness
NumpyDataType::Endianness endianness;
if (type[0] == '<')
endianness = NumpyDataType::Endianness::LITTLE;
else if (type[1] == '>')
endianness = NumpyDataType::Endianness::BIG;
else if (type[0] == '|')
endianness = NumpyDataType::Endianness::NONE;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong header data");
/// Parse type
if (type[1] == 'i')
return std::make_shared<NumpyDataTypeInt>(endianness, parseTypeSize(type.substr(2)), true);
else if (type[1] == 'b')
return std::make_shared<NumpyDataTypeInt>(endianness, parseTypeSize(type.substr(2)), false);
else if (type[1] == 'u')
return std::make_shared<NumpyDataTypeInt>(endianness, parseTypeSize(type.substr(2)), false);
else if (type[1] == 'f')
return std::make_shared<NumpyDataTypeFloat>(endianness, parseTypeSize(type.substr(2)));
else if (type[1] == 'S')
return std::make_shared<NumpyDataTypeString>(endianness, parseTypeSize(type.substr(2)));
else if (type[1] == 'U')
return std::make_shared<NumpyDataTypeUnicode>(endianness, parseTypeSize(type.substr(2)));
else if (type[1] == 'c')
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouse doesn't support complex numeric type");
else if (type[1] == 'O')
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouse doesn't support object types");
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouse doesn't support numpy type '{}'", type);
}
std::vector<int> parseShape(String shape_string)
{
if (!shape_string.starts_with('(') || !shape_string.ends_with(')'))
throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect shape format: {}", shape_string);
std::vector<std::string> result_str;
boost::split(result_str, std::string_view(shape_string.data() + 1, shape_string.size() - 2), boost::is_any_of(","));
std::vector<int> shape;
if (result_str[result_str.size()-1].empty())
result_str.pop_back();
shape.reserve(result_str.size());
for (const String & item : result_str)
{
int value;
ReadBufferFromString buf(item);
skipWhitespaceIfAny(buf);
if (!tryReadIntText(value, buf))
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid shape format: {}", shape_string);
shape.push_back(value);
}
return shape;
}
NumpyHeader parseHeader(ReadBuffer &buf)
{
/// Check magic bytes
const char * magic_string = "\x93NUMPY";
assertString(magic_string, buf);
/// Read npy version.
UInt8 version_major;
UInt8 version_minor;
readBinary(version_major, buf);
readBinary(version_minor, buf);
/// Read header length.
UInt32 header_length;
/// In v1 header length is 2 bytes, in v2 - 4 bytes.
if (version_major == 1)
{
UInt16 header_length_u16;
readBinaryLittleEndian(header_length_u16, buf);
header_length = header_length_u16;
}
else
{
readBinaryLittleEndian(header_length, buf);
}
/// Remember current count of read bytes to skip remaining
/// bytes in header when we find all required fields.
size_t header_start = buf.count();
/// Start parsing header.
String shape;
String descr;
assertChar('{', buf);
skipWhitespaceIfAny(buf);
bool first = true;
while (!checkChar('}', buf))
{
/// Skip delimiter between key-value pairs.
if (!first)
{
skipWhitespaceIfAny(buf);
}
else
{
first = false;
}
/// Read map key.
String key;
readQuotedString(key, buf);
assertChar(':', buf);
skipWhitespaceIfAny(buf);
/// Read map value.
String value;
readQuotedField(value, buf);
assertChar(',', buf);
skipWhitespaceIfAny(buf);
if (key == "descr")
descr = value;
else if (key == "fortran_order")
{
if (value != "false")
throw Exception(ErrorCodes::INCORRECT_DATA, "Fortran order is not supported");
}
else if (key == "shape")
shape = value;
}
if (shape.empty() || descr.empty())
throw Exception(ErrorCodes::INCORRECT_DATA, "npy file header doesn't contain required field 'shape' or 'descr'");
size_t read_bytes = buf.count() - header_start;
if (read_bytes > header_length)
throw Exception(ErrorCodes::INCORRECT_DATA, "Header size is incorrect");
/// Ignore remaining header data.
buf.ignore(header_length - read_bytes);
if (descr[0] == '\'')
descr = descr.substr(1, descr.length() - 1);
if (descr[descr.length() - 1] == '\'')
descr = descr.substr(0, descr.length() - 1);
if (shape[0] == '\'')
shape = shape.substr(1, shape.length() - 1);
if (shape[shape.length() - 1] == '\'')
shape = shape.substr(0, shape.length() - 1);
NumpyHeader res;
res.shape = parseShape(shape);
res.numpy_type = parseType(descr);
return res;
}
DataTypePtr getNestedType(DataTypePtr type)
{
while (const auto * temp_type = typeid_cast<const DataTypeArray *>(type.get()))
type = temp_type->getNestedType();
return type;
}
}
void NpyRowInputFormat::readPrefix()
{
header = parseHeader(*in);
}
NpyRowInputFormat::NpyRowInputFormat(ReadBuffer & in_, Block header_, Params params_)
: IRowInputFormat(std::move(header_), in_, std::move(params_))
{
auto types = getPort().getHeader().getDataTypes();
if (types.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected number of columns for Npy input format, expected one column, got {} columns", types.size());
nested_type = getNestedType(types[0]);
}
template <typename ColumnValue, typename DataValue>
void NpyRowInputFormat::readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness)
{
DataValue value;
if (endianness == NumpyDataType::Endianness::BIG)
readBinaryBigEndian(value, *in);
else
readBinaryLittleEndian(value, *in);
assert_cast<ColumnVector<ColumnValue> &>(*column).insertValue(static_cast<ColumnValue>(value));
}
template <typename T>
void NpyRowInputFormat::readAndInsertInteger(IColumn * column, const DataTypePtr & data_type, const NumpyDataType & npy_type)
{
switch (npy_type.getTypeIndex())
{
case NumpyDataTypeIndex::Int8: readBinaryValueAndInsert<T, UInt8>(column->getPtr(), npy_type.getEndianness()); break;
case NumpyDataTypeIndex::Int16: readBinaryValueAndInsert<T, UInt16>(column->getPtr(), npy_type.getEndianness()); break;
case NumpyDataTypeIndex::Int32: readBinaryValueAndInsert<T, UInt32>(column->getPtr(), npy_type.getEndianness()); break;
case NumpyDataTypeIndex::Int64: readBinaryValueAndInsert<T, UInt64>(column->getPtr(), npy_type.getEndianness()); break;
case NumpyDataTypeIndex::UInt8: readBinaryValueAndInsert<T, UInt8>(column->getPtr(), npy_type.getEndianness()); break;
case NumpyDataTypeIndex::UInt16: readBinaryValueAndInsert<T, UInt16>(column->getPtr(), npy_type.getEndianness()); break;
case NumpyDataTypeIndex::UInt32: readBinaryValueAndInsert<T, UInt32>(column->getPtr(), npy_type.getEndianness()); break;
case NumpyDataTypeIndex::UInt64: readBinaryValueAndInsert<T, UInt64>(column->getPtr(), npy_type.getEndianness()); break;
default:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert Numpy value with type {} into column with type {}",
magic_enum::enum_name(npy_type.getTypeIndex()), data_type->getName());
}
}
template <typename T>
void NpyRowInputFormat::readAndInsertFloat(IColumn * column, const DataTypePtr & data_type, const NumpyDataType & npy_type)
{
switch (npy_type.getTypeIndex())
{
case NumpyDataTypeIndex::Float32: readBinaryValueAndInsert<T, Float32>(column->getPtr(), npy_type.getEndianness()); break;
case NumpyDataTypeIndex::Float64: readBinaryValueAndInsert<T, Float64>(column->getPtr(), npy_type.getEndianness()); break;
default:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert Numpy value with type {} into column with type {}",
magic_enum::enum_name(npy_type.getTypeIndex()), data_type->getName());
}
}
template <typename T>
void NpyRowInputFormat::readAndInsertString(MutableColumnPtr column, const DataTypePtr & data_type, const NumpyDataType & npy_type, bool is_fixed)
{
size_t size;
if (npy_type.getTypeIndex() == NumpyDataTypeIndex::String)
size = assert_cast<const NumpyDataTypeString &>(npy_type).getSize();
else if (npy_type.getTypeIndex() == NumpyDataTypeIndex::Unicode)
size = assert_cast<const NumpyDataTypeUnicode &>(npy_type).getSize();
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert Numpy value with type {} into column with type {}",
magic_enum::enum_name(npy_type.getTypeIndex()), data_type->getName());
if (is_fixed)
{
auto & fixed_string_column = assert_cast<ColumnFixedString &>(*column);
size_t n = fixed_string_column.getN();
if (size > n)
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string for FixedString column");
auto & chars = fixed_string_column.getChars();
size_t prev_size = chars.size();
chars.resize_fill(prev_size + n);
in->readStrict(reinterpret_cast<char *>(chars.data() + prev_size), size);
}
else
{
auto & column_string = assert_cast<ColumnString &>(*column);
String tmp;
tmp.resize(size);
in->readStrict(tmp.data(), size);
tmp.erase(std::remove(tmp.begin(), tmp.end(), '\0'), tmp.end());
column_string.insertData(tmp.c_str(), tmp.size());
}
}
void NpyRowInputFormat::readValue(IColumn * column)
{
switch (nested_type->getTypeId())
{
case TypeIndex::UInt8: readAndInsertInteger<UInt8>(column, nested_type, *header.numpy_type); break;
case TypeIndex::UInt16: readAndInsertInteger<UInt16>(column, nested_type, *header.numpy_type); break;
case TypeIndex::UInt32: readAndInsertInteger<UInt32>(column, nested_type, *header.numpy_type); break;
case TypeIndex::UInt64: readAndInsertInteger<UInt64>(column, nested_type, *header.numpy_type); break;
case TypeIndex::Int8: readAndInsertInteger<Int8>(column, nested_type, *header.numpy_type); break;
case TypeIndex::Int16: readAndInsertInteger<Int16>(column, nested_type, *header.numpy_type); break;
case TypeIndex::Int32: readAndInsertInteger<Int32>(column, nested_type, *header.numpy_type); break;
case TypeIndex::Int64: readAndInsertInteger<Int64>(column, nested_type, *header.numpy_type); break;
case TypeIndex::Float32: readAndInsertFloat<Float32>(column, nested_type, *header.numpy_type); break;
case TypeIndex::Float64: readAndInsertFloat<Float64>(column, nested_type, *header.numpy_type); break;
case TypeIndex::String: readAndInsertString<String>(column->getPtr(), nested_type, *header.numpy_type, false); break;
case TypeIndex::FixedString: readAndInsertString<String>(column->getPtr(), nested_type, *header.numpy_type, true); break;
default:
throw Exception(ErrorCodes::UNKNOWN_TYPE, "ClickHouse type {} is not supported for import from Npy format", nested_type->getName());
}
}
bool NpyRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & /*ext*/)
{
if (in->eof())
return false;
auto & column = columns[0];
IColumn * current_column = column.get();
size_t elements_in_current_column = 1;
for (size_t i = 1; i != header.shape.size(); ++i)
{
auto * array_column = typeid_cast<ColumnArray *>(current_column);
if (!array_column)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected nesting level of column '{}', expected {}, got {}", column->getName(), header.shape.size() - 1, i - 1);
/// Fill offsets of array columns.
for (size_t j = 0; j != elements_in_current_column; ++j)
array_column->getOffsets().push_back(array_column->getOffsets().back() + header.shape[i]);
current_column = &array_column->getData();
elements_in_current_column *= header.shape[i];
}
for (size_t i = 0; i != elements_in_current_column; ++i)
readValue(current_column);
return true;
}
NpySchemaReader::NpySchemaReader(ReadBuffer & in_)
: ISchemaReader(in_) {}
NamesAndTypesList NpySchemaReader::readSchema()
{
NumpyHeader header = parseHeader(in);
DataTypePtr nested_type = getDataTypeFromNumpyType(header.numpy_type);
DataTypePtr result_type = createNestedArrayType(nested_type, header.shape.size());
return {{"array", result_type}};
}
void registerInputFormatNpy(FormatFactory & factory)
{
factory.registerInputFormat("Npy", [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings &)
{
return std::make_shared<NpyRowInputFormat>(buf, sample, std::move(params));
});
factory.markFormatSupportsSubsetOfColumns("Npy");
}
void registerNpySchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("Npy", [](ReadBuffer & buf, const FormatSettings &)
{
return std::make_shared<NpySchemaReader>(buf);
});
}
}

View File

@ -0,0 +1,65 @@
#pragma once
#include <vector>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Columns/IColumn.h>
#include <Core/Field.h>
#include <Core/NamesAndTypes.h>
#include <Core/Types.h>
#include <Formats/NumpyDataTypes.h>
namespace DB
{
class ReadBuffer;
struct NumpyHeader
{
std::vector<int> shape;
std::shared_ptr<NumpyDataType> numpy_type;
};
class NpyRowInputFormat final : public IRowInputFormat
{
public:
NpyRowInputFormat(ReadBuffer & in_, Block header_, Params params_);
String getName() const override { return "NpyRowInputFormat"; }
private:
void readPrefix() override;
bool readRow(MutableColumns & columns, RowReadExtension &) override;
void readData(MutableColumns & columns);
template <typename T>
void readAndInsertInteger(IColumn * column, const DataTypePtr & data_type, const NumpyDataType & npy_type);
template <typename T>
void readAndInsertFloat(IColumn * column, const DataTypePtr & data_type, const NumpyDataType & npy_type);
template <typename T>
void readAndInsertString(MutableColumnPtr column, const DataTypePtr & data_type, const NumpyDataType & npy_type, bool is_fixed);
template <typename ColumnValue, typename DataValue>
void readBinaryValueAndInsert(MutableColumnPtr column, NumpyDataType::Endianness endianness);
void readRows(MutableColumns & columns);
void readValue(IColumn * column);
DataTypePtr nested_type;
NumpyHeader header;
};
class NpySchemaReader : public ISchemaReader
{
public:
explicit NpySchemaReader(ReadBuffer & in_);
private:
NamesAndTypesList readSchema() override;
};
}

View File

@ -19,6 +19,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const
settings.remove_redundant_distinct = from.query_plan_remove_redundant_distinct;
settings.optimize_projection = from.optimize_use_projections;
settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection;
settings.force_projection_name = from.force_optimize_projection_name;
settings.optimize_use_implicit_projections = settings.optimize_projection && from.optimize_use_implicit_projections;
return settings;
}

View File

@ -41,6 +41,7 @@ struct QueryPlanOptimizationSettings
/// If reading from projection can be applied
bool optimize_projection = false;
bool force_use_projection = false;
String force_projection_name;
bool optimize_use_implicit_projections = false;
static QueryPlanOptimizationSettings fromSettings(const Settings & from);

View File

@ -8,7 +8,7 @@
namespace DB
{
MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity)
MatchedTrees::Matches matchTrees(const ActionsDAG::NodeRawConstPtrs & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity)
{
using Parents = std::set<const ActionsDAG::Node *>;
std::unordered_map<const ActionsDAG::Node *, Parents> inner_parents;
@ -16,7 +16,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG
{
std::stack<const ActionsDAG::Node *> stack;
for (const auto * out : inner_dag.getOutputs())
for (const auto * out : inner_dag)
{
if (inner_parents.contains(out))
continue;

View File

@ -39,5 +39,5 @@ struct MatchedTrees
using Matches = std::unordered_map<const ActionsDAG::Node *, Match>;
};
MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity = true);
MatchedTrees::Matches matchTrees(const ActionsDAG::NodeRawConstPtrs & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity = true);
}

View File

@ -175,6 +175,8 @@ static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expressi
dag->mergeInplace(std::move(*expression->clone()));
else
dag = expression->clone();
dag->projectInput(false);
}
/// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain.
@ -235,15 +237,20 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns &
const auto & array_joined_columns = array_join->arrayJoin()->columns;
/// Remove array joined columns from outputs.
/// Types are changed after ARRAY JOIN, and we can't use this columns anyway.
ActionsDAG::NodeRawConstPtrs outputs;
outputs.reserve(dag->getOutputs().size());
for (const auto & output : dag->getOutputs())
if (dag)
{
if (!array_joined_columns.contains(output->result_name))
outputs.push_back(output);
/// Remove array joined columns from outputs.
/// Types are changed after ARRAY JOIN, and we can't use this columns anyway.
ActionsDAG::NodeRawConstPtrs outputs;
outputs.reserve(dag->getOutputs().size());
for (const auto & output : dag->getOutputs())
{
if (!array_joined_columns.contains(output->result_name))
outputs.push_back(output);
}
dag->getOutputs() = std::move(outputs);
}
}
}
@ -338,7 +345,7 @@ InputOrderInfoPtr buildInputOrderInfo(
if (dag)
{
matches = matchTrees(sorting_key_dag, *dag);
matches = matchTrees(sorting_key_dag.getOutputs(), *dag);
for (const auto & [node, match] : matches)
{
@ -507,7 +514,7 @@ AggregationInputOrder buildInputOrderInfo(
if (dag)
{
matches = matchTrees(sorting_key_dag, *dag);
matches = matchTrees(sorting_key_dag.getOutputs(), *dag);
for (const auto & [node, match] : matches)
{

View File

@ -281,7 +281,7 @@ ActionsDAGPtr analyzeAggregateProjection(
{
auto proj_index = buildDAGIndex(*info.before_aggregation);
MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag, false /* check_monotonicity */);
MatchedTrees::Matches matches = matchTrees(info.before_aggregation->getOutputs(), *query.dag, false /* check_monotonicity */);
// for (const auto & [node, match] : matches)
// {

View File

@ -168,7 +168,7 @@ bool isPartitionKeySuitsGroupByKey(
const auto irreducibe_nodes = removeInjectiveFunctionsFromResultsRecursively(group_by_actions);
const auto matches = matchTrees(*group_by_actions, partition_actions);
const auto matches = matchTrees(group_by_actions->getOutputs(), partition_actions);
return allOutputsDependsOnlyOnAllowedNodes(partition_actions, irreducibe_nodes, matches);
}

View File

@ -128,16 +128,16 @@ DistributedAsyncInsertDirectoryQueue::DistributedAsyncInsertDirectoryQueue(
, path(fs::path(disk->getPath()) / relative_path / "")
, broken_relative_path(fs::path(relative_path) / "broken")
, broken_path(fs::path(path) / "broken" / "")
, should_batch_inserts(storage.getDistributedSettingsRef().monitor_batch_inserts)
, split_batch_on_failure(storage.getDistributedSettingsRef().monitor_split_batch_on_failure)
, should_batch_inserts(storage.getDistributedSettingsRef().background_insert_batch)
, split_batch_on_failure(storage.getDistributedSettingsRef().background_insert_split_batch_on_failure)
, dir_fsync(storage.getDistributedSettingsRef().fsync_directories)
, min_batched_block_size_rows(storage.getContext()->getSettingsRef().min_insert_block_size_rows)
, min_batched_block_size_bytes(storage.getContext()->getSettingsRef().min_insert_block_size_bytes)
, current_batch_file_path(path + "current_batch.txt")
, pending_files(std::numeric_limits<size_t>::max())
, default_sleep_time(storage.getDistributedSettingsRef().monitor_sleep_time_ms.totalMilliseconds())
, default_sleep_time(storage.getDistributedSettingsRef().background_insert_sleep_time_ms.totalMilliseconds())
, sleep_time(default_sleep_time)
, max_sleep_time(storage.getDistributedSettingsRef().monitor_max_sleep_time_ms.totalMilliseconds())
, max_sleep_time(storage.getDistributedSettingsRef().background_insert_max_sleep_time_ms.totalMilliseconds())
, log(&Poco::Logger::get(getLoggerName()))
, monitor_blocker(monitor_blocker_)
, metric_pending_bytes(CurrentMetrics::DistributedBytesToInsert, 0)
@ -234,7 +234,7 @@ void DistributedAsyncInsertDirectoryQueue::run()
}
}
else
LOG_TEST(log, "Skipping send data over distributed table.");
LOG_TEST(LogFrequencyLimiter(log, 30), "Skipping send data over distributed table.");
const auto now = std::chrono::system_clock::now();
if (now - last_decrease_time > decrease_error_count_period)
@ -726,7 +726,7 @@ SyncGuardPtr DistributedAsyncInsertDirectoryQueue::getDirectorySyncGuard(const s
std::string DistributedAsyncInsertDirectoryQueue::getLoggerName() const
{
return storage.getStorageID().getFullTableName() + ".DirectoryMonitor." + disk->getName();
return storage.getStorageID().getFullTableName() + ".DistributedInsertQueue." + disk->getName();
}
void DistributedAsyncInsertDirectoryQueue::updatePath(const std::string & new_relative_path)

View File

@ -27,7 +27,7 @@ using ProcessorPtr = std::shared_ptr<IProcessor>;
class ISource;
/** Queue for async INSERT Into Distributed engine (insert_distributed_sync=0).
/** Queue for async INSERT Into Distributed engine (distributed_foreground_insert=0).
*
* Files are added from two places:
* - from filesystem at startup (StorageDistributed::startup())
@ -36,12 +36,10 @@ class ISource;
* Later, in background, those files will be send to the remote nodes.
*
* The behaviour of this queue can be configured via the following settings:
* - distributed_directory_monitor_batch_inserts
* - distributed_directory_monitor_split_batch_on_failure
* - distributed_directory_monitor_sleep_time_ms
* - distributed_directory_monitor_max_sleep_time_ms
* NOTE: It worth to rename the settings too
* ("directory_monitor" in settings looks too internal).
* - distributed_background_insert_batch
* - distributed_background_insert_split_batch_on_failure
* - distributed_background_insert_sleep_time_ms
* - distributed_background_insert_max_sleep_time_ms
*/
class DistributedAsyncInsertDirectoryQueue
{

View File

@ -15,17 +15,17 @@ namespace DB
class ASTStorage;
#define LIST_OF_DISTRIBUTED_SETTINGS(M, ALIAS) \
M(Bool, fsync_after_insert, false, "Do fsync for every inserted. Will decreases performance of inserts (only for async INSERT, i.e. insert_distributed_sync=false)", 0) \
M(Bool, fsync_directories, false, "Do fsync for temporary directory (that is used for async INSERT only) after all part operations (writes, renames, etc.).", 0) \
M(Bool, fsync_after_insert, false, "Do fsync for every inserted. Will decreases performance of inserts (only for background INSERT, i.e. distributed_foreground_insert=false)", 0) \
M(Bool, fsync_directories, false, "Do fsync for temporary directory (that is used for background INSERT only) after all part operations (writes, renames, etc.).", 0) \
/** Inserts settings. */ \
M(UInt64, bytes_to_throw_insert, 0, "If more than this number of compressed bytes will be pending for async INSERT, an exception will be thrown. 0 - do not throw.", 0) \
M(UInt64, bytes_to_delay_insert, 0, "If more than this number of compressed bytes will be pending for async INSERT, the query will be delayed. 0 - do not delay.", 0) \
M(UInt64, max_delay_to_insert, 60, "Max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for async send.", 0) \
/** Directory monitor settings */ \
M(UInt64, monitor_batch_inserts, 0, "Default - distributed_directory_monitor_batch_inserts", 0) \
M(UInt64, monitor_split_batch_on_failure, 0, "Default - distributed_directory_monitor_split_batch_on_failure", 0) \
M(Milliseconds, monitor_sleep_time_ms, 0, "Default - distributed_directory_monitor_sleep_time_ms", 0) \
M(Milliseconds, monitor_max_sleep_time_ms, 0, "Default - distributed_directory_monitor_max_sleep_time_ms", 0) \
M(UInt64, bytes_to_throw_insert, 0, "If more than this number of compressed bytes will be pending for background INSERT, an exception will be thrown. 0 - do not throw.", 0) \
M(UInt64, bytes_to_delay_insert, 0, "If more than this number of compressed bytes will be pending for background INSERT, the query will be delayed. 0 - do not delay.", 0) \
M(UInt64, max_delay_to_insert, 60, "Max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for background send.", 0) \
/** Async INSERT settings */ \
M(UInt64, background_insert_batch, 0, "Default - distributed_background_insert_batch", 0) ALIAS(monitor_batch_inserts) \
M(UInt64, background_insert_split_batch_on_failure, 0, "Default - distributed_background_insert_split_batch_on_failure", 0) ALIAS(monitor_split_batch_on_failure) \
M(Milliseconds, background_insert_sleep_time_ms, 0, "Default - distributed_background_insert_sleep_time_ms", 0) ALIAS(monitor_sleep_time_ms) \
M(Milliseconds, background_insert_max_sleep_time_ms, 0, "Default - distributed_background_insert_max_sleep_time_ms", 0) ALIAS(monitor_max_sleep_time_ms) \
M(Bool, flush_on_detach, true, "Flush data to remote nodes on DETACH/DROP/server shutdown", 0) \
DECLARE_SETTINGS_TRAITS(DistributedSettingsTraits, LIST_OF_DISTRIBUTED_SETTINGS)

View File

@ -763,7 +763,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const
return guard;
};
auto sleep_ms = context->getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds();
auto sleep_ms = context->getSettingsRef().distributed_background_insert_sleep_time_ms.totalMilliseconds();
size_t file_size;
auto it = dir_names.begin();
@ -789,7 +789,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const
NativeWriter stream{compress, DBMS_TCP_PROTOCOL_VERSION, block.cloneEmpty()};
/// Prepare the header.
/// See also readDistributedHeader() in DirectoryMonitor (for reading side)
/// See also DistributedAsyncInsertHeader::read() in DistributedInsertQueue (for reading side)
///
/// We wrap the header into a string for compatibility with older versions:
/// a shard will able to read the header partly and ignore other parts based on its version.

View File

@ -4,7 +4,7 @@
namespace DB
{
class ReadBufferFromFileLog;
class FileLogConsumer;
using ReadBufferFromFileLogPtr = std::shared_ptr<ReadBufferFromFileLog>;
using ReadBufferFromFileLogPtr = std::shared_ptr<FileLogConsumer>;
}

View File

@ -1,8 +1,9 @@
#include <Interpreters/Context.h>
#include <Storages/FileLog/ReadBufferFromFileLog.h>
#include <Storages/FileLog/FileLogConsumer.h>
#include <Common/Stopwatch.h>
#include <Common/logger_useful.h>
#include <IO/ReadBufferFromString.h>
#include <algorithm>
#include <filesystem>
@ -14,15 +15,14 @@ namespace ErrorCodes
extern const int CANNOT_READ_ALL_DATA;
}
ReadBufferFromFileLog::ReadBufferFromFileLog(
FileLogConsumer::FileLogConsumer(
StorageFileLog & storage_,
size_t max_batch_size,
size_t poll_timeout_,
ContextPtr context_,
size_t stream_number_,
size_t max_streams_number_)
: ReadBuffer(nullptr, 0)
, log(&Poco::Logger::get("ReadBufferFromFileLog " + toString(stream_number_)))
: log(&Poco::Logger::get("FileLogConsumer " + toString(stream_number_)))
, storage(storage_)
, batch_size(max_batch_size)
, poll_timeout(poll_timeout_)
@ -31,23 +31,19 @@ ReadBufferFromFileLog::ReadBufferFromFileLog(
, max_streams_number(max_streams_number_)
{
current = records.begin();
allowed = false;
}
bool ReadBufferFromFileLog::poll()
ReadBufferPtr FileLogConsumer::consume()
{
if (hasMorePolledRecords())
{
allowed = true;
return true;
}
return getNextRecord();
auto new_records = pollBatch(batch_size);
if (new_records.empty())
{
buffer_status = BufferStatus::NO_RECORD_RETURNED;
LOG_TRACE(log, "No new records to read");
return false;
return nullptr;
}
else
{
@ -57,12 +53,11 @@ bool ReadBufferFromFileLog::poll()
LOG_TRACE(log, "Polled batch of {} records. ", records.size());
buffer_status = BufferStatus::POLLED_OK;
allowed = true;
return true;
return getNextRecord();
}
}
ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_size_)
FileLogConsumer::Records FileLogConsumer::pollBatch(size_t batch_size_)
{
Records new_records;
new_records.reserve(batch_size_);
@ -84,7 +79,7 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz
return new_records;
}
void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_)
void FileLogConsumer::readNewRecords(FileLogConsumer::Records & new_records, size_t batch_size_)
{
size_t need_records_size = batch_size_ - new_records.size();
size_t read_records_size = 0;
@ -155,21 +150,14 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_
}
}
bool ReadBufferFromFileLog::nextImpl()
ReadBufferPtr FileLogConsumer::getNextRecord()
{
if (!allowed || !hasMorePolledRecords())
return false;
auto * new_position = const_cast<char *>(current->data.data());
BufferBase::set(new_position, current->data.size(), 0);
allowed = false;
current_file = current->file_name;
current_offset = current->offset;
if (!hasMorePolledRecords())
return nullptr;
auto buf = std::make_unique<ReadBufferFromString>(current->data);
++current;
return true;
return buf;
}
}

View File

@ -9,10 +9,10 @@
namespace DB
{
class ReadBufferFromFileLog : public ReadBuffer
class FileLogConsumer
{
public:
ReadBufferFromFileLog(
FileLogConsumer(
StorageFileLog & storage_,
size_t max_batch_size,
size_t poll_timeout_,
@ -20,18 +20,17 @@ public:
size_t stream_number_,
size_t max_streams_number_);
~ReadBufferFromFileLog() override = default;
auto pollTimeout() const { return poll_timeout; }
bool hasMorePolledRecords() const { return current != records.end(); }
bool poll();
ReadBufferPtr consume();
bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; }
auto getFileName() const { return current_file; }
auto getOffset() const { return current_offset; }
auto getFileName() const { return current[-1].file_name; }
auto getOffset() const { return current[-1].offset; }
const String & getCurrentRecord() const { return current[-1].data; }
private:
enum class BufferStatus
@ -57,8 +56,6 @@ private:
size_t stream_number;
size_t max_streams_number;
bool allowed = true;
using RecordData = std::string;
struct Record
{
@ -72,15 +69,12 @@ private:
Records records;
Records::const_iterator current;
String current_file;
UInt64 current_offset = 0;
using TaskThread = BackgroundSchedulePool::TaskHolder;
Records pollBatch(size_t batch_size_);
void readNewRecords(Records & new_records, size_t batch_size_);
bool nextImpl() override;
ReadBufferPtr getNextRecord();
};
}

View File

@ -17,7 +17,8 @@ class ASTStorage;
M(UInt64, max_threads, 0, "Number of max threads to parse files, default is 0, which means the number will be max(1, physical_cpu_cores / 4)", 0) \
M(Milliseconds, poll_directory_watch_events_backoff_init, 500, "The initial sleep value for watch directory thread.", 0) \
M(Milliseconds, poll_directory_watch_events_backoff_max, 32000, "The max sleep value for watch directory thread.", 0) \
M(UInt64, poll_directory_watch_events_backoff_factor, 2, "The speed of backoff, exponential by default", 0)
M(UInt64, poll_directory_watch_events_backoff_factor, 2, "The speed of backoff, exponential by default", 0) \
M(StreamingHandleErrorMode, handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for FileLog engine. Possible values: default (throw an exception after nats_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \
#define LIST_OF_FILELOG_SETTINGS(M, ALIAS) \
FILELOG_RELATED_SETTINGS(M, ALIAS) \

View File

@ -1,8 +1,8 @@
#include <Formats/FormatFactory.h>
#include <Interpreters/Context.h>
#include <Processors/Executors/StreamingFormatExecutor.h>
#include <Storages/FileLog/FileLogConsumer.h>
#include <Storages/FileLog/FileLogSource.h>
#include <Storages/FileLog/ReadBufferFromFileLog.h>
#include <Common/Stopwatch.h>
#include <Common/logger_useful.h>
@ -18,7 +18,8 @@ FileLogSource::FileLogSource(
size_t max_block_size_,
size_t poll_time_out_,
size_t stream_number_,
size_t max_streams_number_)
size_t max_streams_number_,
StreamingHandleErrorMode handle_error_mode_)
: ISource(storage_snapshot_->getSampleBlockForColumns(columns))
, storage(storage_)
, storage_snapshot(storage_snapshot_)
@ -28,10 +29,11 @@ FileLogSource::FileLogSource(
, poll_time_out(poll_time_out_)
, stream_number(stream_number_)
, max_streams_number(max_streams_number_)
, handle_error_mode(handle_error_mode_)
, non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized())
, virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames()))
, virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtuals().getNames()))
{
buffer = std::make_unique<ReadBufferFromFileLog>(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_);
consumer = std::make_unique<FileLogConsumer>(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_);
const auto & file_infos = storage.getFileInfos();
@ -67,7 +69,7 @@ Chunk FileLogSource::generate()
/// Store metas of last written chunk into disk
storage.storeMetas(start, end);
if (!buffer || buffer->noRecords())
if (!consumer || consumer->noRecords())
{
/// There is no onFinish for ISource, we call it
/// when no records return to close files
@ -77,29 +79,72 @@ Chunk FileLogSource::generate()
MutableColumns virtual_columns = virtual_header.cloneEmptyColumns();
EmptyReadBuffer empty_buf;
auto input_format = FormatFactory::instance().getInput(
storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size, std::nullopt, 1);
StreamingFormatExecutor executor(non_virtual_header, input_format);
storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1);
std::optional<String> exception_message;
size_t total_rows = 0;
auto on_error = [&](const MutableColumns & result_columns, Exception & e)
{
if (handle_error_mode == StreamingHandleErrorMode::STREAM)
{
exception_message = e.message();
for (const auto & column : result_columns)
{
// We could already push some rows to result_columns
// before exception, we need to fix it.
auto cur_rows = column->size();
if (cur_rows > total_rows)
column->popBack(cur_rows - total_rows);
// All data columns will get default value in case of error.
column->insertDefault();
}
return 1;
}
else
{
throw std::move(e);
}
};
StreamingFormatExecutor executor(non_virtual_header, input_format, on_error);
size_t failed_poll_attempts = 0;
Stopwatch watch;
while (true)
{
exception_message.reset();
size_t new_rows = 0;
if (buffer->poll())
new_rows = executor.execute();
if (auto buf = consumer->consume())
new_rows = executor.execute(*buf);
if (new_rows)
{
auto file_name = buffer->getFileName();
auto offset = buffer->getOffset();
auto file_name = consumer->getFileName();
auto offset = consumer->getOffset();
for (size_t i = 0; i < new_rows; ++i)
{
virtual_columns[0]->insert(file_name);
virtual_columns[1]->insert(offset);
if (handle_error_mode == StreamingHandleErrorMode::STREAM)
{
if (exception_message)
{
const auto & current_record = consumer->getCurrentRecord();
virtual_columns[2]->insertData(current_record.data(), current_record.size());
virtual_columns[3]->insertData(exception_message->data(), exception_message->size());
}
else
{
virtual_columns[2]->insertDefault();
virtual_columns[3]->insertDefault();
}
}
}
total_rows = total_rows + new_rows;
}
@ -108,7 +153,7 @@ Chunk FileLogSource::generate()
++failed_poll_attempts;
}
if (!buffer->hasMorePolledRecords()
if (!consumer->hasMorePolledRecords()
&& ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out
|| failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS))
{

View File

@ -1,7 +1,7 @@
#pragma once
#include <Processors/ISource.h>
#include <Storages/FileLog/ReadBufferFromFileLog.h>
#include <Storages/FileLog/FileLogConsumer.h>
#include <Storages/FileLog/StorageFileLog.h>
namespace Poco
@ -21,11 +21,12 @@ public:
size_t max_block_size_,
size_t poll_time_out_,
size_t stream_number_,
size_t max_streams_number_);
size_t max_streams_number_,
StreamingHandleErrorMode handle_error_mode_);
String getName() const override { return "FileLog"; }
bool noRecords() { return !buffer || buffer->noRecords(); }
bool noRecords() { return !consumer || consumer->noRecords(); }
void onFinish();
@ -45,8 +46,9 @@ private:
size_t stream_number;
size_t max_streams_number;
StreamingHandleErrorMode handle_error_mode;
std::unique_ptr<ReadBufferFromFileLog> buffer;
std::unique_ptr<FileLogConsumer> consumer;
Block non_virtual_header;
Block virtual_header;

View File

@ -1,6 +1,7 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <Disks/StoragePolicy.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
@ -346,7 +347,8 @@ Pipe StorageFileLog::read(
getMaxBlockSize(),
getPollTimeoutMillisecond(),
stream_number,
max_streams_number));
max_streams_number,
filelog_settings->handle_error_mode));
}
return Pipe::unitePipes(std::move(pipes));
@ -708,7 +710,8 @@ bool StorageFileLog::streamToViews()
getPollMaxBatchSize(),
getPollTimeoutMillisecond(),
stream_number,
max_streams_number));
max_streams_number,
filelog_settings->handle_error_mode));
}
auto input= Pipe::unitePipes(std::move(pipes));
@ -978,13 +981,17 @@ bool StorageFileLog::updateFileInfos()
NamesAndTypesList StorageFileLog::getVirtuals() const
{
return NamesAndTypesList{
auto virtuals = NamesAndTypesList{
{"_filename", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{"_offset", std::make_shared<DataTypeUInt64>()}};
if (filelog_settings->handle_error_mode == StreamingHandleErrorMode::STREAM)
{
virtuals.push_back({"_raw_record", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())});
virtuals.push_back({"_error", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())});
}
return virtuals;
}
Names StorageFileLog::getVirtualColumnNames()
{
return {"_filename", "_offset"};
}
}

View File

@ -103,8 +103,6 @@ public:
NamesAndTypesList getVirtuals() const override;
static Names getVirtualColumnNames();
static UInt64 getInode(const String & file_name);
void openFilesAndSetPos();

View File

@ -104,7 +104,7 @@ public:
auto currentPartition() const { return current[-1].get_partition(); }
auto currentTimestamp() const { return current[-1].get_timestamp(); }
const auto & currentHeaderList() const { return current[-1].get_header_list(); }
String currentPayload() const { return current[-1].get_payload(); }
const cppkafka::Buffer & currentPayload() const { return current[-1].get_payload(); }
void setExceptionInfo(const cppkafka::Error & err, bool with_stacktrace = true);
void setExceptionInfo(const std::string & text, bool with_stacktrace = true);
void setRDKafkaStat(const std::string & stat_json_string)

View File

@ -15,7 +15,6 @@ class ASTStorage;
M(String, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \
/* those are mapped to format factory settings */ \
M(String, kafka_format, "", "The message format for Kafka engine.", 0) \
M(Char, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \
M(String, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \
M(UInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \
/* default is = max_insert_block_size / kafka_num_consumers */ \
@ -29,17 +28,21 @@ class ASTStorage;
/* default is stream_flush_interval_ms */ \
M(Milliseconds, kafka_flush_interval_ms, 0, "Timeout for flushing data from Kafka.", 0) \
M(Bool, kafka_thread_per_consumer, false, "Provide independent thread for each consumer", 0) \
M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default, stream.", 0) \
M(StreamingHandleErrorMode, kafka_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \
M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \
M(UInt64, kafka_max_rows_per_message, 1, "The maximum number of rows produced in one kafka message for row-based formats.", 0) \
#define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \
MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \
/** TODO: */
/* https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md */
/* https://github.com/edenhill/librdkafka/blob/v1.4.2/src/rdkafka_conf.c */
#define LIST_OF_KAFKA_SETTINGS(M, ALIAS) \
KAFKA_RELATED_SETTINGS(M, ALIAS) \
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS)
#define LIST_OF_KAFKA_SETTINGS(M, ALIAS) \
KAFKA_RELATED_SETTINGS(M, ALIAS) \
OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS) \
DECLARE_SETTINGS_TRAITS(KafkaSettingsTraits, LIST_OF_KAFKA_SETTINGS)

View File

@ -46,7 +46,7 @@ KafkaSource::KafkaSource(
, commit_in_suffix(commit_in_suffix_)
, non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized())
, virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames()))
, handle_error_mode(storage.getHandleKafkaErrorMode())
, handle_error_mode(storage.getStreamingHandleErrorMode())
{
}
@ -98,7 +98,7 @@ Chunk KafkaSource::generateImpl()
// otherwise external iteration will reuse that and logic will became even more fuzzy
MutableColumns virtual_columns = virtual_header.cloneEmptyColumns();
auto put_error_to_stream = handle_error_mode == HandleKafkaErrorMode::STREAM;
auto put_error_to_stream = handle_error_mode == StreamingHandleErrorMode::STREAM;
EmptyReadBuffer empty_buf;
auto input_format = FormatFactory::instance().getInput(
@ -207,9 +207,9 @@ Chunk KafkaSource::generateImpl()
{
if (exception_message)
{
auto payload = consumer->currentPayload();
virtual_columns[8]->insert(payload);
virtual_columns[9]->insert(*exception_message);
const auto & payload = consumer->currentPayload();
virtual_columns[8]->insertData(reinterpret_cast<const char *>(payload.get_data()), payload.get_size());
virtual_columns[9]->insertData(exception_message->data(), exception_message->size());
}
else
{

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