mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Merge remote-tracking branch 'blessed/master' into r52159
This commit is contained in:
commit
728c20f36d
@ -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
2
contrib/libhdfs3
vendored
@ -1 +1 @@
|
||||
Subproject commit 377220ef351ae24994a5fcd2b5fa3930d00c4db0
|
||||
Subproject commit bdcb91354b1c05b21e73043a112a6f1e3b013497
|
@ -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")
|
||||
|
@ -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" \
|
||||
|
@ -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 project’s 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 project’s 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}
|
||||
|
||||
|
@ -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, [Cap’n 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.
|
||||
|
@ -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, [Cap’n 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.
|
||||
|
@ -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, [Cap’n 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.
|
||||
|
@ -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.
|
||||
|
||||
|
105
docs/en/engines/table-engines/special/filelog.md
Normal file
105
docs/en/engines/table-engines/special/filelog.md
Normal 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.
|
@ -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.
|
||||
|
@ -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.
|
||||
|
@ -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.
|
||||
@ -4780,6 +4790,10 @@ a Tuple(
|
||||
)
|
||||
```
|
||||
|
||||
## analyze_index_with_space_filling_curves
|
||||
|
||||
If a table has a space-filling curve in its index, e.g. `ORDER BY mortonEncode(x, y)`, and the query has conditions on its arguments, e.g. `x >= 10 AND x <= 20 AND y >= 20 AND y <= 30`, use the space-filling curve for index analysis.
|
||||
|
||||
## dictionary_use_async_executor {#dictionary_use_async_executor}
|
||||
|
||||
Execute a pipeline for reading dictionary source in several threads. It's supported only by dictionaries with local CLICKHOUSE source.
|
||||
|
@ -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)
|
||||
|
@ -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.
|
||||
|
@ -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**
|
||||
|
@ -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
|
||||
|
||||
|
@ -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
|
||||
|
||||
|
@ -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 и больше не используется.
|
||||
|
||||
|
@ -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, you’ll learn how to set up a simple ClickHouse cluster. It’ll 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, we’ll 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, it’s 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 won’t 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 it’s time to fill our ClickHouse server with some sample data. In this tutorial, we’ll 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, we’ll 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”. There’s a `default` database, but we’ll 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`, содержащую посещения — преднастроенные сессии вместо каждого действия.
|
||||
|
||||
Let’s 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 here’s 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, it’s 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 cluster’s 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, let’s 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 there’s 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).
|
||||
|
||||
Let’s 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. There’s 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. It’s 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 won’t 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.
|
||||
Репликация работает в режиме мультимастера. Это означает, что данные могут быть загружены на любую из реплик и система автоматически синхронизирует данные между остальными репликами. Репликация асинхронна, то есть в конкретный момент времнени не все реплики могут содержать недавно добавленные данные. Как минимум одна реплика должна быть в строю для приёма данных. Прочие реплики синхронизируются и восстановят согласованное состояния как только снова станут активными. Заметим, что при таком подходе есть вероятность утраты недавно добавленных данных.
|
||||
|
@ -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}
|
||||
|
||||
Максимальное количество строк до сортировки. Позволяет ограничить потребление оперативки при сортировке.
|
||||
|
@ -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`).
|
||||
|
||||
Возможные значения:
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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}
|
||||
|
||||
|
@ -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»子目录,并不再使用。
|
||||
|
||||
|
@ -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}
|
||||
|
||||
启用/禁用批量发送插入的数据。
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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));
|
||||
|
@ -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") \
|
||||
|
217
src/Common/MortonUtils.h
Normal file
217
src/Common/MortonUtils.h
Normal file
@ -0,0 +1,217 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/BitHelpers.h>
|
||||
#include <base/defines.h>
|
||||
#include <array>
|
||||
#include <set>
|
||||
|
||||
|
||||
/** Functions to analyze the Morton space-filling curve on ranges.
|
||||
* There are two operations:
|
||||
*
|
||||
* 1. Inverting the Morton curve on a range.
|
||||
* Given a range of values of Morton curve,
|
||||
* mortonEncode(x, y) in [a, b]
|
||||
* get possible set of values of its arguments.
|
||||
* This set is represented by a set of hyperrectangles in (x, y) space.
|
||||
*
|
||||
* 2. Calculating the Morton curve on a hyperrectangle.
|
||||
* Given a hyperrectangle in (x, y) space
|
||||
* (x, y) in [x_min, x_max] × [y_min, y_max]
|
||||
* get possible intervals of the mortonEncode(x, y).
|
||||
*
|
||||
* These operations could be used for index analysis.
|
||||
*
|
||||
* Note: currently it is only tested in 2d.
|
||||
*/
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
/// After the most significant bit 1, set all subsequent less significant bits to 1 as well.
|
||||
inline UInt64 toMask(UInt64 n)
|
||||
{
|
||||
n |= n >> 1;
|
||||
n |= n >> 2;
|
||||
n |= n >> 4;
|
||||
n |= n >> 8;
|
||||
n |= n >> 16;
|
||||
n |= n >> 32;
|
||||
return n;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Splits the interval [first, last] to a set of intervals [first_i, last_i],
|
||||
* each of them determined by a bit prefix: [xxxxxx0000, xxxxxx1111],
|
||||
*
|
||||
* For example, the interval [6, 13] = {5, 7, 8, 9, 10, 11, 12, 13}
|
||||
* will be represented by the set of intervals:
|
||||
* - [6, 7] 0000011*
|
||||
* - [8, 11] 000010**
|
||||
* - [12, 13] 0000110*
|
||||
*
|
||||
* It means that if you have a binary space partition by powers of two,
|
||||
* every of the resulting intervals will fully occupy one of the levels of this partition.
|
||||
*/
|
||||
template <typename F>
|
||||
void intervalBinaryPartition(UInt64 first, UInt64 last, F && callback)
|
||||
{
|
||||
/// first = 6: 00000110
|
||||
/// last = 13: 00001101
|
||||
/// first ^ last: 00001011
|
||||
/// mask: 00000111
|
||||
/// split = 7: 00000111
|
||||
|
||||
/// first = 8: 00001000
|
||||
/// last = 13: 00001101
|
||||
/// first ^ last: 00000101
|
||||
/// mask: 00000011
|
||||
/// split = 11: 00001011
|
||||
|
||||
/// first = 8: 00001000
|
||||
/// last = 11: 00001011
|
||||
/// first ^ last: 00000011
|
||||
/// mask: 00000001
|
||||
/// split = 9: 00001001
|
||||
|
||||
/// Another example:
|
||||
|
||||
/// first = 15: 00001111
|
||||
/// last = 31: 00011111
|
||||
/// first ^ last: 00010000
|
||||
/// mask: 00001111
|
||||
/// split = 15: 00001111
|
||||
|
||||
UInt64 diff = first ^ last;
|
||||
UInt64 mask = toMask(diff) >> 1;
|
||||
|
||||
/// The current interval represents a whole range with fixed prefix.
|
||||
if ((first & mask) == 0 && (last & mask) == mask)
|
||||
{
|
||||
chassert(((last - first + 1) & (last - first)) == 0); /// The interval length is one less than a power of two.
|
||||
callback(first, last);
|
||||
return;
|
||||
}
|
||||
|
||||
UInt64 split = first | mask;
|
||||
|
||||
chassert(split >= first);
|
||||
chassert(split <= last);
|
||||
|
||||
intervalBinaryPartition(first, split, std::forward<F>(callback));
|
||||
if (split < last)
|
||||
intervalBinaryPartition(split + 1, last, std::forward<F>(callback));
|
||||
}
|
||||
|
||||
|
||||
/** Multidimensional version of binary space partitioning.
|
||||
* It takes a hyperrectangle - a direct product of intervals (in each dimension),
|
||||
* and splits it into smaller hyperrectangles - a direct product of partitions across each dimension.
|
||||
*/
|
||||
template <size_t N, size_t start_idx, typename F>
|
||||
void hyperrectangleBinaryPartitionImpl(
|
||||
std::array<std::pair<UInt64, UInt64>, N> hyperrectangle,
|
||||
F && callback)
|
||||
{
|
||||
intervalBinaryPartition(hyperrectangle[start_idx].first, hyperrectangle[start_idx].second,
|
||||
[&](UInt64 a, UInt64 b) mutable
|
||||
{
|
||||
auto new_hyperrectangle = hyperrectangle;
|
||||
new_hyperrectangle[start_idx].first = a;
|
||||
new_hyperrectangle[start_idx].second = b;
|
||||
|
||||
if constexpr (start_idx + 1 < N)
|
||||
hyperrectangleBinaryPartitionImpl<N, start_idx + 1>(new_hyperrectangle, std::forward<F>(callback));
|
||||
else
|
||||
callback(new_hyperrectangle);
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
template <size_t N, typename F>
|
||||
void hyperrectangleBinaryPartition(
|
||||
std::array<std::pair<UInt64, UInt64>, N> hyperrectangle,
|
||||
F && callback)
|
||||
{
|
||||
hyperrectangleBinaryPartitionImpl<N, 0>(hyperrectangle, std::forward<F>(callback));
|
||||
}
|
||||
|
||||
|
||||
/** Unpack an interval of Morton curve to hyperrectangles covered by it across N dimensions.
|
||||
*/
|
||||
template <size_t N, typename F>
|
||||
void mortonIntervalToHyperrectangles(UInt64 first, UInt64 last, F && callback)
|
||||
{
|
||||
intervalBinaryPartition(first, last, [&](UInt64 a, UInt64 b)
|
||||
{
|
||||
std::array<std::pair<UInt64, UInt64>, N> unpacked{};
|
||||
|
||||
for (size_t bit_idx = 0; bit_idx < 64; ++bit_idx)
|
||||
{
|
||||
size_t source_bit = 63 - bit_idx;
|
||||
size_t result_bit = (63 - bit_idx) / N;
|
||||
|
||||
unpacked[source_bit % N].first |= ((a >> source_bit) & 1) << result_bit;
|
||||
unpacked[source_bit % N].second |= ((b >> source_bit) & 1) << result_bit;
|
||||
}
|
||||
|
||||
callback(unpacked);
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
/** Given a hyperrectangle, find intervals of Morton curve that cover this hyperrectangle.
|
||||
* Note: to avoid returning too many intervals, the intervals can be returned larger than exactly needed
|
||||
* (covering some other points, not belonging to the hyperrectangle).
|
||||
* We do it by extending hyperrectangles to hypercubes.
|
||||
*/
|
||||
template <size_t N, typename F>
|
||||
void hyperrectangleToPossibleMortonIntervals(
|
||||
std::array<std::pair<UInt64, UInt64>, N> hyperrectangle,
|
||||
F && callback)
|
||||
{
|
||||
/// Due to extension to cubes, there could be duplicates. Filter them.
|
||||
std::set<std::pair<UInt64, UInt64>> found_intervals;
|
||||
|
||||
hyperrectangleBinaryPartition<N>(hyperrectangle, [&](auto part)
|
||||
{
|
||||
size_t suffix_size = 0;
|
||||
for (size_t i = 0; i < N; ++i)
|
||||
if (part[i].second != part[i].first)
|
||||
suffix_size = std::max<size_t>(suffix_size,
|
||||
1 + bitScanReverse(part[i].second ^ part[i].first));
|
||||
|
||||
UInt64 first = 0;
|
||||
UInt64 last = 0;
|
||||
|
||||
size_t source_bit_idx = 0;
|
||||
size_t result_bit_idx = 0;
|
||||
|
||||
while (result_bit_idx < 64)
|
||||
{
|
||||
for (size_t i = 0; i < N; ++i)
|
||||
{
|
||||
if (source_bit_idx < suffix_size)
|
||||
{
|
||||
last |= (1 << result_bit_idx);
|
||||
}
|
||||
else
|
||||
{
|
||||
UInt64 bit = (((part[i].first >> source_bit_idx) & 1) << result_bit_idx);
|
||||
first |= bit;
|
||||
last |= bit;
|
||||
}
|
||||
|
||||
++result_bit_idx;
|
||||
if (!(result_bit_idx < 64))
|
||||
break;
|
||||
}
|
||||
++source_bit_idx;
|
||||
}
|
||||
|
||||
if (found_intervals.insert({first, last}).second)
|
||||
callback(first, last);
|
||||
});
|
||||
}
|
@ -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") \
|
||||
|
131
src/Common/tests/gtest_morton_utils.cpp
Normal file
131
src/Common/tests/gtest_morton_utils.cpp
Normal file
@ -0,0 +1,131 @@
|
||||
#include <gtest/gtest.h>
|
||||
#include <iostream>
|
||||
#include <Common/MortonUtils.h>
|
||||
|
||||
|
||||
GTEST_TEST(MortonUtils, Intervals)
|
||||
{
|
||||
{
|
||||
std::stringstream res;
|
||||
intervalBinaryPartition(6, 13, [&](UInt64 first, UInt64 last)
|
||||
{
|
||||
res << first << ", " << last << "; ";
|
||||
});
|
||||
ASSERT_EQ(res.str(), "6, 7; 8, 11; 12, 13; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::stringstream res;
|
||||
intervalBinaryPartition(15, 31, [&](UInt64 first, UInt64 last)
|
||||
{
|
||||
res << first << ", " << last << "; ";
|
||||
});
|
||||
ASSERT_EQ(res.str(), "15, 15; 16, 31; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::stringstream res;
|
||||
intervalBinaryPartition(15, 16, [&](UInt64 first, UInt64 last)
|
||||
{
|
||||
res << first << ", " << last << "; ";
|
||||
});
|
||||
ASSERT_EQ(res.str(), "15, 15; 16, 16; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::stringstream res;
|
||||
intervalBinaryPartition(191, 769, [&](UInt64 first, UInt64 last)
|
||||
{
|
||||
res << first << ", " << last << "; ";
|
||||
});
|
||||
ASSERT_EQ(res.str(), "191, 191; 192, 255; 256, 511; 512, 767; 768, 769; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::array<std::pair<UInt64, UInt64>, 2> input = {std::pair{6, 13}, std::pair{15, 31}};
|
||||
|
||||
std::stringstream res;
|
||||
hyperrectangleBinaryPartition<2>(input, [&](auto hyperrectangle)
|
||||
{
|
||||
res << "[" << hyperrectangle[0].first << ", " << hyperrectangle[0].second
|
||||
<< "] x [" << hyperrectangle[1].first << ", " << hyperrectangle[1].second
|
||||
<< "]; ";
|
||||
});
|
||||
|
||||
ASSERT_EQ(res.str(), "[6, 7] x [15, 15]; [6, 7] x [16, 31]; [8, 11] x [15, 15]; [8, 11] x [16, 31]; [12, 13] x [15, 15]; [12, 13] x [16, 31]; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::array<std::pair<UInt64, UInt64>, 2> input = {std::pair{23, 24}, std::pair{15, 16}};
|
||||
|
||||
std::stringstream res;
|
||||
hyperrectangleBinaryPartition<2>(input, [&](auto hyperrectangle)
|
||||
{
|
||||
res << "[" << hyperrectangle[0].first << ", " << hyperrectangle[0].second
|
||||
<< "] x [" << hyperrectangle[1].first << ", " << hyperrectangle[1].second
|
||||
<< "]; ";
|
||||
});
|
||||
|
||||
ASSERT_EQ(res.str(), "[23, 23] x [15, 15]; [23, 23] x [16, 16]; [24, 24] x [15, 15]; [24, 24] x [16, 16]; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::stringstream res;
|
||||
mortonIntervalToHyperrectangles<2>(191, 769, [&](auto hyperrectangle)
|
||||
{
|
||||
res << "[" << hyperrectangle[0].first << ", " << hyperrectangle[0].second
|
||||
<< "] x [" << hyperrectangle[1].first << ", " << hyperrectangle[1].second
|
||||
<< "]; ";
|
||||
});
|
||||
|
||||
ASSERT_EQ(res.str(), "[7, 7] x [15, 15]; [8, 15] x [8, 15]; [16, 31] x [0, 15]; [0, 15] x [16, 31]; [16, 17] x [16, 16]; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::stringstream res;
|
||||
mortonIntervalToHyperrectangles<2>(500, 600, [&](auto hyperrectangle)
|
||||
{
|
||||
res << "[" << hyperrectangle[0].first << ", " << hyperrectangle[0].second
|
||||
<< "] x [" << hyperrectangle[1].first << ", " << hyperrectangle[1].second
|
||||
<< "]; ";
|
||||
});
|
||||
|
||||
ASSERT_EQ(res.str(), "[30, 31] x [12, 13]; [28, 31] x [14, 15]; [0, 7] x [16, 23]; [8, 11] x [16, 19]; [12, 15] x [16, 17]; [12, 12] x [18, 18]; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::array<std::pair<UInt64, UInt64>, 2> input = {std::pair{23, 24}, std::pair{15, 16}};
|
||||
|
||||
std::stringstream res;
|
||||
hyperrectangleToPossibleMortonIntervals<2>(input, [&](UInt64 first, UInt64 last)
|
||||
{
|
||||
res << first << ", " << last << "; ";
|
||||
});
|
||||
|
||||
ASSERT_EQ(res.str(), "447, 447; 789, 789; 490, 490; 832, 832; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::array<std::pair<UInt64, UInt64>, 2> input = {std::pair{6, 7}, std::pair{16, 31}};
|
||||
|
||||
std::stringstream res;
|
||||
hyperrectangleToPossibleMortonIntervals<2>(input, [&](UInt64 first, UInt64 last)
|
||||
{
|
||||
res << first << ", " << last << "; ";
|
||||
});
|
||||
|
||||
ASSERT_EQ(res.str(), "512, 767; ");
|
||||
}
|
||||
|
||||
{
|
||||
std::array<std::pair<UInt64, UInt64>, 2> input = {std::pair{6, 13}, std::pair{15, 31}};
|
||||
|
||||
std::stringstream res;
|
||||
hyperrectangleToPossibleMortonIntervals<2>(input, [&](UInt64 first, UInt64 last)
|
||||
{
|
||||
res << first << ", " << last << "; ";
|
||||
});
|
||||
|
||||
ASSERT_EQ(res.str(), "188, 191; 512, 767; 224, 239; 248, 251; ");
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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};
|
||||
|
||||
|
@ -7,7 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
Range::Range(const FieldRef & point) /// NOLINT
|
||||
: left(point), right(point), left_included(true), right_included(true) {}
|
||||
|
||||
@ -160,6 +159,52 @@ void Range::invert()
|
||||
std::swap(left_included, right_included);
|
||||
}
|
||||
|
||||
Range intersect(const Range & a, const Range & b)
|
||||
{
|
||||
Range res = Range::createWholeUniverse();
|
||||
|
||||
if (Range::less(a.left, b.left))
|
||||
{
|
||||
res.left = b.left;
|
||||
res.left_included = b.left_included;
|
||||
}
|
||||
else if (Range::equals(a.left, b.left))
|
||||
{
|
||||
res.left = a.left;
|
||||
res.left_included = a.left_included && b.left_included;
|
||||
}
|
||||
else
|
||||
{
|
||||
res.left = a.left;
|
||||
res.left_included = a.left_included;
|
||||
}
|
||||
|
||||
if (Range::less(a.right, b.right))
|
||||
{
|
||||
res.right = a.right;
|
||||
res.right_included = a.right_included;
|
||||
}
|
||||
else if (Range::equals(a.right, b.right))
|
||||
{
|
||||
res.right = a.right;
|
||||
res.right_included = a.right_included && b.right_included;
|
||||
}
|
||||
else
|
||||
{
|
||||
res.right = b.right;
|
||||
res.right_included = b.right_included;
|
||||
}
|
||||
|
||||
if (res.empty())
|
||||
{
|
||||
res.right = res.left;
|
||||
res.right_included = false;
|
||||
res.left_included = false;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
String Range::toString() const
|
||||
{
|
||||
WriteBufferFromOwnString str;
|
||||
@ -170,4 +215,33 @@ String Range::toString() const
|
||||
return str.str();
|
||||
}
|
||||
|
||||
Hyperrectangle intersect(const Hyperrectangle & a, const Hyperrectangle & b)
|
||||
{
|
||||
size_t result_size = std::min(a.size(), b.size());
|
||||
|
||||
Hyperrectangle res;
|
||||
res.reserve(result_size);
|
||||
|
||||
for (size_t i = 0; i < result_size; ++i)
|
||||
res.push_back(intersect(a[i], b[i]));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
String toString(const Hyperrectangle & x)
|
||||
{
|
||||
WriteBufferFromOwnString str;
|
||||
|
||||
bool first = true;
|
||||
for (const auto & range : x)
|
||||
{
|
||||
if (!first)
|
||||
str << " × ";
|
||||
str << range.toString();
|
||||
first = false;
|
||||
}
|
||||
|
||||
return str.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -59,8 +59,8 @@ public:
|
||||
static Range createRightBounded(const FieldRef & right_point, bool right_included, bool with_null = false);
|
||||
static Range createLeftBounded(const FieldRef & left_point, bool left_included, bool with_null = false);
|
||||
|
||||
static ALWAYS_INLINE bool equals(const Field & lhs, const Field & rhs);
|
||||
static ALWAYS_INLINE bool less(const Field & lhs, const Field & rhs);
|
||||
static bool equals(const Field & lhs, const Field & rhs);
|
||||
static bool less(const Field & lhs, const Field & rhs);
|
||||
|
||||
/** Optimize the range. If it has an open boundary and the Field type is "loose"
|
||||
* - then convert it to closed, narrowing by one.
|
||||
@ -88,8 +88,13 @@ public:
|
||||
String toString() const;
|
||||
};
|
||||
|
||||
Range intersect(const Range & a, const Range & b);
|
||||
|
||||
/** Hyperrectangle is a product of ranges: each range across each coordinate.
|
||||
*/
|
||||
using Hyperrectangle = std::vector<Range>;
|
||||
|
||||
Hyperrectangle intersect(const Hyperrectangle & a, const Hyperrectangle & b);
|
||||
String toString(const Hyperrectangle & x);
|
||||
|
||||
}
|
||||
|
@ -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) \
|
||||
@ -335,6 +335,7 @@ class IColumn;
|
||||
M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \
|
||||
M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \
|
||||
M(UInt64, use_index_for_in_with_subqueries_max_values, 0, "The maximum size of set in the right hand side of the IN operator to use table index for filtering. It allows to avoid performance degradation and higher memory usage due to preparation of additional data structures for large queries. Zero means no limit.", 0) \
|
||||
M(Bool, analyze_index_with_space_filling_curves, true, "If a table has a space-filling curve in its index, e.g. `ORDER BY mortonEncode(x, y)`, and the query has conditions on its arguments, e.g. `x >= 10 AND x <= 20 AND y >= 20 AND y <= 30`, use the space-filling curve for index analysis.", 0) \
|
||||
M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \
|
||||
M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \
|
||||
M(Bool, empty_result_for_aggregation_by_constant_keys_on_empty_set, true, "Return empty result when aggregating by constant keys on empty set.", 0) \
|
||||
@ -601,6 +602,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 +837,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) \
|
||||
|
@ -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},
|
||||
|
@ -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
|
||||
{
|
||||
|
123
src/Formats/NumpyDataTypes.h
Normal file
123
src/Formats/NumpyDataTypes.h
Normal 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;
|
||||
};
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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",
|
||||
|
@ -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>>();
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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()));
|
||||
|
@ -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>
|
||||
|
@ -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;
|
||||
|
@ -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(); }
|
||||
|
@ -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 (...)
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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())
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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
|
||||
|
@ -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.
|
||||
|
@ -431,7 +431,8 @@ static void buildORCSearchArgumentImpl(
|
||||
case KeyCondition::RPNElement::FUNCTION_IN_SET:
|
||||
case KeyCondition::RPNElement::FUNCTION_NOT_IN_SET:
|
||||
case KeyCondition::RPNElement::FUNCTION_IS_NULL:
|
||||
case KeyCondition::RPNElement::FUNCTION_IS_NOT_NULL: {
|
||||
case KeyCondition::RPNElement::FUNCTION_IS_NOT_NULL:
|
||||
{
|
||||
const bool need_wrap_not = curr.function == KeyCondition::RPNElement::FUNCTION_IS_NOT_NULL
|
||||
|| curr.function == KeyCondition::RPNElement::FUNCTION_NOT_IN_RANGE
|
||||
|| curr.function == KeyCondition::RPNElement::FUNCTION_NOT_IN_SET;
|
||||
@ -625,19 +626,24 @@ static void buildORCSearchArgumentImpl(
|
||||
|
||||
break;
|
||||
}
|
||||
case KeyCondition::RPNElement::FUNCTION_UNKNOWN: {
|
||||
/// There is no optimization with space-filling curves for ORC.
|
||||
case KeyCondition::RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE:
|
||||
case KeyCondition::RPNElement::FUNCTION_UNKNOWN:
|
||||
{
|
||||
builder.literal(orc::TruthValue::YES_NO_NULL);
|
||||
rpn_stack.pop_back();
|
||||
break;
|
||||
}
|
||||
case KeyCondition::RPNElement::FUNCTION_NOT: {
|
||||
case KeyCondition::RPNElement::FUNCTION_NOT:
|
||||
{
|
||||
builder.startNot();
|
||||
rpn_stack.pop_back();
|
||||
buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, builder, format_settings);
|
||||
builder.end();
|
||||
break;
|
||||
}
|
||||
case KeyCondition::RPNElement::FUNCTION_AND: {
|
||||
case KeyCondition::RPNElement::FUNCTION_AND:
|
||||
{
|
||||
builder.startAnd();
|
||||
rpn_stack.pop_back();
|
||||
buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, builder, format_settings);
|
||||
@ -645,7 +651,8 @@ static void buildORCSearchArgumentImpl(
|
||||
builder.end();
|
||||
break;
|
||||
}
|
||||
case KeyCondition::RPNElement::FUNCTION_OR: {
|
||||
case KeyCondition::RPNElement::FUNCTION_OR:
|
||||
{
|
||||
builder.startOr();
|
||||
rpn_stack.pop_back();
|
||||
buildORCSearchArgumentImpl(key_condition, header, schema, rpn_stack, builder, format_settings);
|
||||
@ -653,12 +660,14 @@ static void buildORCSearchArgumentImpl(
|
||||
builder.end();
|
||||
break;
|
||||
}
|
||||
case KeyCondition::RPNElement::ALWAYS_FALSE: {
|
||||
case KeyCondition::RPNElement::ALWAYS_FALSE:
|
||||
{
|
||||
builder.literal(orc::TruthValue::NO);
|
||||
rpn_stack.pop_back();
|
||||
break;
|
||||
}
|
||||
case KeyCondition::RPNElement::ALWAYS_TRUE: {
|
||||
case KeyCondition::RPNElement::ALWAYS_TRUE:
|
||||
{
|
||||
builder.literal(orc::TruthValue::YES);
|
||||
rpn_stack.pop_back();
|
||||
break;
|
||||
|
426
src/Processors/Formats/Impl/NpyRowInputFormat.cpp
Normal file
426
src/Processors/Formats/Impl/NpyRowInputFormat.cpp
Normal 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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
65
src/Processors/Formats/Impl/NpyRowInputFormat.h
Normal file
65
src/Processors/Formats/Impl/NpyRowInputFormat.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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)
|
||||
|
@ -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.
|
||||
|
@ -4,7 +4,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ReadBufferFromFileLog;
|
||||
class FileLogConsumer;
|
||||
|
||||
using ReadBufferFromFileLogPtr = std::shared_ptr<ReadBufferFromFileLog>;
|
||||
using ReadBufferFromFileLogPtr = std::shared_ptr<FileLogConsumer>;
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
@ -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();
|
||||
};
|
||||
}
|
@ -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) \
|
||||
|
@ -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))
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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"};
|
||||
}
|
||||
}
|
||||
|
@ -103,8 +103,6 @@ public:
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
static Names getVirtualColumnNames();
|
||||
|
||||
static UInt64 getInode(const String & file_name);
|
||||
|
||||
void openFilesAndSetPos();
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -51,7 +51,7 @@ private:
|
||||
|
||||
const Block non_virtual_header;
|
||||
const Block virtual_header;
|
||||
const HandleKafkaErrorMode handle_error_mode;
|
||||
const StreamingHandleErrorMode handle_error_mode;
|
||||
|
||||
Poco::Timespan max_execution_time = 0;
|
||||
Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE};
|
||||
|
@ -268,7 +268,7 @@ StorageKafka::StorageKafka(
|
||||
, thread_per_consumer(kafka_settings->kafka_thread_per_consumer.value)
|
||||
, collection_name(collection_name_)
|
||||
{
|
||||
if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM)
|
||||
if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM)
|
||||
{
|
||||
kafka_settings->input_format_allow_errors_num = 0;
|
||||
kafka_settings->input_format_allow_errors_ratio = 0;
|
||||
@ -513,8 +513,9 @@ KafkaConsumerPtr StorageKafka::createConsumer(size_t consumer_number)
|
||||
// that allows to prevent fast draining of the librdkafka queue
|
||||
// during building of single insert block. Improves performance
|
||||
// significantly, but may lead to bigger memory consumption.
|
||||
size_t default_queued_min_messages = 100000; // we don't want to decrease the default
|
||||
conf.set("queued.min.messages", std::max(getMaxBlockSize(),default_queued_min_messages));
|
||||
size_t default_queued_min_messages = 100000; // must be greater than or equal to default
|
||||
size_t max_allowed_queued_min_messages = 10000000; // must be less than or equal to max allowed value
|
||||
conf.set("queued.min.messages", std::min(std::max(getMaxBlockSize(), default_queued_min_messages), max_allowed_queued_min_messages));
|
||||
|
||||
/// a reference to the consumer is needed in statistic callback
|
||||
/// although the consumer does not exist when callback is being registered
|
||||
@ -1056,7 +1057,7 @@ NamesAndTypesList StorageKafka::getVirtuals() const
|
||||
{"_timestamp_ms", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeDateTime64>(3))},
|
||||
{"_headers.name", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"_headers.value", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}};
|
||||
if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM)
|
||||
if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM)
|
||||
{
|
||||
result.push_back({"_raw_message", std::make_shared<DataTypeString>()});
|
||||
result.push_back({"_error", std::make_shared<DataTypeString>()});
|
||||
@ -1076,7 +1077,7 @@ Names StorageKafka::getVirtualColumnNames() const
|
||||
"_headers.name",
|
||||
"_headers.value",
|
||||
};
|
||||
if (kafka_settings->kafka_handle_error_mode == HandleKafkaErrorMode::STREAM)
|
||||
if (kafka_settings->kafka_handle_error_mode == StreamingHandleErrorMode::STREAM)
|
||||
{
|
||||
result.push_back({"_raw_message"});
|
||||
result.push_back({"_error"});
|
||||
|
@ -78,7 +78,7 @@ public:
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
Names getVirtualColumnNames() const;
|
||||
HandleKafkaErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; }
|
||||
StreamingHandleErrorMode getStreamingHandleErrorMode() const { return kafka_settings->kafka_handle_error_mode; }
|
||||
|
||||
struct SafeConsumers
|
||||
{
|
||||
|
@ -667,6 +667,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
|
||||
{
|
||||
// There could be conditions that data part to be loaded is broken, but some of meta infos are already written
|
||||
// into meta data before exception, need to clean them all.
|
||||
LOG_ERROR(storage.log, "Part {} is broken and need manual correction", getDataPartStorage().getFullPath());
|
||||
metadata_manager->deleteAll(/*include_projection*/ true);
|
||||
metadata_manager->assertAllDeleted(/*include_projection*/ true);
|
||||
throw;
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Functions/CastOverloadResolver.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/MortonUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
@ -26,7 +27,6 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
|
||||
#include <base/defines.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <cassert>
|
||||
@ -453,26 +453,28 @@ const KeyCondition::AtomMap KeyCondition::atom_map
|
||||
|
||||
static const std::map<std::string, std::string> inverse_relations =
|
||||
{
|
||||
{"equals", "notEquals"},
|
||||
{"notEquals", "equals"},
|
||||
{"less", "greaterOrEquals"},
|
||||
{"greaterOrEquals", "less"},
|
||||
{"greater", "lessOrEquals"},
|
||||
{"lessOrEquals", "greater"},
|
||||
{"in", "notIn"},
|
||||
{"notIn", "in"},
|
||||
{"globalIn", "globalNotIn"},
|
||||
{"globalNotIn", "globalIn"},
|
||||
{"nullIn", "notNullIn"},
|
||||
{"notNullIn", "nullIn"},
|
||||
{"globalNullIn", "globalNotNullIn"},
|
||||
{"globalNullNotIn", "globalNullIn"},
|
||||
{"isNull", "isNotNull"},
|
||||
{"isNotNull", "isNull"},
|
||||
{"like", "notLike"},
|
||||
{"notLike", "like"},
|
||||
{"empty", "notEmpty"},
|
||||
{"notEmpty", "empty"},
|
||||
{"equals", "notEquals"},
|
||||
{"notEquals", "equals"},
|
||||
{"less", "greaterOrEquals"},
|
||||
{"greaterOrEquals", "less"},
|
||||
{"greater", "lessOrEquals"},
|
||||
{"lessOrEquals", "greater"},
|
||||
{"in", "notIn"},
|
||||
{"notIn", "in"},
|
||||
{"globalIn", "globalNotIn"},
|
||||
{"globalNotIn", "globalIn"},
|
||||
{"nullIn", "notNullIn"},
|
||||
{"notNullIn", "nullIn"},
|
||||
{"globalNullIn", "globalNotNullIn"},
|
||||
{"globalNullNotIn", "globalNullIn"},
|
||||
{"isNull", "isNotNull"},
|
||||
{"isNotNull", "isNull"},
|
||||
{"like", "notLike"},
|
||||
{"notLike", "like"},
|
||||
{"ilike", "notILike"},
|
||||
{"notILike", "ilike"},
|
||||
{"empty", "notEmpty"},
|
||||
{"notEmpty", "empty"},
|
||||
};
|
||||
|
||||
|
||||
@ -723,6 +725,40 @@ static NameSet getAllSubexpressionNames(const ExpressionActions & key_expr)
|
||||
return names;
|
||||
}
|
||||
|
||||
void KeyCondition::getAllSpaceFillingCurves()
|
||||
{
|
||||
/// So far the only supported function is mortonEncode (Morton curve).
|
||||
|
||||
for (const auto & action : key_expr->getActions())
|
||||
{
|
||||
if (action.node->type == ActionsDAG::ActionType::FUNCTION
|
||||
&& action.node->children.size() >= 2
|
||||
&& action.node->function_base->getName() == "mortonEncode")
|
||||
{
|
||||
SpaceFillingCurveDescription curve;
|
||||
curve.function_name = action.node->function_base->getName();
|
||||
curve.key_column_pos = key_columns.at(action.node->result_name);
|
||||
for (const auto & child : action.node->children)
|
||||
{
|
||||
/// All arguments should be regular input columns.
|
||||
if (child->type == ActionsDAG::ActionType::INPUT)
|
||||
{
|
||||
curve.arguments.push_back(child->result_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
curve.arguments.clear();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// So far we only support the case of two arguments.
|
||||
if (2 == curve.arguments.size())
|
||||
key_space_filling_curves.push_back(std::move(curve));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
KeyCondition::KeyCondition(
|
||||
const ASTPtr & query,
|
||||
const ASTs & additional_filter_asts,
|
||||
@ -751,6 +787,9 @@ KeyCondition::KeyCondition(
|
||||
++key_index;
|
||||
}
|
||||
|
||||
if (context->getSettingsRef().analyze_index_with_space_filling_curves)
|
||||
getAllSpaceFillingCurves();
|
||||
|
||||
ASTPtr filter_node;
|
||||
if (query)
|
||||
filter_node = buildFilterNode(query, additional_filter_asts);
|
||||
@ -776,7 +815,10 @@ KeyCondition::KeyCondition(
|
||||
std::move(block_with_constants),
|
||||
std::move(prepared_sets),
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
|
||||
|
||||
rpn = std::move(builder).extractRPN();
|
||||
|
||||
findHyperrectanglesForArgumentsOfSpaceFillingCurves();
|
||||
}
|
||||
|
||||
KeyCondition::KeyCondition(
|
||||
@ -825,6 +867,9 @@ KeyCondition::KeyCondition(
|
||||
++key_index;
|
||||
}
|
||||
|
||||
if (context->getSettingsRef().analyze_index_with_space_filling_curves)
|
||||
getAllSpaceFillingCurves();
|
||||
|
||||
if (!filter_dag)
|
||||
{
|
||||
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
@ -842,6 +887,8 @@ KeyCondition::KeyCondition(
|
||||
});
|
||||
|
||||
rpn = std::move(builder).extractRPN();
|
||||
|
||||
findHyperrectanglesForArgumentsOfSpaceFillingCurves();
|
||||
}
|
||||
|
||||
bool KeyCondition::addCondition(const String & column, const Range & range)
|
||||
@ -1204,7 +1251,9 @@ bool KeyCondition::tryPrepareSetIndex(
|
||||
MergeTreeSetIndex::KeyTuplePositionMapping index_mapping;
|
||||
index_mapping.tuple_index = tuple_index;
|
||||
DataTypePtr data_type;
|
||||
if (isKeyPossiblyWrappedByMonotonicFunctions(node, index_mapping.key_index, data_type, index_mapping.functions))
|
||||
std::optional<size_t> key_space_filling_curve_argument_pos;
|
||||
if (isKeyPossiblyWrappedByMonotonicFunctions(node, index_mapping.key_index, key_space_filling_curve_argument_pos, data_type, index_mapping.functions)
|
||||
&& !key_space_filling_curve_argument_pos) /// We don't support the analysis of space-filling curves and IN set.
|
||||
{
|
||||
indexes_mapping.push_back(index_mapping);
|
||||
data_types.push_back(data_type);
|
||||
@ -1412,13 +1461,15 @@ private:
|
||||
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
|
||||
const RPNBuilderTreeNode & node,
|
||||
size_t & out_key_column_num,
|
||||
std::optional<size_t> & out_argument_num_of_space_filling_curve,
|
||||
DataTypePtr & out_key_res_column_type,
|
||||
MonotonicFunctionsChain & out_functions_chain)
|
||||
{
|
||||
std::vector<RPNBuilderFunctionTreeNode> chain_not_tested_for_monotonicity;
|
||||
DataTypePtr key_column_type;
|
||||
|
||||
if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_key_column_num, key_column_type, chain_not_tested_for_monotonicity))
|
||||
if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
||||
node, out_key_column_num, out_argument_num_of_space_filling_curve, key_column_type, chain_not_tested_for_monotonicity))
|
||||
return false;
|
||||
|
||||
for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it)
|
||||
@ -1474,6 +1525,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
|
||||
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
||||
const RPNBuilderTreeNode & node,
|
||||
size_t & out_key_column_num,
|
||||
std::optional<size_t> & out_argument_num_of_space_filling_curve,
|
||||
DataTypePtr & out_key_column_type,
|
||||
std::vector<RPNBuilderFunctionTreeNode> & out_functions_chain)
|
||||
{
|
||||
@ -1492,10 +1544,31 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
||||
if (key_columns.end() != it)
|
||||
{
|
||||
out_key_column_num = it->second;
|
||||
out_key_column_type = sample_block.getByName(it->first).type;
|
||||
out_key_column_type = sample_block.getByName(name).type;
|
||||
return true;
|
||||
}
|
||||
|
||||
/** The case of space-filling curves.
|
||||
* When the node is not a key column (e.g. mortonEncode(x, y))
|
||||
* but one of the arguments of a key column (e.g. x or y).
|
||||
*
|
||||
* For example, the table has ORDER BY mortonEncode(x, y)
|
||||
* and query has WHERE x >= 10 AND x < 15 AND y > 20 AND y <= 25
|
||||
*/
|
||||
for (const auto & curve : key_space_filling_curves)
|
||||
{
|
||||
for (size_t i = 0, size = curve.arguments.size(); i < size; ++i)
|
||||
{
|
||||
if (curve.arguments[i] == name)
|
||||
{
|
||||
out_key_column_num = curve.key_column_pos;
|
||||
out_argument_num_of_space_filling_curve = i;
|
||||
out_key_column_type = sample_block.getByName(name).type;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (node.isFunction())
|
||||
{
|
||||
auto function_node = node.toFunctionNode();
|
||||
@ -1511,16 +1584,31 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
||||
{
|
||||
if (function_node.getArgumentAt(0).isConstant())
|
||||
{
|
||||
result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(1), out_key_column_num, out_key_column_type, out_functions_chain);
|
||||
result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
||||
function_node.getArgumentAt(1),
|
||||
out_key_column_num,
|
||||
out_argument_num_of_space_filling_curve,
|
||||
out_key_column_type,
|
||||
out_functions_chain);
|
||||
}
|
||||
else if (function_node.getArgumentAt(1).isConstant())
|
||||
{
|
||||
result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain);
|
||||
result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
||||
function_node.getArgumentAt(0),
|
||||
out_key_column_num,
|
||||
out_argument_num_of_space_filling_curve,
|
||||
out_key_column_type,
|
||||
out_functions_chain);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain);
|
||||
result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
||||
function_node.getArgumentAt(0),
|
||||
out_key_column_num,
|
||||
out_argument_num_of_space_filling_curve,
|
||||
out_key_column_type,
|
||||
out_functions_chain);
|
||||
}
|
||||
|
||||
return result;
|
||||
@ -1548,7 +1636,12 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
|
||||
{
|
||||
/** Functions < > = != <= >= in `notIn` isNull isNotNull, where one argument is a constant, and the other is one of columns of key,
|
||||
* or itself, wrapped in a chain of possibly-monotonic functions,
|
||||
* or constant expression - number.
|
||||
* (for example, if the table has ORDER BY time, we will check the conditions like
|
||||
* toDate(time) = '2023-10-14', toMonth(time) = 12, etc)
|
||||
* or any of arguments of a space-filling curve function if it is in the key,
|
||||
* (for example, if the table has ORDER BY mortonEncode(x, y), we will check the conditions like x > c, y <= c, etc.)
|
||||
* or constant expression - number
|
||||
* (for example x AND 0)
|
||||
*/
|
||||
Field const_value;
|
||||
DataTypePtr const_type;
|
||||
@ -1557,8 +1650,15 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
|
||||
auto func = node.toFunctionNode();
|
||||
size_t num_args = func.getArgumentsSize();
|
||||
|
||||
DataTypePtr key_expr_type; /// Type of expression containing key column
|
||||
size_t key_column_num = -1; /// Number of a key column (inside key_column_names array)
|
||||
/// Type of expression containing key column
|
||||
DataTypePtr key_expr_type;
|
||||
|
||||
/// Number of a key column (inside key_column_names array)
|
||||
size_t key_column_num = -1;
|
||||
|
||||
/// For example, if the key is mortonEncode(x, y), and the atom is x, then the argument num is 0.
|
||||
std::optional<size_t> argument_num_of_space_filling_curve;
|
||||
|
||||
MonotonicFunctionsChain chain;
|
||||
std::string func_name = func.getFunctionName();
|
||||
|
||||
@ -1567,7 +1667,8 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
|
||||
|
||||
if (num_args == 1)
|
||||
{
|
||||
if (!(isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, chain)))
|
||||
if (!(isKeyPossiblyWrappedByMonotonicFunctions(
|
||||
func.getArgumentAt(0), key_column_num, argument_num_of_space_filling_curve, key_expr_type, chain)))
|
||||
return false;
|
||||
|
||||
if (key_column_num == static_cast<size_t>(-1))
|
||||
@ -1615,7 +1716,8 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
|
||||
return true;
|
||||
}
|
||||
|
||||
if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, chain))
|
||||
if (isKeyPossiblyWrappedByMonotonicFunctions(
|
||||
func.getArgumentAt(0), key_column_num, argument_num_of_space_filling_curve, key_expr_type, chain))
|
||||
{
|
||||
key_arg_pos = 0;
|
||||
}
|
||||
@ -1645,7 +1747,8 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
|
||||
return true;
|
||||
}
|
||||
|
||||
if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(1), key_column_num, key_expr_type, chain))
|
||||
if (isKeyPossiblyWrappedByMonotonicFunctions(
|
||||
func.getArgumentAt(1), key_column_num, argument_num_of_space_filling_curve, key_expr_type, chain))
|
||||
{
|
||||
key_arg_pos = 1;
|
||||
}
|
||||
@ -1767,6 +1870,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
|
||||
|
||||
out.key_column = key_column_num;
|
||||
out.monotonic_functions_chain = std::move(chain);
|
||||
out.argument_num_of_space_filling_curve = argument_num_of_space_filling_curve;
|
||||
|
||||
return atom_it->second(out, const_value);
|
||||
}
|
||||
@ -1793,6 +1897,95 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
void KeyCondition::findHyperrectanglesForArgumentsOfSpaceFillingCurves()
|
||||
{
|
||||
/// Traverse chains of AND with conditions on arguments of a space filling curve, and construct hyperrectangles from them.
|
||||
/// For example, a chain:
|
||||
/// x >= 10 AND x <= 20 AND y >= 20 AND y <= 30
|
||||
/// will be transformed to a single atom:
|
||||
/// args in [10, 20] × [20, 30]
|
||||
|
||||
RPN new_rpn;
|
||||
new_rpn.reserve(rpn.size());
|
||||
|
||||
auto num_arguments_of_a_curve = [&](size_t key_column_pos)
|
||||
{
|
||||
for (const auto & curve : key_space_filling_curves)
|
||||
if (curve.key_column_pos == key_column_pos)
|
||||
return curve.arguments.size();
|
||||
return 0uz;
|
||||
};
|
||||
|
||||
for (const auto & elem : rpn)
|
||||
{
|
||||
if (elem.function == RPNElement::FUNCTION_IN_RANGE && elem.argument_num_of_space_filling_curve.has_value())
|
||||
{
|
||||
/// A range of an argument of a space-filling curve
|
||||
|
||||
size_t arg_num = *elem.argument_num_of_space_filling_curve;
|
||||
size_t curve_total_args = num_arguments_of_a_curve(elem.key_column);
|
||||
|
||||
if (!curve_total_args)
|
||||
{
|
||||
/// If we didn't find a space-filling curve - replace the condition to unknown.
|
||||
new_rpn.emplace_back();
|
||||
continue;
|
||||
}
|
||||
|
||||
chassert(arg_num < curve_total_args);
|
||||
|
||||
/// Replace the condition to a hyperrectangle
|
||||
|
||||
Hyperrectangle hyperrectangle(curve_total_args, Range::createWholeUniverseWithoutNull());
|
||||
hyperrectangle[arg_num] = elem.range;
|
||||
|
||||
RPNElement collapsed_elem;
|
||||
collapsed_elem.function = RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE;
|
||||
collapsed_elem.key_column = elem.key_column;
|
||||
collapsed_elem.space_filling_curve_args_hyperrectangle = std::move(hyperrectangle);
|
||||
|
||||
new_rpn.push_back(std::move(collapsed_elem));
|
||||
continue;
|
||||
}
|
||||
else if (elem.function == RPNElement::FUNCTION_AND && new_rpn.size() >= 2)
|
||||
{
|
||||
/// AND of two conditions
|
||||
|
||||
const auto & cond1 = new_rpn[new_rpn.size() - 2];
|
||||
const auto & cond2 = new_rpn[new_rpn.size() - 1];
|
||||
|
||||
/// Related to the same column of the key, represented by a space-filling curve
|
||||
|
||||
if (cond1.key_column == cond2.key_column
|
||||
&& cond1.function == RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE
|
||||
&& cond2.function == RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE)
|
||||
{
|
||||
/// Intersect these two conditions (applying AND)
|
||||
|
||||
RPNElement collapsed_elem;
|
||||
collapsed_elem.function = RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE;
|
||||
collapsed_elem.key_column = cond1.key_column;
|
||||
collapsed_elem.space_filling_curve_args_hyperrectangle = intersect(
|
||||
cond1.space_filling_curve_args_hyperrectangle,
|
||||
cond2.space_filling_curve_args_hyperrectangle);
|
||||
|
||||
/// Replace the AND operation with its arguments to the collapsed condition
|
||||
|
||||
new_rpn.pop_back();
|
||||
new_rpn.pop_back();
|
||||
new_rpn.push_back(std::move(collapsed_elem));
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
new_rpn.push_back(elem);
|
||||
}
|
||||
|
||||
rpn = std::move(new_rpn);
|
||||
}
|
||||
|
||||
|
||||
String KeyCondition::toString() const
|
||||
{
|
||||
String res;
|
||||
@ -1912,7 +2105,8 @@ KeyCondition::Description KeyCondition::getDescription() const
|
||||
|| element.function == RPNElement::FUNCTION_IS_NULL
|
||||
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL
|
||||
|| element.function == RPNElement::FUNCTION_IN_SET
|
||||
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
|
||||
|| element.function == RPNElement::FUNCTION_NOT_IN_SET
|
||||
|| element.function == RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE)
|
||||
{
|
||||
auto can_be_true = std::make_unique<Node>(Node{.type = Node::Type::Leaf, .element = &element, .negate = false});
|
||||
auto can_be_false = std::make_unique<Node>(Node{.type = Node::Type::Leaf, .element = &element, .negate = true});
|
||||
@ -1974,9 +2168,9 @@ KeyCondition::Description KeyCondition::getDescription() const
|
||||
}
|
||||
|
||||
if (rpn_stack.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::checkInRange");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::getDescription");
|
||||
|
||||
std::vector<std::string_view> key_names(key_columns.size());
|
||||
std::vector<String> key_names(key_columns.size());
|
||||
std::vector<bool> is_key_used(key_columns.size(), false);
|
||||
|
||||
for (const auto & key : key_columns)
|
||||
@ -2061,13 +2255,13 @@ KeyCondition::Description KeyCondition::getDescription() const
|
||||
*
|
||||
* The range of tuples can always be represented as a combination (union) of hyperrectangles.
|
||||
* For example, the range [ x1 y1 .. x2 y2 ] given x1 != x2 is equal to the union of the following three hyperrectangles:
|
||||
* [x1] x [y1 .. +inf)
|
||||
* (x1 .. x2) x (-inf .. +inf)
|
||||
* [x2] x (-inf .. y2]
|
||||
* [x1] × [y1 .. +inf)
|
||||
* (x1 .. x2) × (-inf .. +inf)
|
||||
* [x2] × (-inf .. y2]
|
||||
*
|
||||
* Or, for example, the range [ x1 y1 .. +inf ] is equal to the union of the following two hyperrectangles:
|
||||
* [x1] x [y1 .. +inf)
|
||||
* (x1 .. +inf) x (-inf .. +inf)
|
||||
* [x1] × [y1 .. +inf)
|
||||
* (x1 .. +inf) × (-inf .. +inf)
|
||||
* It's easy to see that this is a special case of the variant above.
|
||||
*
|
||||
* This is important because it is easy for us to check the feasibility of the condition over the hyperrectangle,
|
||||
@ -2075,6 +2269,10 @@ KeyCondition::Description KeyCondition::getDescription() const
|
||||
* over at least one hyperrectangle from which this range consists.
|
||||
*/
|
||||
|
||||
/** For the range between tuples, determined by left_keys, left_bounded, right_keys, right_bounded,
|
||||
* invoke the callback on every parallelogram composing this range (see the description above),
|
||||
* and returns the OR of the callback results (meaning if callback returned true on any part of the range).
|
||||
*/
|
||||
template <typename F>
|
||||
static BoolMask forAnyHyperrectangle(
|
||||
size_t key_size,
|
||||
@ -2082,7 +2280,7 @@ static BoolMask forAnyHyperrectangle(
|
||||
const FieldRef * right_keys,
|
||||
bool left_bounded,
|
||||
bool right_bounded,
|
||||
Hyperrectangle & hyperrectangle,
|
||||
Hyperrectangle & hyperrectangle, /// This argument is modified in-place for the callback
|
||||
const DataTypes & data_types,
|
||||
size_t prefix_size,
|
||||
BoolMask initial_mask,
|
||||
@ -2122,7 +2320,7 @@ static BoolMask forAnyHyperrectangle(
|
||||
return callback(hyperrectangle);
|
||||
}
|
||||
|
||||
/// (x1 .. x2) x (-inf .. +inf)
|
||||
/// (x1 .. x2) × (-inf .. +inf)
|
||||
|
||||
if (left_bounded && right_bounded)
|
||||
hyperrectangle[prefix_size] = Range(left_keys[prefix_size], false, right_keys[prefix_size], false);
|
||||
@ -2139,7 +2337,6 @@ static BoolMask forAnyHyperrectangle(
|
||||
hyperrectangle[i] = Range::createWholeUniverseWithoutNull();
|
||||
}
|
||||
|
||||
|
||||
BoolMask result = initial_mask;
|
||||
result = result | callback(hyperrectangle);
|
||||
|
||||
@ -2150,26 +2347,26 @@ static BoolMask forAnyHyperrectangle(
|
||||
if (result.isComplete())
|
||||
return result;
|
||||
|
||||
/// [x1] x [y1 .. +inf)
|
||||
/// [x1] × [y1 .. +inf)
|
||||
|
||||
if (left_bounded)
|
||||
{
|
||||
hyperrectangle[prefix_size] = Range(left_keys[prefix_size]);
|
||||
result = result
|
||||
| forAnyHyperrectangle(
|
||||
key_size, left_keys, right_keys, true, false, hyperrectangle, data_types, prefix_size + 1, initial_mask, callback);
|
||||
key_size, left_keys, right_keys, true, false, hyperrectangle, data_types, prefix_size + 1, initial_mask, callback);
|
||||
if (result.isComplete())
|
||||
return result;
|
||||
}
|
||||
|
||||
/// [x2] x (-inf .. y2]
|
||||
/// [x2] × (-inf .. y2]
|
||||
|
||||
if (right_bounded)
|
||||
{
|
||||
hyperrectangle[prefix_size] = Range(right_keys[prefix_size]);
|
||||
result = result
|
||||
| forAnyHyperrectangle(
|
||||
key_size, left_keys, right_keys, false, true, hyperrectangle, data_types, prefix_size + 1, initial_mask, callback);
|
||||
key_size, left_keys, right_keys, false, true, hyperrectangle, data_types, prefix_size + 1, initial_mask, callback);
|
||||
if (result.isComplete())
|
||||
return result;
|
||||
}
|
||||
@ -2196,14 +2393,14 @@ BoolMask KeyCondition::checkInRange(
|
||||
key_ranges.push_back(Range::createWholeUniverseWithoutNull());
|
||||
}
|
||||
|
||||
// std::cerr << "Checking for: [";
|
||||
// for (size_t i = 0; i != used_key_size; ++i)
|
||||
// std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), left_keys[i]);
|
||||
// std::cerr << " ... ";
|
||||
/* std::cerr << "Checking for: [";
|
||||
for (size_t i = 0; i != used_key_size; ++i)
|
||||
std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), left_keys[i]);
|
||||
std::cerr << " ... ";
|
||||
|
||||
// for (size_t i = 0; i != used_key_size; ++i)
|
||||
// std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), right_keys[i]);
|
||||
// std::cerr << "]\n";
|
||||
for (size_t i = 0; i != used_key_size; ++i)
|
||||
std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), right_keys[i]);
|
||||
std::cerr << "]\n";*/
|
||||
|
||||
return forAnyHyperrectangle(used_key_size, left_keys, right_keys, true, true, key_ranges, data_types, 0, initial_mask,
|
||||
[&] (const Hyperrectangle & key_ranges_hyperrectangle)
|
||||
@ -2212,7 +2409,7 @@ BoolMask KeyCondition::checkInRange(
|
||||
|
||||
// std::cerr << "Hyperrectangle: ";
|
||||
// for (size_t i = 0, size = key_ranges.size(); i != size; ++i)
|
||||
// std::cerr << (i != 0 ? " x " : "") << key_ranges[i].toString();
|
||||
// std::cerr << (i != 0 ? " × " : "") << key_ranges[i].toString();
|
||||
// std::cerr << ": " << res.can_be_true << "\n";
|
||||
|
||||
return res;
|
||||
@ -2347,7 +2544,13 @@ BoolMask KeyCondition::checkInHyperrectangle(
|
||||
std::vector<BoolMask> rpn_stack;
|
||||
for (const auto & element : rpn)
|
||||
{
|
||||
if (element.function == RPNElement::FUNCTION_UNKNOWN)
|
||||
if (element.argument_num_of_space_filling_curve.has_value())
|
||||
{
|
||||
/// If a condition on argument of a space filling curve wasn't collapsed into FUNCTION_ARGS_IN_HYPERRECTANGLE,
|
||||
/// we cannot process it.
|
||||
rpn_stack.emplace_back(true, true);
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_UNKNOWN)
|
||||
{
|
||||
rpn_stack.emplace_back(true, true);
|
||||
}
|
||||
@ -2383,6 +2586,97 @@ BoolMask KeyCondition::checkInHyperrectangle(
|
||||
if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
|
||||
rpn_stack.back() = !rpn_stack.back();
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE)
|
||||
{
|
||||
/** The case of space-filling curves.
|
||||
* We unpack the range of a space filling curve into hyperrectangles of their arguments,
|
||||
* and then check the intersection of them with the given hyperrectangle from the key condition.
|
||||
*
|
||||
* Note: you might find this code hard to understand,
|
||||
* because there are three different hyperrectangles involved:
|
||||
*
|
||||
* 1. A hyperrectangle derived from the range of the table's sparse index (marks granule): `hyperrectangle`
|
||||
* We analyze its dimension `key_range`, corresponding to the `key_column`.
|
||||
* For example, the table's key is a single column `mortonEncode(x, y)`,
|
||||
* the current granule is [500, 600], and it means that
|
||||
* mortonEncode(x, y) in [500, 600]
|
||||
*
|
||||
* 2. A hyperrectangle derived from the key condition, e.g.
|
||||
* `x >= 10 AND x <= 20 AND y >= 20 AND y <= 30` defines: (x, y) in [10, 20] × [20, 30]
|
||||
*
|
||||
* 3. A set of hyperrectangles that we obtain by inverting the space-filling curve on the range:
|
||||
* From mortonEncode(x, y) in [500, 600]
|
||||
* We get (x, y) in [30, 31] × [12, 13]
|
||||
* or (x, y) in [28, 31] × [14, 15];
|
||||
* or (x, y) in [0, 7] × [16, 23];
|
||||
* or (x, y) in [8, 11] × [16, 19];
|
||||
* or (x, y) in [12, 15] × [16, 17];
|
||||
* or (x, y) in [12, 12] × [18, 18];
|
||||
*
|
||||
* And we analyze the intersection of (2) and (3).
|
||||
*/
|
||||
|
||||
Range key_range = hyperrectangle[element.key_column];
|
||||
|
||||
/// The only possible result type of a space filling curve is UInt64.
|
||||
/// We also only check bounded ranges.
|
||||
if (key_range.left.getType() == Field::Types::UInt64
|
||||
&& key_range.right.getType() == Field::Types::UInt64)
|
||||
{
|
||||
key_range.shrinkToIncludedIfPossible();
|
||||
|
||||
size_t num_dimensions = element.space_filling_curve_args_hyperrectangle.size();
|
||||
|
||||
/// Let's support only the case of 2d, because I'm not confident in other cases.
|
||||
if (num_dimensions == 2)
|
||||
{
|
||||
UInt64 left = key_range.left.get<UInt64>();
|
||||
UInt64 right = key_range.right.get<UInt64>();
|
||||
|
||||
BoolMask mask(false, true);
|
||||
mortonIntervalToHyperrectangles<2>(left, right,
|
||||
[&](std::array<std::pair<UInt64, UInt64>, 2> morton_hyperrectangle)
|
||||
{
|
||||
BoolMask current_intersection(true, false);
|
||||
for (size_t dim = 0; dim < num_dimensions; ++dim)
|
||||
{
|
||||
const Range & condition_arg_range = element.space_filling_curve_args_hyperrectangle[dim];
|
||||
|
||||
const Range morton_arg_range(
|
||||
morton_hyperrectangle[dim].first, true,
|
||||
morton_hyperrectangle[dim].second, true);
|
||||
|
||||
bool intersects = condition_arg_range.intersectsRange(morton_arg_range);
|
||||
bool contains = condition_arg_range.containsRange(morton_arg_range);
|
||||
|
||||
current_intersection = current_intersection & BoolMask(intersects, !contains);
|
||||
}
|
||||
|
||||
mask = mask | current_intersection;
|
||||
});
|
||||
|
||||
rpn_stack.emplace_back(mask);
|
||||
}
|
||||
else
|
||||
rpn_stack.emplace_back(true, true);
|
||||
}
|
||||
else
|
||||
rpn_stack.emplace_back(true, true);
|
||||
|
||||
/** Note: we can consider implementing a simpler solution, based on "hidden keys".
|
||||
* It means, when we have a table's key like (a, b, mortonCurve(x, y))
|
||||
* we extract the arguments from the curves, and append them to the key,
|
||||
* imagining that we have the key (a, b, mortonCurve(x, y), x, y)
|
||||
*
|
||||
* Then while we analyze the granule's range between (a, b, mortonCurve(x, y))
|
||||
* and decompose it to the series of hyperrectangles,
|
||||
* we can construct a series of hyperrectangles of the extended key (a, b, mortonCurve(x, y), x, y),
|
||||
* and then do everything as usual.
|
||||
*
|
||||
* This approach is generalizable to any functions, that have preimage of interval
|
||||
* represented by a set of hyperrectangles.
|
||||
*/
|
||||
}
|
||||
else if (
|
||||
element.function == RPNElement::FUNCTION_IS_NULL
|
||||
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL)
|
||||
@ -2445,7 +2739,7 @@ BoolMask KeyCondition::checkInHyperrectangle(
|
||||
}
|
||||
|
||||
if (rpn_stack.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::checkInRange");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::checkInHyperrectangle");
|
||||
|
||||
return rpn_stack[0];
|
||||
}
|
||||
@ -2459,11 +2753,17 @@ bool KeyCondition::mayBeTrueInRange(
|
||||
return checkInRange(used_key_size, left_keys, right_keys, data_types, BoolMask::consider_only_can_be_true).can_be_true;
|
||||
}
|
||||
|
||||
String KeyCondition::RPNElement::toString() const { return toString("column " + std::to_string(key_column), false); }
|
||||
String KeyCondition::RPNElement::toString() const
|
||||
{
|
||||
if (argument_num_of_space_filling_curve)
|
||||
return toString(fmt::format("argument {} of column {}", *argument_num_of_space_filling_curve, key_column), false);
|
||||
else
|
||||
return toString(fmt::format("column {}", key_column), false);
|
||||
}
|
||||
|
||||
String KeyCondition::RPNElement::toString(std::string_view column_name, bool print_constants) const
|
||||
{
|
||||
auto print_wrapped_column = [this, &column_name, print_constants](WriteBuffer & buf)
|
||||
auto print_wrapped_column = [this, column_name, print_constants](WriteBuffer & buf)
|
||||
{
|
||||
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
|
||||
{
|
||||
@ -2527,6 +2827,15 @@ String KeyCondition::RPNElement::toString(std::string_view column_name, bool pri
|
||||
buf << ")";
|
||||
return buf.str();
|
||||
}
|
||||
case FUNCTION_ARGS_IN_HYPERRECTANGLE:
|
||||
{
|
||||
buf << "(";
|
||||
print_wrapped_column(buf);
|
||||
buf << " has args in ";
|
||||
buf << DB::toString(space_filling_curve_args_hyperrectangle);
|
||||
buf << ")";
|
||||
return buf.str();
|
||||
}
|
||||
case FUNCTION_IS_NULL:
|
||||
case FUNCTION_IS_NOT_NULL:
|
||||
{
|
||||
@ -2579,6 +2888,7 @@ bool KeyCondition::unknownOrAlwaysTrue(bool unknown_any) const
|
||||
|| element.function == RPNElement::FUNCTION_IN_RANGE
|
||||
|| element.function == RPNElement::FUNCTION_IN_SET
|
||||
|| element.function == RPNElement::FUNCTION_NOT_IN_SET
|
||||
|| element.function == RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE
|
||||
|| element.function == RPNElement::FUNCTION_IS_NULL
|
||||
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL
|
||||
|| element.function == RPNElement::ALWAYS_FALSE)
|
||||
@ -2635,6 +2945,7 @@ bool KeyCondition::alwaysFalse() const
|
||||
|| element.function == RPNElement::FUNCTION_IN_RANGE
|
||||
|| element.function == RPNElement::FUNCTION_IN_SET
|
||||
|| element.function == RPNElement::FUNCTION_NOT_IN_SET
|
||||
|| element.function == RPNElement::FUNCTION_ARGS_IN_HYPERRECTANGLE
|
||||
|| element.function == RPNElement::FUNCTION_IS_NULL
|
||||
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL
|
||||
|| element.function == RPNElement::FUNCTION_UNKNOWN)
|
||||
|
@ -174,7 +174,15 @@ public:
|
||||
FUNCTION_NOT_IN_SET,
|
||||
FUNCTION_IS_NULL,
|
||||
FUNCTION_IS_NOT_NULL,
|
||||
FUNCTION_UNKNOWN, /// Can take any value.
|
||||
/// Special for space-filling curves.
|
||||
/// For example, if key is mortonEncode(x, y),
|
||||
/// and the condition contains its arguments, e.g.:
|
||||
/// x >= 10 AND x <= 20 AND y >= 20 AND y <= 30,
|
||||
/// this expression will be analyzed and then represented by following:
|
||||
/// args in hyperrectangle [10, 20] × [20, 30].
|
||||
FUNCTION_ARGS_IN_HYPERRECTANGLE,
|
||||
/// Can take any value.
|
||||
FUNCTION_UNKNOWN,
|
||||
/// Operators of the logical expression.
|
||||
FUNCTION_NOT,
|
||||
FUNCTION_AND,
|
||||
@ -198,10 +206,19 @@ public:
|
||||
/// For FUNCTION_IN_RANGE and FUNCTION_NOT_IN_RANGE.
|
||||
Range range = Range::createWholeUniverse();
|
||||
size_t key_column = 0;
|
||||
|
||||
/// If the key_column is a space filling curve, e.g. mortonEncode(x, y),
|
||||
/// we will analyze expressions of its arguments (x and y) similarly how we do for a normal key columns,
|
||||
/// and this designates the argument number (0 for x, 1 for y):
|
||||
std::optional<size_t> argument_num_of_space_filling_curve;
|
||||
|
||||
/// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET
|
||||
using MergeTreeSetIndexPtr = std::shared_ptr<const MergeTreeSetIndex>;
|
||||
MergeTreeSetIndexPtr set_index;
|
||||
|
||||
/// For FUNCTION_ARGS_IN_HYPERRECTANGLE
|
||||
Hyperrectangle space_filling_curve_args_hyperrectangle;
|
||||
|
||||
MonotonicFunctionsChain monotonic_functions_chain;
|
||||
};
|
||||
|
||||
@ -225,21 +242,25 @@ private:
|
||||
|
||||
bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out);
|
||||
|
||||
/** Is node the key column
|
||||
* or expression in which column of key is wrapped by chain of functions,
|
||||
/** Is node the key column, or an argument of a space-filling curve that is a key column,
|
||||
* or expression in which that column is wrapped by a chain of functions,
|
||||
* that can be monotonic on certain ranges?
|
||||
* If these conditions are true, then returns number of column in key, type of resulting expression
|
||||
* If these conditions are true, then returns number of column in key,
|
||||
* optionally the argument position of a space-filling curve,
|
||||
* type of resulting expression
|
||||
* and fills chain of possibly-monotonic functions.
|
||||
*/
|
||||
bool isKeyPossiblyWrappedByMonotonicFunctions(
|
||||
const RPNBuilderTreeNode & node,
|
||||
size_t & out_key_column_num,
|
||||
std::optional<size_t> & out_argument_num_of_space_filling_curve,
|
||||
DataTypePtr & out_key_res_column_type,
|
||||
MonotonicFunctionsChain & out_functions_chain);
|
||||
|
||||
bool isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
||||
const RPNBuilderTreeNode & node,
|
||||
size_t & out_key_column_num,
|
||||
std::optional<size_t> & out_argument_num_of_space_filling_curve,
|
||||
DataTypePtr & out_key_column_type,
|
||||
std::vector<RPNBuilderFunctionTreeNode> & out_functions_chain);
|
||||
|
||||
@ -298,6 +319,11 @@ private:
|
||||
/// and all, two, partitions will be scanned, but due to filtering later none of rows will be matched.
|
||||
bool unknownOrAlwaysTrue(bool unknown_any) const;
|
||||
|
||||
/** Iterates over RPN and collapses FUNCTION_IN_RANGE over the arguments of space-filling curve function
|
||||
* into atom of type FUNCTION_ARGS_IN_HYPERRECTANGLE.
|
||||
*/
|
||||
void findHyperrectanglesForArgumentsOfSpaceFillingCurves();
|
||||
|
||||
RPN rpn;
|
||||
|
||||
ColumnIndices key_columns;
|
||||
@ -308,6 +334,17 @@ private:
|
||||
/// All intermediate columns are used to calculate key_expr.
|
||||
const NameSet key_subexpr_names;
|
||||
|
||||
/// Space-filling curves in the key
|
||||
struct SpaceFillingCurveDescription
|
||||
{
|
||||
size_t key_column_pos;
|
||||
String function_name;
|
||||
std::vector<String> arguments;
|
||||
};
|
||||
using SpaceFillingCurveDescriptions = std::vector<SpaceFillingCurveDescription>;
|
||||
SpaceFillingCurveDescriptions key_space_filling_curves;
|
||||
void getAllSpaceFillingCurves();
|
||||
|
||||
/// Array joined column names
|
||||
NameSet array_joined_column_names;
|
||||
|
||||
|
@ -3930,7 +3930,9 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT
|
||||
|
||||
auto is_appropriate_state = [] (DataPartState state)
|
||||
{
|
||||
return state == DataPartState::Active || state == DataPartState::Outdated;
|
||||
if (state != DataPartState::Outdated)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to restore a part from unexpected state: {}", state);
|
||||
return true;
|
||||
};
|
||||
|
||||
auto activate_part = [this, &restored_active_part](auto it)
|
||||
|
@ -64,7 +64,7 @@ void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t
|
||||
return;
|
||||
|
||||
LOG_TRACE(log, "Enqueueing {} for check after after {}s", name, delay_to_check_seconds);
|
||||
parts_queue.emplace_back(name, time(nullptr) + delay_to_check_seconds);
|
||||
parts_queue.emplace_back(name, std::chrono::steady_clock::now() + std::chrono::seconds(delay_to_check_seconds));
|
||||
parts_set.insert(name);
|
||||
task->schedule();
|
||||
}
|
||||
@ -81,8 +81,8 @@ void ReplicatedMergeTreePartCheckThread::cancelRemovedPartsCheck(const MergeTree
|
||||
{
|
||||
std::lock_guard lock(parts_mutex);
|
||||
for (const auto & elem : parts_queue)
|
||||
if (drop_range_info.contains(MergeTreePartInfo::fromPartName(elem.first, storage.format_version)))
|
||||
parts_to_remove.push_back(elem.first);
|
||||
if (drop_range_info.contains(MergeTreePartInfo::fromPartName(elem.name, storage.format_version)))
|
||||
parts_to_remove.push_back(elem.name);
|
||||
}
|
||||
|
||||
/// We have to remove parts that were not removed by removePartAndEnqueueFetch
|
||||
@ -102,11 +102,11 @@ void ReplicatedMergeTreePartCheckThread::cancelRemovedPartsCheck(const MergeTree
|
||||
std::lock_guard lock(parts_mutex);
|
||||
for (const auto & elem : parts_queue)
|
||||
{
|
||||
bool is_removed = removed_parts.contains(elem.first);
|
||||
bool should_have_been_removed = drop_range_info.contains(MergeTreePartInfo::fromPartName(elem.first, storage.format_version));
|
||||
bool is_removed = removed_parts.contains(elem.name);
|
||||
bool should_have_been_removed = drop_range_info.contains(MergeTreePartInfo::fromPartName(elem.name, storage.format_version));
|
||||
if (is_removed != should_have_been_removed)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent parts_queue: name={}, is_removed={}, should_have_been_removed={}",
|
||||
elem.first, is_removed, should_have_been_removed);
|
||||
elem.name, is_removed, should_have_been_removed);
|
||||
count += is_removed;
|
||||
}
|
||||
|
||||
@ -116,7 +116,7 @@ void ReplicatedMergeTreePartCheckThread::cancelRemovedPartsCheck(const MergeTree
|
||||
|
||||
auto new_end = std::remove_if(parts_queue.begin(), parts_queue.end(), [&removed_parts] (const auto & elem)
|
||||
{
|
||||
return removed_parts.contains(elem.first);
|
||||
return removed_parts.contains(elem.name);
|
||||
});
|
||||
|
||||
parts_queue.erase(new_end, parts_queue.end());
|
||||
@ -293,10 +293,10 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St
|
||||
time_t lifetime = time(nullptr) - outdated->remove_time;
|
||||
time_t max_lifetime = storage.getSettings()->old_parts_lifetime.totalSeconds();
|
||||
time_t delay = lifetime >= max_lifetime ? 0 : max_lifetime - lifetime;
|
||||
result.recheck_after = delay + 30;
|
||||
result.recheck_after_seconds = delay + 30;
|
||||
|
||||
auto message = PreformattedMessage::create("Part {} is Outdated, will wait for cleanup thread to handle it "
|
||||
"and check again after {}s", part_name, result.recheck_after);
|
||||
"and check again after {}s", part_name, result.recheck_after_seconds);
|
||||
LOG_WARNING(log, message);
|
||||
result.status = {part_name, true, message.text};
|
||||
result.action = ReplicatedCheckResult::RecheckLater;
|
||||
@ -411,7 +411,7 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St
|
||||
auto message = PreformattedMessage::create("Young part {} with age {} seconds hasn't been added to ZooKeeper yet. It's ok.",
|
||||
part_name, (current_time - part->modification_time));
|
||||
LOG_INFO(log, message);
|
||||
result.recheck_after = part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER - current_time;
|
||||
result.recheck_after_seconds = part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER - current_time;
|
||||
result.status = {part_name, true, message};
|
||||
result.action = ReplicatedCheckResult::RecheckLater;
|
||||
return result;
|
||||
@ -436,9 +436,9 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPartAndFix(const String & p
|
||||
case ReplicatedCheckResult::RecheckLater:
|
||||
/// NOTE We cannot enqueue it from the check thread itself
|
||||
if (recheck_after)
|
||||
*recheck_after = result.recheck_after;
|
||||
*recheck_after = result.recheck_after_seconds;
|
||||
else
|
||||
enqueuePart(part_name, result.recheck_after);
|
||||
enqueuePart(part_name, result.recheck_after_seconds);
|
||||
break;
|
||||
|
||||
case ReplicatedCheckResult::DetachUnexpected:
|
||||
@ -539,7 +539,7 @@ void ReplicatedMergeTreePartCheckThread::run()
|
||||
|
||||
try
|
||||
{
|
||||
time_t current_time = time(nullptr);
|
||||
const auto current_time = std::chrono::steady_clock::now();
|
||||
|
||||
/// Take part from the queue for verification.
|
||||
PartsToCheckQueue::iterator selected = parts_queue.end(); /// end from std::list is not get invalidated
|
||||
@ -555,17 +555,29 @@ void ReplicatedMergeTreePartCheckThread::run()
|
||||
|
||||
selected = std::find_if(parts_queue.begin(), parts_queue.end(), [current_time](const auto & elem)
|
||||
{
|
||||
return elem.second <= current_time;
|
||||
return elem.time <= current_time;
|
||||
});
|
||||
if (selected == parts_queue.end())
|
||||
{
|
||||
// Find next part to check in the queue and schedule the check
|
||||
// Otherwise, scheduled for later checks won't be executed until
|
||||
// a new check is enqueued (i.e. task is scheduled again)
|
||||
auto next_it = std::min_element(
|
||||
begin(parts_queue), end(parts_queue), [](const auto & l, const auto & r) { return l.time < r.time; });
|
||||
if (next_it != parts_queue.end())
|
||||
{
|
||||
auto delay = next_it->time - current_time;
|
||||
task->scheduleAfter(duration_cast<std::chrono::milliseconds>(delay).count());
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
/// Move selected part to the end of the queue
|
||||
parts_queue.splice(parts_queue.end(), parts_queue, selected);
|
||||
}
|
||||
|
||||
std::optional<time_t> recheck_after;
|
||||
checkPartAndFix(selected->first, &recheck_after);
|
||||
checkPartAndFix(selected->name, &recheck_after);
|
||||
|
||||
if (need_stop)
|
||||
return;
|
||||
@ -580,12 +592,12 @@ void ReplicatedMergeTreePartCheckThread::run()
|
||||
}
|
||||
else if (recheck_after.has_value())
|
||||
{
|
||||
LOG_TRACE(log, "Will recheck part {} after after {}s", selected->first, *recheck_after);
|
||||
selected->second = time(nullptr) + *recheck_after;
|
||||
LOG_TRACE(log, "Will recheck part {} after after {}s", selected->name, *recheck_after);
|
||||
selected->time = std::chrono::steady_clock::now() + std::chrono::seconds(*recheck_after);
|
||||
}
|
||||
else
|
||||
{
|
||||
parts_set.erase(selected->first);
|
||||
parts_set.erase(selected->name);
|
||||
parts_queue.erase(selected);
|
||||
}
|
||||
}
|
||||
|
@ -37,7 +37,7 @@ struct ReplicatedCheckResult
|
||||
|
||||
bool exists_in_zookeeper;
|
||||
MergeTreeDataPartPtr part;
|
||||
time_t recheck_after = 0;
|
||||
time_t recheck_after_seconds = 0;
|
||||
};
|
||||
|
||||
/** Checks the integrity of the parts requested for validation.
|
||||
@ -90,7 +90,12 @@ private:
|
||||
Poco::Logger * log;
|
||||
|
||||
using StringSet = std::set<String>;
|
||||
using PartToCheck = std::pair<String, time_t>; /// The name of the part and the minimum time to check (or zero, if not important).
|
||||
struct PartToCheck
|
||||
{
|
||||
using TimePoint = std::chrono::steady_clock::time_point;
|
||||
String name;
|
||||
TimePoint time;
|
||||
};
|
||||
using PartsToCheckQueue = std::list<PartToCheck>;
|
||||
|
||||
/** Parts for which you want to check one of two:
|
||||
|
@ -816,6 +816,10 @@ std::pair<std::vector<String>, bool> ReplicatedMergeTreeSinkImpl<async_insert>::
|
||||
NameSet unused;
|
||||
/// if we found part in deduplication hashes part must exists on some replica
|
||||
storage.checkPartChecksumsAndAddCommitOps(zookeeper, part, ops, existing_part_name, unused);
|
||||
|
||||
/// We have to check that the block_id still exists to avoid a race condition with DROP_RANGE
|
||||
block_unlock_op_idx = ops.size();
|
||||
ops.emplace_back(zkutil::makeCheckRequest(storage.zookeeper_path + "/blocks/" + block_id, -1));
|
||||
}
|
||||
catch (const zkutil::KeeperException &)
|
||||
{
|
||||
@ -924,8 +928,13 @@ std::pair<std::vector<String>, bool> ReplicatedMergeTreeSinkImpl<async_insert>::
|
||||
}
|
||||
else if (multi_code == Coordination::Error::ZNONODE && zkutil::getFailedOpIndex(multi_code, responses) == block_unlock_op_idx)
|
||||
{
|
||||
if (block_number_lock)
|
||||
throw Exception(ErrorCodes::QUERY_WAS_CANCELLED,
|
||||
"Insert query (for block {}) was canceled by concurrent ALTER PARTITION or TRUNCATE", block_number_lock->getPath());
|
||||
|
||||
chassert(!existing_part_name.empty());
|
||||
throw Exception(ErrorCodes::QUERY_WAS_CANCELLED,
|
||||
"Insert query (for block {}) was cancelled by concurrent ALTER PARTITION", block_number_lock->getPath());
|
||||
"Insert query (for existing part {}, deduplicated) was canceled by concurrent ALTER PARTITION, TRUNCATE, or the part became outdated", existing_part_name);
|
||||
}
|
||||
else if (Coordination::isHardwareError(multi_code))
|
||||
{
|
||||
|
@ -45,6 +45,7 @@ public:
|
||||
size_t queueSize() { return received.size(); }
|
||||
|
||||
auto getSubject() const { return current.subject; }
|
||||
const String & getCurrentMessage() const { return current.message; }
|
||||
|
||||
/// Return read buffer containing next available message
|
||||
/// or nullptr if there are no messages to process.
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -11,7 +12,6 @@ class ASTStorage;
|
||||
M(String, nats_url, "", "A host-port to connect to NATS server.", 0) \
|
||||
M(String, nats_subjects, "", "List of subject for NATS table to subscribe/publish to.", 0) \
|
||||
M(String, nats_format, "", "The message format.", 0) \
|
||||
M(Char, nats_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \
|
||||
M(String, nats_schema, "", "Schema identifier (used by schema-based formats) for NATS engine", 0) \
|
||||
M(UInt64, nats_num_consumers, 1, "The number of consumer channels per table.", 0) \
|
||||
M(String, nats_queue_group, "", "Name for queue group of NATS subscribers.", 0) \
|
||||
@ -27,10 +27,15 @@ class ASTStorage;
|
||||
M(String, nats_token, "", "NATS token", 0) \
|
||||
M(UInt64, nats_startup_connect_tries, 5, "Number of connect tries at startup", 0) \
|
||||
M(UInt64, nats_max_rows_per_message, 1, "The maximum number of rows produced in one message for row-based formats.", 0) \
|
||||
M(StreamingHandleErrorMode, nats_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for NATS 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_NATS_SETTINGS(M, ALIAS) \
|
||||
NATS_RELATED_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS)
|
||||
#define OBSOLETE_NATS_SETTINGS(M, ALIAS) \
|
||||
MAKE_OBSOLETE(M, Char, nats_row_delimiter, '\0') \
|
||||
|
||||
#define LIST_OF_NATS_SETTINGS(M, ALIAS) \
|
||||
NATS_RELATED_SETTINGS(M, ALIAS) \
|
||||
OBSOLETE_NATS_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(NATSSettingsTraits, LIST_OF_NATS_SETTINGS)
|
||||
|
||||
|
@ -9,10 +9,10 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static std::pair<Block, Block> getHeaders(const StorageSnapshotPtr & storage_snapshot)
|
||||
static std::pair<Block, Block> getHeaders(StorageNATS & storage, const StorageSnapshotPtr & storage_snapshot)
|
||||
{
|
||||
auto non_virtual_header = storage_snapshot->metadata->getSampleBlockNonMaterialized();
|
||||
auto virtual_header = storage_snapshot->getSampleBlockForColumns({"_subject"});
|
||||
auto virtual_header = storage_snapshot->getSampleBlockForColumns(storage.getVirtuals().getNames());
|
||||
|
||||
return {non_virtual_header, virtual_header};
|
||||
}
|
||||
@ -31,8 +31,9 @@ NATSSource::NATSSource(
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
ContextPtr context_,
|
||||
const Names & columns,
|
||||
size_t max_block_size_)
|
||||
: NATSSource(storage_, storage_snapshot_, getHeaders(storage_snapshot_), context_, columns, max_block_size_)
|
||||
size_t max_block_size_,
|
||||
StreamingHandleErrorMode handle_error_mode_)
|
||||
: NATSSource(storage_, storage_snapshot_, getHeaders(storage_, storage_snapshot_), context_, columns, max_block_size_, handle_error_mode_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -42,13 +43,15 @@ NATSSource::NATSSource(
|
||||
std::pair<Block, Block> headers,
|
||||
ContextPtr context_,
|
||||
const Names & columns,
|
||||
size_t max_block_size_)
|
||||
size_t max_block_size_,
|
||||
StreamingHandleErrorMode handle_error_mode_)
|
||||
: ISource(getSampleBlock(headers.first, headers.second))
|
||||
, storage(storage_)
|
||||
, storage_snapshot(storage_snapshot_)
|
||||
, context(context_)
|
||||
, column_names(columns)
|
||||
, max_block_size(max_block_size_)
|
||||
, handle_error_mode(handle_error_mode_)
|
||||
, non_virtual_header(std::move(headers.first))
|
||||
, virtual_header(std::move(headers.second))
|
||||
{
|
||||
@ -97,16 +100,41 @@ Chunk NATSSource::generate()
|
||||
EmptyReadBuffer empty_buf;
|
||||
auto input_format = FormatFactory::instance().getInput(
|
||||
storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1);
|
||||
|
||||
StreamingFormatExecutor executor(non_virtual_header, input_format);
|
||||
|
||||
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);
|
||||
|
||||
while (true)
|
||||
{
|
||||
if (consumer->queueEmpty())
|
||||
break;
|
||||
|
||||
exception_message.reset();
|
||||
size_t new_rows = 0;
|
||||
if (auto buf = consumer->consume())
|
||||
new_rows = executor.execute(*buf);
|
||||
@ -115,6 +143,21 @@ Chunk NATSSource::generate()
|
||||
{
|
||||
auto subject = consumer->getSubject();
|
||||
virtual_columns[0]->insertMany(subject, new_rows);
|
||||
if (handle_error_mode == StreamingHandleErrorMode::STREAM)
|
||||
{
|
||||
if (exception_message)
|
||||
{
|
||||
const auto & current_message = consumer->getCurrentMessage();
|
||||
virtual_columns[1]->insertData(current_message.data(), current_message.size());
|
||||
virtual_columns[2]->insertData(exception_message->data(), exception_message->size());
|
||||
|
||||
}
|
||||
else
|
||||
{
|
||||
virtual_columns[1]->insertDefault();
|
||||
virtual_columns[2]->insertDefault();
|
||||
}
|
||||
}
|
||||
|
||||
total_rows = total_rows + new_rows;
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user